All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCH 0/7 v3] xfs: log recovery fixes
@ 2022-03-15  6:42 Dave Chinner
  2022-03-15  6:42 ` [PATCH 1/7] xfs: log worker needs to start before intent/unlink recovery Dave Chinner
                   ` (6 more replies)
  0 siblings, 7 replies; 29+ messages in thread
From: Dave Chinner @ 2022-03-15  6:42 UTC (permalink / raw)
  To: linux-xfs

Willy reported generic/530 had started hanging on his test machines
and I've tried to reproduce the problem he reported. While I haven't
reproduced the exact hang he's been having, I've found a couple of
others while running g/530 in a tight loop on a couple of test
machines.

The first 3 patches are defensive fixes - the log worker acts as a
watchdog, and the issues in patch 2 and 3 were triggered on my
testing of g/530 and lead to 30s delays that the log worker watchdog
caught. Without the watchdog, these may actually be deadlock
triggers.

The 4th patch is the one that fixes the problem Willy reported.
It is a regression from conversion of the AIL pushing to use
non-blocking CIL flushes. It is unknown why this suddenly started
showing up on Willy's test machine right now, and why only on that
machine, but it is clearly a problem. This patch catches the state
that leads to the deadlock and breaks it with an immediate log
force to flush any pending iclogs.

In testing these patches, I found generic/388 was causing frequent
failures with recovery failing because inode clusters were being
found uninitialised in log recovery. That turns out to be a zero day
race condition in the forced shutdown code, and that is fixed over
the patches 5-7. In short, we can't abort writeback of log items
before we shut down the log (because that's separate to -mount-
shutdown) as removing aborted log items can move the tail of the log
forward and that can be propagated to the on-disk log and corrupt it
if timing is just right.

Fixing this takes failures of g/388 from 1 in 5-10 runs to 1 in 100
runs. There is a change in patch 7 that I mention "I'm not sure how
this can happen here, but it's handled elsewhere like this" that
avoids a double remove of an aborted inode from the AIL that results
in an ASSERT failure. I *think* I now know how that can occur, but
fixing it is another set of patches, and it may be a recent
regression rather than a long standing issue.

Version 3:
- added fixes for generic/388 failures.

Version 2:
- https://lore.kernel.org/linux-xfs/20220309015512.2648074-1-david@fromorbit.com/
- updated to 5.17-rc7
- tested by Willy.

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


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

* [PATCH 1/7] xfs: log worker needs to start before intent/unlink recovery
  2022-03-15  6:42 [PATCH 0/7 v3] xfs: log recovery fixes Dave Chinner
@ 2022-03-15  6:42 ` Dave Chinner
  2022-03-15  9:14   ` Chandan Babu R
  2022-03-15  6:42 ` [PATCH 2/7] xfs: check buffer pin state after locking in delwri_submit Dave Chinner
                   ` (5 subsequent siblings)
  6 siblings, 1 reply; 29+ messages in thread
From: Dave Chinner @ 2022-03-15  6:42 UTC (permalink / raw)
  To: linux-xfs

From: Dave Chinner <dchinner@redhat.com>

After 963 iterations of generic/530, it deadlocked during recovery
on a pinned inode cluster buffer like so:

XFS (pmem1): Starting recovery (logdev: internal)
INFO: task kworker/8:0:306037 blocked for more than 122 seconds.
      Not tainted 5.17.0-rc6-dgc+ #975
"echo 0 > /proc/sys/kernel/hung_task_timeout_secs" disables this message.
task:kworker/8:0     state:D stack:13024 pid:306037 ppid:     2 flags:0x00004000
Workqueue: xfs-inodegc/pmem1 xfs_inodegc_worker
Call Trace:
 <TASK>
 __schedule+0x30d/0x9e0
 schedule+0x55/0xd0
 schedule_timeout+0x114/0x160
 __down+0x99/0xf0
 down+0x5e/0x70
 xfs_buf_lock+0x36/0xf0
 xfs_buf_find+0x418/0x850
 xfs_buf_get_map+0x47/0x380
 xfs_buf_read_map+0x54/0x240
 xfs_trans_read_buf_map+0x1bd/0x490
 xfs_imap_to_bp+0x4f/0x70
 xfs_iunlink_map_ino+0x66/0xd0
 xfs_iunlink_map_prev.constprop.0+0x148/0x2f0
 xfs_iunlink_remove_inode+0xf2/0x1d0
 xfs_inactive_ifree+0x1a3/0x900
 xfs_inode_unlink+0xcc/0x210
 xfs_inodegc_worker+0x1ac/0x2f0
 process_one_work+0x1ac/0x390
 worker_thread+0x56/0x3c0
 kthread+0xf6/0x120
 ret_from_fork+0x1f/0x30
 </TASK>
task:mount           state:D stack:13248 pid:324509 ppid:324233 flags:0x00004000
Call Trace:
 <TASK>
 __schedule+0x30d/0x9e0
 schedule+0x55/0xd0
 schedule_timeout+0x114/0x160
 __down+0x99/0xf0
 down+0x5e/0x70
 xfs_buf_lock+0x36/0xf0
 xfs_buf_find+0x418/0x850
 xfs_buf_get_map+0x47/0x380
 xfs_buf_read_map+0x54/0x240
 xfs_trans_read_buf_map+0x1bd/0x490
 xfs_imap_to_bp+0x4f/0x70
 xfs_iget+0x300/0xb40
 xlog_recover_process_one_iunlink+0x4c/0x170
 xlog_recover_process_iunlinks.isra.0+0xee/0x130
 xlog_recover_finish+0x57/0x110
 xfs_log_mount_finish+0xfc/0x1e0
 xfs_mountfs+0x540/0x910
 xfs_fs_fill_super+0x495/0x850
 get_tree_bdev+0x171/0x270
 xfs_fs_get_tree+0x15/0x20
 vfs_get_tree+0x24/0xc0
 path_mount+0x304/0xba0
 __x64_sys_mount+0x108/0x140
 do_syscall_64+0x35/0x80
 entry_SYSCALL_64_after_hwframe+0x44/0xae
 </TASK>
task:xfsaild/pmem1   state:D stack:14544 pid:324525 ppid:     2 flags:0x00004000
Call Trace:
 <TASK>
 __schedule+0x30d/0x9e0
 schedule+0x55/0xd0
 io_schedule+0x4b/0x80
 xfs_buf_wait_unpin+0x9e/0xf0
 __xfs_buf_submit+0x14a/0x230
 xfs_buf_delwri_submit_buffers+0x107/0x280
 xfs_buf_delwri_submit_nowait+0x10/0x20
 xfsaild+0x27e/0x9d0
 kthread+0xf6/0x120
 ret_from_fork+0x1f/0x30

We have the mount process waiting on an inode cluster buffer read,
inodegc doing unlink waiting on the same inode cluster buffer, and
the AIL push thread blocked in writeback waiting for the inode to
become unpinned.

What has happened here is that the AIL push thread has raced with
the inodegc process modifying, committing and pinning the inode
cluster buffer here in xfs_buf_delwri_submit_buffers() here:

	blk_start_plug(&plug);
	list_for_each_entry_safe(bp, n, buffer_list, b_list) {
		if (!wait_list) {
			if (xfs_buf_ispinned(bp)) {
				pinned++;
				continue;
			}
Here >>>>>>
			if (!xfs_buf_trylock(bp))
				continue;

Basically, the AIL has found the buffer wasn't pinned and got the
lock without blocking, but then the buffer was pinned. This implies
the processing here was pre-empted between the pin check and the
lock, because the pin count can only be increased while holding the
buffer locked. Hence when it has gone to submit the IO, it has
blocked waiting for the buffer to be unpinned.

With all executing threads now waiting on the buffer to be unpinned,
we normally get out of situations like this via the background log
worker issuing a log force which will unpinned stuck buffers like
this. But at this point in recovery, we haven't started the log
worker. In fact, the first thing we do after processing intents and
unlinked inodes is *start the log worker*. IOWs, we start it too
late to have it break deadlocks like this.

Avoid this and any other similar deadlock vectors in intent and
unlinked inode recovery by starting the log worker before we recover
intents and unlinked inodes. This part of recovery runs as though
the filesystem is fully active, so we really should have the same
infrastructure running as we normally do at runtime.

Signed-off-by: Dave Chinner <dchinner@redhat.com>
Reviewed-by: Darrick J. Wong <djwong@kernel.org>
---
 fs/xfs/xfs_log.c | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)

diff --git a/fs/xfs/xfs_log.c b/fs/xfs/xfs_log.c
index 9a49acd94516..b0e05fa902d4 100644
--- a/fs/xfs/xfs_log.c
+++ b/fs/xfs/xfs_log.c
@@ -833,10 +833,9 @@ xfs_log_mount_finish(
 	 * mount failure occurs.
 	 */
 	mp->m_super->s_flags |= SB_ACTIVE;
+	xfs_log_work_queue(mp);
 	if (xlog_recovery_needed(log))
 		error = xlog_recover_finish(log);
-	if (!error)
-		xfs_log_work_queue(mp);
 	mp->m_super->s_flags &= ~SB_ACTIVE;
 	evict_inodes(mp->m_super);
 
-- 
2.35.1


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

* [PATCH 2/7] xfs: check buffer pin state after locking in delwri_submit
  2022-03-15  6:42 [PATCH 0/7 v3] xfs: log recovery fixes Dave Chinner
  2022-03-15  6:42 ` [PATCH 1/7] xfs: log worker needs to start before intent/unlink recovery Dave Chinner
@ 2022-03-15  6:42 ` Dave Chinner
  2022-03-15 10:04   ` Chandan Babu R
  2022-03-15 19:13   ` Darrick J. Wong
  2022-03-15  6:42 ` [PATCH 3/7] xfs: xfs_ail_push_all_sync() stalls when racing with updates Dave Chinner
                   ` (4 subsequent siblings)
  6 siblings, 2 replies; 29+ messages in thread
From: Dave Chinner @ 2022-03-15  6:42 UTC (permalink / raw)
  To: linux-xfs

From: Dave Chinner <dchinner@redhat.com>

AIL flushing can get stuck here:

[316649.005769] INFO: task xfsaild/pmem1:324525 blocked for more than 123 seconds.
[316649.007807]       Not tainted 5.17.0-rc6-dgc+ #975
[316649.009186] "echo 0 > /proc/sys/kernel/hung_task_timeout_secs" disables this message.
[316649.011720] task:xfsaild/pmem1   state:D stack:14544 pid:324525 ppid:     2 flags:0x00004000
[316649.014112] Call Trace:
[316649.014841]  <TASK>
[316649.015492]  __schedule+0x30d/0x9e0
[316649.017745]  schedule+0x55/0xd0
[316649.018681]  io_schedule+0x4b/0x80
[316649.019683]  xfs_buf_wait_unpin+0x9e/0xf0
[316649.021850]  __xfs_buf_submit+0x14a/0x230
[316649.023033]  xfs_buf_delwri_submit_buffers+0x107/0x280
[316649.024511]  xfs_buf_delwri_submit_nowait+0x10/0x20
[316649.025931]  xfsaild+0x27e/0x9d0
[316649.028283]  kthread+0xf6/0x120
[316649.030602]  ret_from_fork+0x1f/0x30

in the situation where flushing gets preempted between the unpin
check and the buffer trylock under nowait conditions:

	blk_start_plug(&plug);
	list_for_each_entry_safe(bp, n, buffer_list, b_list) {
		if (!wait_list) {
			if (xfs_buf_ispinned(bp)) {
				pinned++;
				continue;
			}
Here >>>>>>
			if (!xfs_buf_trylock(bp))
				continue;

This means submission is stuck until something else triggers a log
force to unpin the buffer.

To get onto the delwri list to begin with, the buffer pin state has
already been checked, and hence it's relatively rare we get a race
between flushing and encountering a pinned buffer in delwri
submission to begin with. Further, to increase the pin count the
buffer has to be locked, so the only way we can hit this race
without failing the trylock is to be preempted between the pincount
check seeing zero and the trylock being run.

Hence to avoid this problem, just invert the order of trylock vs
pin check. We shouldn't hit that many pinned buffers here, so
optimising away the trylock for pinned buffers should not matter for
performance at all.

Signed-off-by: Dave Chinner <dchinner@redhat.com>
---
 fs/xfs/xfs_buf.c | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)

diff --git a/fs/xfs/xfs_buf.c b/fs/xfs/xfs_buf.c
index b45e0d50a405..8867f143598e 100644
--- a/fs/xfs/xfs_buf.c
+++ b/fs/xfs/xfs_buf.c
@@ -2094,12 +2094,13 @@ xfs_buf_delwri_submit_buffers(
 	blk_start_plug(&plug);
 	list_for_each_entry_safe(bp, n, buffer_list, b_list) {
 		if (!wait_list) {
+			if (!xfs_buf_trylock(bp))
+				continue;
 			if (xfs_buf_ispinned(bp)) {
+				xfs_buf_unlock(bp);
 				pinned++;
 				continue;
 			}
-			if (!xfs_buf_trylock(bp))
-				continue;
 		} else {
 			xfs_buf_lock(bp);
 		}
-- 
2.35.1


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

* [PATCH 3/7] xfs: xfs_ail_push_all_sync() stalls when racing with updates
  2022-03-15  6:42 [PATCH 0/7 v3] xfs: log recovery fixes Dave Chinner
  2022-03-15  6:42 ` [PATCH 1/7] xfs: log worker needs to start before intent/unlink recovery Dave Chinner
  2022-03-15  6:42 ` [PATCH 2/7] xfs: check buffer pin state after locking in delwri_submit Dave Chinner
@ 2022-03-15  6:42 ` Dave Chinner
  2022-03-15 15:14   ` Chandan Babu R
  2022-03-15 19:17   ` Darrick J. Wong
  2022-03-15  6:42 ` [PATCH 4/7] xfs: async CIL flushes need pending pushes to be made stable Dave Chinner
                   ` (3 subsequent siblings)
  6 siblings, 2 replies; 29+ messages in thread
From: Dave Chinner @ 2022-03-15  6:42 UTC (permalink / raw)
  To: linux-xfs

From: Dave Chinner <dchinner@redhat.com>

xfs_ail_push_all_sync() has a loop like this:

while max_ail_lsn {
	prepare_to_wait(ail_empty)
	target = max_ail_lsn
	wake_up(ail_task);
	schedule()
}

Which is designed to sleep until the AIL is emptied. When
xfs_ail_finish_update() moves the tail of the log, it does:

	if (list_empty(&ailp->ail_head))
		wake_up_all(&ailp->ail_empty);

So it will only wake up the sync push waiter when the AIL goes
empty. If, by the time the push waiter has woken, the AIL has more
in it, it will reset the target, wake the push task and go back to
sleep.

The problem here is that if the AIL is having items added to it
when xfs_ail_push_all_sync() is called, then they may get inserted
into the AIL at a LSN higher than the target LSN. At this point,
xfsaild_push() will see that the target is X, the item LSNs are
(X+N) and skip over them, hence never pushing the out.

The result of this the AIL will not get emptied by the AIL push
thread, hence xfs_ail_finish_update() will never see the AIL being
empty even if it moves the tail. Hence xfs_ail_push_all_sync() never
gets woken and hence cannot update the push target to capture the
items beyond the current target on the LSN.

This is a TOCTOU type of issue so the way to avoid it is to not
use the push target at all for sync pushes. We know that a sync push
is being requested by the fact the ail_empty wait queue is active,
hence the xfsaild can just set the target to max_ail_lsn on every
push that we see the wait queue active. Hence we no longer will
leave items on the AIL that are beyond the LSN sampled at the start
of a sync push.

Signed-off-by: Dave Chinner <dchinner@redhat.com>
---
 fs/xfs/xfs_trans_ail.c | 21 ++++++++++++++++-----
 1 file changed, 16 insertions(+), 5 deletions(-)

diff --git a/fs/xfs/xfs_trans_ail.c b/fs/xfs/xfs_trans_ail.c
index 2a8c8dc54c95..1b52952097c1 100644
--- a/fs/xfs/xfs_trans_ail.c
+++ b/fs/xfs/xfs_trans_ail.c
@@ -448,10 +448,22 @@ xfsaild_push(
 
 	spin_lock(&ailp->ail_lock);
 
-	/* barrier matches the ail_target update in xfs_ail_push() */
-	smp_rmb();
-	target = ailp->ail_target;
-	ailp->ail_target_prev = target;
+	/*
+	 * If we have a sync push waiter, we always have to push till the AIL is
+	 * empty. Update the target to point to the end of the AIL so that
+	 * capture updates that occur after the sync push waiter has gone to
+	 * sleep.
+	 */
+	if (waitqueue_active(&ailp->ail_empty)) {
+		lip = xfs_ail_max(ailp);
+		if (lip)
+			target = lip->li_lsn;
+	} else {
+		/* barrier matches the ail_target update in xfs_ail_push() */
+		smp_rmb();
+		target = ailp->ail_target;
+		ailp->ail_target_prev = target;
+	}
 
 	/* we're done if the AIL is empty or our push has reached the end */
 	lip = xfs_trans_ail_cursor_first(ailp, &cur, ailp->ail_last_pushed_lsn);
@@ -724,7 +736,6 @@ xfs_ail_push_all_sync(
 	spin_lock(&ailp->ail_lock);
 	while ((lip = xfs_ail_max(ailp)) != NULL) {
 		prepare_to_wait(&ailp->ail_empty, &wait, TASK_UNINTERRUPTIBLE);
-		ailp->ail_target = lip->li_lsn;
 		wake_up_process(ailp->ail_task);
 		spin_unlock(&ailp->ail_lock);
 		schedule();
-- 
2.35.1


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

* [PATCH 4/7] xfs: async CIL flushes need pending pushes to be made stable
  2022-03-15  6:42 [PATCH 0/7 v3] xfs: log recovery fixes Dave Chinner
                   ` (2 preceding siblings ...)
  2022-03-15  6:42 ` [PATCH 3/7] xfs: xfs_ail_push_all_sync() stalls when racing with updates Dave Chinner
@ 2022-03-15  6:42 ` Dave Chinner
  2022-03-15 19:36   ` Darrick J. Wong
  2022-03-16 10:34   ` Chandan Babu R
  2022-03-15  6:42 ` [PATCH 5/7] xfs: log items should have a xlog pointer, not a mount Dave Chinner
                   ` (2 subsequent siblings)
  6 siblings, 2 replies; 29+ messages in thread
From: Dave Chinner @ 2022-03-15  6:42 UTC (permalink / raw)
  To: linux-xfs

From: Dave Chinner <dchinner@redhat.com>

When the AIL tries to flush the CIL, it relies on the CIL push
ending up on stable storage without having to wait for and
manipulate iclog state directly. However, if there is already a
pending CIL push when the AIL tries to flush the CIL, it won't set
the cil->xc_push_commit_stable flag and so the CIL push will not
actively flush the commit record iclog.

generic/530 when run on a single CPU test VM can trigger this fairly
reliably. This test exercises unlinked inode recovery, and can
result in inodes being pinned in memory by ongoing modifications to
the inode cluster buffer to record unlinked list modifications. As a
result, the first inode unlinked in a buffer can pin the tail of the
log whilst the inode cluster buffer is pinned by the current
checkpoint that has been pushed but isn't on stable storage because
because the cil->xc_push_commit_stable was not set. This results in
the log/AIL effectively deadlocking until something triggers the
commit record iclog to be pushed to stable storage (i.e. the
periodic log worker calling xfs_log_force()).

The fix is two-fold - first we should always set the
cil->xc_push_commit_stable when xlog_cil_flush() is called,
regardless of whether there is already a pending push or not.

Second, if the CIL is empty, we should trigger an iclog flush to
ensure that the iclogs of the last checkpoint have actually been
submitted to disk as that checkpoint may not have been run under
stable completion constraints.

Reported-and-tested-by: Matthew Wilcox <willy@infradead.org>
Fixes: 0020a190cf3e ("xfs: AIL needs asynchronous CIL forcing")
Signed-off-by: Dave Chinner <dchinner@redhat.com>
---
 fs/xfs/xfs_log_cil.c | 22 +++++++++++++++++++---
 1 file changed, 19 insertions(+), 3 deletions(-)

diff --git a/fs/xfs/xfs_log_cil.c b/fs/xfs/xfs_log_cil.c
index 3d8ebf2a1e55..48b16a5feb27 100644
--- a/fs/xfs/xfs_log_cil.c
+++ b/fs/xfs/xfs_log_cil.c
@@ -1369,18 +1369,27 @@ xlog_cil_push_now(
 	if (!async)
 		flush_workqueue(cil->xc_push_wq);
 
+	spin_lock(&cil->xc_push_lock);
+
+	/*
+	 * If this is an async flush request, we always need to set the
+	 * xc_push_commit_stable flag even if something else has already queued
+	 * a push. The flush caller is asking for the CIL to be on stable
+	 * storage when the next push completes, so regardless of who has queued
+	 * the push, the flush requires stable semantics from it.
+	 */
+	cil->xc_push_commit_stable = async;
+
 	/*
 	 * If the CIL is empty or we've already pushed the sequence then
-	 * there's no work we need to do.
+	 * there's no more work that we need to do.
 	 */
-	spin_lock(&cil->xc_push_lock);
 	if (list_empty(&cil->xc_cil) || push_seq <= cil->xc_push_seq) {
 		spin_unlock(&cil->xc_push_lock);
 		return;
 	}
 
 	cil->xc_push_seq = push_seq;
-	cil->xc_push_commit_stable = async;
 	queue_work(cil->xc_push_wq, &cil->xc_ctx->push_work);
 	spin_unlock(&cil->xc_push_lock);
 }
@@ -1520,6 +1529,13 @@ xlog_cil_flush(
 
 	trace_xfs_log_force(log->l_mp, seq, _RET_IP_);
 	xlog_cil_push_now(log, seq, true);
+
+	/*
+	 * If the CIL is empty, make sure that any previous checkpoint that may
+	 * still be in an active iclog is pushed to stable storage.
+	 */
+	if (list_empty(&log->l_cilp->xc_cil))
+		xfs_log_force(log->l_mp, 0);
 }
 
 /*
-- 
2.35.1


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

* [PATCH 5/7] xfs: log items should have a xlog pointer, not a mount
  2022-03-15  6:42 [PATCH 0/7 v3] xfs: log recovery fixes Dave Chinner
                   ` (3 preceding siblings ...)
  2022-03-15  6:42 ` [PATCH 4/7] xfs: async CIL flushes need pending pushes to be made stable Dave Chinner
@ 2022-03-15  6:42 ` Dave Chinner
  2022-03-15 19:37   ` Darrick J. Wong
  2022-03-16 11:06   ` Chandan Babu R
  2022-03-15  6:42 ` [PATCH 6/7] xfs: AIL should be log centric Dave Chinner
  2022-03-15  6:42 ` [PATCH 7/7] xfs: xfs_is_shutdown vs xlog_is_shutdown cage fight Dave Chinner
  6 siblings, 2 replies; 29+ messages in thread
From: Dave Chinner @ 2022-03-15  6:42 UTC (permalink / raw)
  To: linux-xfs

From: Dave Chinner <dchinner@redhat.com>

Log items belong to the log, not the xfs_mount. Convert the mount
pointer in the log item to a xlog pointer in preparation for
upcoming log centric changes to the log items.

Signed-off-by: Dave Chinner <dchinner@redhat.com>
---
 fs/xfs/xfs_bmap_item.c     | 2 +-
 fs/xfs/xfs_buf_item.c      | 5 +++--
 fs/xfs/xfs_extfree_item.c  | 2 +-
 fs/xfs/xfs_log.c           | 2 +-
 fs/xfs/xfs_log_cil.c       | 2 +-
 fs/xfs/xfs_refcount_item.c | 2 +-
 fs/xfs/xfs_rmap_item.c     | 2 +-
 fs/xfs/xfs_trace.h         | 4 ++--
 fs/xfs/xfs_trans.c         | 2 +-
 fs/xfs/xfs_trans.h         | 3 ++-
 10 files changed, 14 insertions(+), 12 deletions(-)

diff --git a/fs/xfs/xfs_bmap_item.c b/fs/xfs/xfs_bmap_item.c
index fa710067aac2..65ac261b3b28 100644
--- a/fs/xfs/xfs_bmap_item.c
+++ b/fs/xfs/xfs_bmap_item.c
@@ -476,7 +476,7 @@ xfs_bui_item_recover(
 	struct xfs_bui_log_item		*buip = BUI_ITEM(lip);
 	struct xfs_trans		*tp;
 	struct xfs_inode		*ip = NULL;
-	struct xfs_mount		*mp = lip->li_mountp;
+	struct xfs_mount		*mp = lip->li_log->l_mp;
 	struct xfs_map_extent		*bmap;
 	struct xfs_bud_log_item		*budp;
 	xfs_filblks_t			count;
diff --git a/fs/xfs/xfs_buf_item.c b/fs/xfs/xfs_buf_item.c
index a7a8e4528881..522d450a94b1 100644
--- a/fs/xfs/xfs_buf_item.c
+++ b/fs/xfs/xfs_buf_item.c
@@ -21,6 +21,7 @@
 #include "xfs_dquot.h"
 #include "xfs_trace.h"
 #include "xfs_log.h"
+#include "xfs_log_priv.h"
 
 
 struct kmem_cache	*xfs_buf_item_cache;
@@ -428,7 +429,7 @@ xfs_buf_item_format(
 	 * occurs during recovery.
 	 */
 	if (bip->bli_flags & XFS_BLI_INODE_BUF) {
-		if (xfs_has_v3inodes(lip->li_mountp) ||
+		if (xfs_has_v3inodes(lip->li_log->l_mp) ||
 		    !((bip->bli_flags & XFS_BLI_INODE_ALLOC_BUF) &&
 		      xfs_log_item_in_current_chkpt(lip)))
 			bip->__bli_format.blf_flags |= XFS_BLF_INODE_BUF;
@@ -616,7 +617,7 @@ xfs_buf_item_put(
 	 * that case, the bli is freed on buffer writeback completion.
 	 */
 	aborted = test_bit(XFS_LI_ABORTED, &lip->li_flags) ||
-		  xfs_is_shutdown(lip->li_mountp);
+			xlog_is_shutdown(lip->li_log);
 	dirty = bip->bli_flags & XFS_BLI_DIRTY;
 	if (dirty && !aborted)
 		return false;
diff --git a/fs/xfs/xfs_extfree_item.c b/fs/xfs/xfs_extfree_item.c
index 36eeac9413f5..893a7dd15cbb 100644
--- a/fs/xfs/xfs_extfree_item.c
+++ b/fs/xfs/xfs_extfree_item.c
@@ -615,7 +615,7 @@ xfs_efi_item_recover(
 	struct list_head		*capture_list)
 {
 	struct xfs_efi_log_item		*efip = EFI_ITEM(lip);
-	struct xfs_mount		*mp = lip->li_mountp;
+	struct xfs_mount		*mp = lip->li_log->l_mp;
 	struct xfs_efd_log_item		*efdp;
 	struct xfs_trans		*tp;
 	struct xfs_extent		*extp;
diff --git a/fs/xfs/xfs_log.c b/fs/xfs/xfs_log.c
index b0e05fa902d4..5c4ef45f42d2 100644
--- a/fs/xfs/xfs_log.c
+++ b/fs/xfs/xfs_log.c
@@ -1136,7 +1136,7 @@ xfs_log_item_init(
 	int			type,
 	const struct xfs_item_ops *ops)
 {
-	item->li_mountp = mp;
+	item->li_log = mp->m_log;
 	item->li_ailp = mp->m_ail;
 	item->li_type = type;
 	item->li_ops = ops;
diff --git a/fs/xfs/xfs_log_cil.c b/fs/xfs/xfs_log_cil.c
index 48b16a5feb27..e9b80036268a 100644
--- a/fs/xfs/xfs_log_cil.c
+++ b/fs/xfs/xfs_log_cil.c
@@ -76,7 +76,7 @@ bool
 xfs_log_item_in_current_chkpt(
 	struct xfs_log_item *lip)
 {
-	return xlog_item_in_current_chkpt(lip->li_mountp->m_log->l_cilp, lip);
+	return xlog_item_in_current_chkpt(lip->li_log->l_cilp, lip);
 }
 
 /*
diff --git a/fs/xfs/xfs_refcount_item.c b/fs/xfs/xfs_refcount_item.c
index d4632f2ceb89..1b82b818f515 100644
--- a/fs/xfs/xfs_refcount_item.c
+++ b/fs/xfs/xfs_refcount_item.c
@@ -468,7 +468,7 @@ xfs_cui_item_recover(
 	struct xfs_cud_log_item		*cudp;
 	struct xfs_trans		*tp;
 	struct xfs_btree_cur		*rcur = NULL;
-	struct xfs_mount		*mp = lip->li_mountp;
+	struct xfs_mount		*mp = lip->li_log->l_mp;
 	xfs_fsblock_t			new_fsb;
 	xfs_extlen_t			new_len;
 	unsigned int			refc_type;
diff --git a/fs/xfs/xfs_rmap_item.c b/fs/xfs/xfs_rmap_item.c
index cb0490919b2c..546bd824cdf7 100644
--- a/fs/xfs/xfs_rmap_item.c
+++ b/fs/xfs/xfs_rmap_item.c
@@ -523,7 +523,7 @@ xfs_rui_item_recover(
 	struct xfs_rud_log_item		*rudp;
 	struct xfs_trans		*tp;
 	struct xfs_btree_cur		*rcur = NULL;
-	struct xfs_mount		*mp = lip->li_mountp;
+	struct xfs_mount		*mp = lip->li_log->l_mp;
 	enum xfs_rmap_intent_type	type;
 	xfs_exntst_t			state;
 	int				i;
diff --git a/fs/xfs/xfs_trace.h b/fs/xfs/xfs_trace.h
index 585bd9853b6b..cc69b7c066e8 100644
--- a/fs/xfs/xfs_trace.h
+++ b/fs/xfs/xfs_trace.h
@@ -1308,7 +1308,7 @@ DECLARE_EVENT_CLASS(xfs_log_item_class,
 		__field(xfs_lsn_t, lsn)
 	),
 	TP_fast_assign(
-		__entry->dev = lip->li_mountp->m_super->s_dev;
+		__entry->dev = lip->li_log->l_mp->m_super->s_dev;
 		__entry->lip = lip;
 		__entry->type = lip->li_type;
 		__entry->flags = lip->li_flags;
@@ -1364,7 +1364,7 @@ DECLARE_EVENT_CLASS(xfs_ail_class,
 		__field(xfs_lsn_t, new_lsn)
 	),
 	TP_fast_assign(
-		__entry->dev = lip->li_mountp->m_super->s_dev;
+		__entry->dev = lip->li_log->l_mp->m_super->s_dev;
 		__entry->lip = lip;
 		__entry->type = lip->li_type;
 		__entry->flags = lip->li_flags;
diff --git a/fs/xfs/xfs_trans.c b/fs/xfs/xfs_trans.c
index 82590007e6c5..de87fb136b51 100644
--- a/fs/xfs/xfs_trans.c
+++ b/fs/xfs/xfs_trans.c
@@ -646,7 +646,7 @@ xfs_trans_add_item(
 	struct xfs_trans	*tp,
 	struct xfs_log_item	*lip)
 {
-	ASSERT(lip->li_mountp == tp->t_mountp);
+	ASSERT(lip->li_log == tp->t_mountp->m_log);
 	ASSERT(lip->li_ailp == tp->t_mountp->m_ail);
 	ASSERT(list_empty(&lip->li_trans));
 	ASSERT(!test_bit(XFS_LI_DIRTY, &lip->li_flags));
diff --git a/fs/xfs/xfs_trans.h b/fs/xfs/xfs_trans.h
index 85dca2c9b559..1c5c5d7f522f 100644
--- a/fs/xfs/xfs_trans.h
+++ b/fs/xfs/xfs_trans.h
@@ -8,6 +8,7 @@
 
 /* kernel only transaction subsystem defines */
 
+struct xlog;
 struct xfs_buf;
 struct xfs_buftarg;
 struct xfs_efd_log_item;
@@ -31,7 +32,7 @@ struct xfs_log_item {
 	struct list_head		li_ail;		/* AIL pointers */
 	struct list_head		li_trans;	/* transaction list */
 	xfs_lsn_t			li_lsn;		/* last on-disk lsn */
-	struct xfs_mount		*li_mountp;	/* ptr to fs mount */
+	struct xlog			*li_log;
 	struct xfs_ail			*li_ailp;	/* ptr to AIL */
 	uint				li_type;	/* item type */
 	unsigned long			li_flags;	/* misc flags */
-- 
2.35.1


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

* [PATCH 6/7] xfs: AIL should be log centric
  2022-03-15  6:42 [PATCH 0/7 v3] xfs: log recovery fixes Dave Chinner
                   ` (4 preceding siblings ...)
  2022-03-15  6:42 ` [PATCH 5/7] xfs: log items should have a xlog pointer, not a mount Dave Chinner
@ 2022-03-15  6:42 ` Dave Chinner
  2022-03-15 19:39   ` Darrick J. Wong
  2022-03-16 11:12   ` Chandan Babu R
  2022-03-15  6:42 ` [PATCH 7/7] xfs: xfs_is_shutdown vs xlog_is_shutdown cage fight Dave Chinner
  6 siblings, 2 replies; 29+ messages in thread
From: Dave Chinner @ 2022-03-15  6:42 UTC (permalink / raw)
  To: linux-xfs

From: Dave Chinner <dchinner@redhat.com>

The AIL operates purely on log items, so it is a log centric
subsystem. Divorce it from the xfs_mount and instead have it pass
around xlog pointers.

Signed-off-by: Dave Chinner <dchinner@redhat.com>
---
 fs/xfs/xfs_trans.c      |  2 +-
 fs/xfs/xfs_trans_ail.c  | 26 +++++++++++++-------------
 fs/xfs/xfs_trans_priv.h |  3 ++-
 3 files changed, 16 insertions(+), 15 deletions(-)

diff --git a/fs/xfs/xfs_trans.c b/fs/xfs/xfs_trans.c
index de87fb136b51..831d355c3258 100644
--- a/fs/xfs/xfs_trans.c
+++ b/fs/xfs/xfs_trans.c
@@ -773,7 +773,7 @@ xfs_trans_committed_bulk(
 		 * object into the AIL as we are in a shutdown situation.
 		 */
 		if (aborted) {
-			ASSERT(xfs_is_shutdown(ailp->ail_mount));
+			ASSERT(xlog_is_shutdown(ailp->ail_log));
 			if (lip->li_ops->iop_unpin)
 				lip->li_ops->iop_unpin(lip, 1);
 			continue;
diff --git a/fs/xfs/xfs_trans_ail.c b/fs/xfs/xfs_trans_ail.c
index 1b52952097c1..c2ccb98c7bcd 100644
--- a/fs/xfs/xfs_trans_ail.c
+++ b/fs/xfs/xfs_trans_ail.c
@@ -398,7 +398,7 @@ xfsaild_push_item(
 	 * If log item pinning is enabled, skip the push and track the item as
 	 * pinned. This can help induce head-behind-tail conditions.
 	 */
-	if (XFS_TEST_ERROR(false, ailp->ail_mount, XFS_ERRTAG_LOG_ITEM_PIN))
+	if (XFS_TEST_ERROR(false, ailp->ail_log->l_mp, XFS_ERRTAG_LOG_ITEM_PIN))
 		return XFS_ITEM_PINNED;
 
 	/*
@@ -418,7 +418,7 @@ static long
 xfsaild_push(
 	struct xfs_ail		*ailp)
 {
-	xfs_mount_t		*mp = ailp->ail_mount;
+	struct xfs_mount	*mp = ailp->ail_log->l_mp;
 	struct xfs_ail_cursor	cur;
 	struct xfs_log_item	*lip;
 	xfs_lsn_t		lsn;
@@ -443,7 +443,7 @@ xfsaild_push(
 		ailp->ail_log_flush = 0;
 
 		XFS_STATS_INC(mp, xs_push_ail_flush);
-		xlog_cil_flush(mp->m_log);
+		xlog_cil_flush(ailp->ail_log);
 	}
 
 	spin_lock(&ailp->ail_lock);
@@ -632,7 +632,7 @@ xfsaild(
 			 * opportunity to release such buffers from the queue.
 			 */
 			ASSERT(list_empty(&ailp->ail_buf_list) ||
-			       xfs_is_shutdown(ailp->ail_mount));
+			       xlog_is_shutdown(ailp->ail_log));
 			xfs_buf_delwri_cancel(&ailp->ail_buf_list);
 			break;
 		}
@@ -695,7 +695,7 @@ xfs_ail_push(
 	struct xfs_log_item	*lip;
 
 	lip = xfs_ail_min(ailp);
-	if (!lip || xfs_is_shutdown(ailp->ail_mount) ||
+	if (!lip || xlog_is_shutdown(ailp->ail_log) ||
 	    XFS_LSN_CMP(threshold_lsn, ailp->ail_target) <= 0)
 		return;
 
@@ -751,7 +751,7 @@ xfs_ail_update_finish(
 	struct xfs_ail		*ailp,
 	xfs_lsn_t		old_lsn) __releases(ailp->ail_lock)
 {
-	struct xfs_mount	*mp = ailp->ail_mount;
+	struct xlog		*log = ailp->ail_log;
 
 	/* if the tail lsn hasn't changed, don't do updates or wakeups. */
 	if (!old_lsn || old_lsn == __xfs_ail_min_lsn(ailp)) {
@@ -759,13 +759,13 @@ xfs_ail_update_finish(
 		return;
 	}
 
-	if (!xfs_is_shutdown(mp))
-		xlog_assign_tail_lsn_locked(mp);
+	if (!xlog_is_shutdown(log))
+		xlog_assign_tail_lsn_locked(log->l_mp);
 
 	if (list_empty(&ailp->ail_head))
 		wake_up_all(&ailp->ail_empty);
 	spin_unlock(&ailp->ail_lock);
-	xfs_log_space_wake(mp);
+	xfs_log_space_wake(log->l_mp);
 }
 
 /*
@@ -873,13 +873,13 @@ xfs_trans_ail_delete(
 	int			shutdown_type)
 {
 	struct xfs_ail		*ailp = lip->li_ailp;
-	struct xfs_mount	*mp = ailp->ail_mount;
+	struct xfs_mount	*mp = ailp->ail_log->l_mp;
 	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 && !xfs_is_shutdown(mp)) {
+		if (shutdown_type && !xlog_is_shutdown(ailp->ail_log)) {
 			xfs_alert_tag(mp, XFS_PTAG_AILDELETE,
 	"%s: attempting to delete a log item that is not in the AIL",
 					__func__);
@@ -904,7 +904,7 @@ xfs_trans_ail_init(
 	if (!ailp)
 		return -ENOMEM;
 
-	ailp->ail_mount = mp;
+	ailp->ail_log = mp->m_log;
 	INIT_LIST_HEAD(&ailp->ail_head);
 	INIT_LIST_HEAD(&ailp->ail_cursors);
 	spin_lock_init(&ailp->ail_lock);
@@ -912,7 +912,7 @@ xfs_trans_ail_init(
 	init_waitqueue_head(&ailp->ail_empty);
 
 	ailp->ail_task = kthread_run(xfsaild, ailp, "xfsaild/%s",
-			ailp->ail_mount->m_super->s_id);
+				mp->m_super->s_id);
 	if (IS_ERR(ailp->ail_task))
 		goto out_free_ailp;
 
diff --git a/fs/xfs/xfs_trans_priv.h b/fs/xfs/xfs_trans_priv.h
index 3004aeac9110..f0d79a9050ba 100644
--- a/fs/xfs/xfs_trans_priv.h
+++ b/fs/xfs/xfs_trans_priv.h
@@ -6,6 +6,7 @@
 #ifndef __XFS_TRANS_PRIV_H__
 #define	__XFS_TRANS_PRIV_H__
 
+struct xlog;
 struct xfs_log_item;
 struct xfs_mount;
 struct xfs_trans;
@@ -50,7 +51,7 @@ struct xfs_ail_cursor {
  * Eventually we need to drive the locking in here as well.
  */
 struct xfs_ail {
-	struct xfs_mount	*ail_mount;
+	struct xlog		*ail_log;
 	struct task_struct	*ail_task;
 	struct list_head	ail_head;
 	xfs_lsn_t		ail_target;
-- 
2.35.1


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

* [PATCH 7/7] xfs: xfs_is_shutdown vs xlog_is_shutdown cage fight
  2022-03-15  6:42 [PATCH 0/7 v3] xfs: log recovery fixes Dave Chinner
                   ` (5 preceding siblings ...)
  2022-03-15  6:42 ` [PATCH 6/7] xfs: AIL should be log centric Dave Chinner
@ 2022-03-15  6:42 ` Dave Chinner
  2022-03-15 20:03   ` Darrick J. Wong
  6 siblings, 1 reply; 29+ messages in thread
From: Dave Chinner @ 2022-03-15  6:42 UTC (permalink / raw)
  To: linux-xfs

From: Dave Chinner <dchinner@redhat.com>

I've been chasing a recent resurgence in generic/388 recovery
failure and/or corruption events. The events have largely been
uninitialised inode chunks being tripped over in log recovery
such as:

 XFS (pmem1): User initiated shutdown received.
 pmem1: writeback error on inode 12621949, offset 1019904, sector 12968096
 XFS (pmem1): Log I/O Error (0x6) detected at xfs_fs_goingdown+0xa3/0xf0 (fs/xfs/xfs_fsops.c:500).  Shutting down filesystem.
 XFS (pmem1): Please unmount the filesystem and rectify the problem(s)
 XFS (pmem1): Unmounting Filesystem
 XFS (pmem1): Mounting V5 Filesystem
 XFS (pmem1): Starting recovery (logdev: internal)
 XFS (pmem1): bad inode magic/vsn daddr 8723584 #0 (magic=1818)
 XFS (pmem1): Metadata corruption detected at xfs_inode_buf_verify+0x180/0x190, xfs_inode block 0x851c80 xfs_inode_buf_verify
 XFS (pmem1): Unmount and run xfs_repair
 XFS (pmem1): First 128 bytes of corrupted metadata buffer:
 00000000: 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18  ................
 00000010: 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18  ................
 00000020: 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18  ................
 00000030: 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18  ................
 00000040: 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18  ................
 00000050: 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18  ................
 00000060: 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18  ................
 00000070: 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18  ................
 XFS (pmem1): metadata I/O error in "xlog_recover_items_pass2+0x52/0xc0" at daddr 0x851c80 len 32 error 117
 XFS (pmem1): log mount/recovery failed: error -117
 XFS (pmem1): log mount failed

There have been isolated random other issues, too - xfs_repair fails
because it finds some corruption in symlink blocks, rmap
inconsistencies, etc - but they are nowhere near as common as the
uninitialised inode chunk failure.

The problem has clearly happened at runtime before recovery has run;
I can see the ICREATE log item in the log shortly before the
actively recovered range of the log. This means the ICREATE was
definitely created and written to the log, but for some reason the
tail of the log has been moved past the ordered buffer log item that
tracks INODE_ALLOC buffers and, supposedly, prevents the tail of the
log moving past the ICREATE log item before the inode chunk buffer
is written to disk.

Tracing the fsstress processes that are running when the filesystem
shut down immediately pin-pointed the problem:

user shutdown marks xfs_mount as shutdown

         godown-213341 [008]  6398.022871: console:              [ 6397.915392] XFS (pmem1): User initiated shutdown received.
.....

aild tries to push ordered inode cluster buffer

  xfsaild/pmem1-213314 [001]  6398.022974: xfs_buf_trylock:      dev 259:1 daddr 0x851c80 bbcount 0x20 hold 16 pincount 0 lock 0 flags DONE|INODES|PAGES caller xfs_inode_item_push+0x8e
  xfsaild/pmem1-213314 [001]  6398.022976: xfs_ilock_nowait:     dev 259:1 ino 0x851c80 flags ILOCK_SHARED caller xfs_iflush_cluster+0xae

xfs_iflush_cluster() checks xfs_is_shutdown(), returns true,
calls xfs_iflush_abort() to kill writeback of the inode.
Inode is removed from AIL, drops cluster buffer reference.

  xfsaild/pmem1-213314 [001]  6398.022977: xfs_ail_delete:       dev 259:1 lip 0xffff88880247ed80 old lsn 7/20344 new lsn 7/21000 type XFS_LI_INODE flags IN_AIL
  xfsaild/pmem1-213314 [001]  6398.022978: xfs_buf_rele:         dev 259:1 daddr 0x851c80 bbcount 0x20 hold 17 pincount 0 lock 0 flags DONE|INODES|PAGES caller xfs_iflush_abort+0xd7

.....

All inodes on cluster buffer are aborted, then the cluster buffer
itself is aborted and removed from the AIL *without writeback*:

xfsaild/pmem1-213314 [001]  6398.023011: xfs_buf_error_relse:  dev 259:1 daddr 0x851c80 bbcount 0x20 hold 2 pincount 0 lock 0 flags ASYNC|DONE|STALE|INODES|PAGES caller xfs_buf_ioend_fail+0x33
   xfsaild/pmem1-213314 [001]  6398.023012: xfs_ail_delete:       dev 259:1 lip 0xffff8888053efde8 old lsn 7/20344 new lsn 7/20344 type XFS_LI_BUF flags IN_AIL

The inode buffer was at 7/20344 when it was removed from the AIL.

   xfsaild/pmem1-213314 [001]  6398.023012: xfs_buf_item_relse:   dev 259:1 daddr 0x851c80 bbcount 0x20 hold 2 pincount 0 lock 0 flags ASYNC|DONE|STALE|INODES|PAGES caller xfs_buf_item_done+0x31
   xfsaild/pmem1-213314 [001]  6398.023012: xfs_buf_rele:         dev 259:1 daddr 0x851c80 bbcount 0x20 hold 2 pincount 0 lock 0 flags ASYNC|DONE|STALE|INODES|PAGES caller xfs_buf_item_relse+0x39

.....

Userspace is still running, doing stuff. an fsstress process runs
syncfs() or sync() and we end up in sync_fs_one_sb() which issues
a log force. This pushes on the CIL:

        fsstress-213322 [001]  6398.024430: xfs_fs_sync_fs:       dev 259:1 m_features 0x20000000019ff6e9 opstate (clean|shutdown|inodegc|blockgc) s_flags 0x70810000 caller sync_fs_one_sb+0x26
        fsstress-213322 [001]  6398.024430: xfs_log_force:        dev 259:1 lsn 0x0 caller xfs_fs_sync_fs+0x82
        fsstress-213322 [001]  6398.024430: xfs_log_force:        dev 259:1 lsn 0x5f caller xfs_log_force+0x7c
           <...>-194402 [001]  6398.024467: kmem_alloc:           size 176 flags 0x14 caller xlog_cil_push_work+0x9f

And the CIL fills up iclogs with pending changes. This picks up
the current tail from the AIL:

           <...>-194402 [001]  6398.024497: xlog_iclog_get_space: dev 259:1 state XLOG_STATE_ACTIVE refcnt 1 offset 0 lsn 0x0 flags  caller xlog_write+0x149
           <...>-194402 [001]  6398.024498: xlog_iclog_switch:    dev 259:1 state XLOG_STATE_ACTIVE refcnt 1 offset 0 lsn 0x700005408 flags  caller xlog_state_get_iclog_space+0x37e
           <...>-194402 [001]  6398.024521: xlog_iclog_release:   dev 259:1 state XLOG_STATE_WANT_SYNC refcnt 1 offset 32256 lsn 0x700005408 flags  caller xlog_write+0x5f9
           <...>-194402 [001]  6398.024522: xfs_log_assign_tail_lsn: dev 259:1 new tail lsn 7/21000, old lsn 7/20344, last sync 7/21448

And it moves the tail of the log to 7/21000 from 7/20344. This
*moves the tail of the log beyond the ICREATE transaction* that was
at 7/20344 and pinned by the inode cluster buffer that was cancelled
above.

....

         godown-213341 [008]  6398.027005: xfs_force_shutdown:   dev 259:1 tag logerror flags log_io|force_umount file fs/xfs/xfs_fsops.c line_num 500
          godown-213341 [008]  6398.027022: console:              [ 6397.915406] pmem1: writeback error on inode 12621949, offset 1019904, sector 12968096
          godown-213341 [008]  6398.030551: console:              [ 6397.919546] XFS (pmem1): Log I/O Error (0x6) detected at xfs_fs_goingdown+0xa3/0xf0 (fs/

And finally the log itself is now shutdown, stopping all further
writes to the log. But this is too late to prevent the corruption
that moving the tail of the log forwards after we start cancelling
writeback causes.

The fundamental problem here is that we are using the wrong shutdown
checks for log items. We've long conflated mount shutdown with log
shutdown state, and I started separating that recently with the
atomic shutdown state changes in commit b36d4651e165 ("xfs: make
forced shutdown processing atomic"). The changes in that commit
series are directly responsible for being able to diagnose this
issue because it clearly separated mount shutdown from log shutdown.

Essentially, once we start cancelling writeback of log items and
removing them from the AIL because the filesystem is shut down, we
*cannot* update the journal because we may have cancelled the items
that pin the tail of the log. That moves the tail of the log
forwards without having written the metadata back, hence we have
corrupt in memory state and writing to the journal propagates that
to the on-disk state.

What commit b36d4651e165 makes clear is that log item state needs to
change relative to log shutdown, not mount shutdown. IOWs, anything
that aborts metadata writeback needs to check log shutdown state
because log items directly affect log consistency. Having them check
mount shutdown state introduces the above race condition where we
cancel metadata writeback before the log shuts down.

To fix this, this patch works through all log items and converts
shutdown checks to use xlog_is_shutdown() rather than
xfs_is_shutdown(), so that we don't start aborting metadata
writeback before we shut off journal writes.

AFAICT, this race condition is a zero day IO error handling bug in
XFS that dates back to the introduction of XLOG_IO_ERROR,
XLOG_STATE_IOERROR and XFS_FORCED_SHUTDOWN back in January 1997.

Signed-off-by: Dave Chinner <dchinner@redhat.com>
---
 fs/xfs/xfs_buf.c        | 30 +++++++++++++++++++++++-------
 fs/xfs/xfs_icache.c     |  3 ++-
 fs/xfs/xfs_inode.c      | 15 +++++++++++++--
 fs/xfs/xfs_inode_item.c | 12 ++++++++++++
 fs/xfs/xfs_qm.c         |  8 ++++----
 5 files changed, 54 insertions(+), 14 deletions(-)

diff --git a/fs/xfs/xfs_buf.c b/fs/xfs/xfs_buf.c
index 8867f143598e..b6073b5a990a 100644
--- a/fs/xfs/xfs_buf.c
+++ b/fs/xfs/xfs_buf.c
@@ -14,6 +14,7 @@
 #include "xfs_trace.h"
 #include "xfs_log.h"
 #include "xfs_log_recover.h"
+#include "xfs_log_priv.h"
 #include "xfs_trans.h"
 #include "xfs_buf_item.h"
 #include "xfs_errortag.h"
@@ -813,7 +814,7 @@ xfs_buf_read_map(
 	 * buffer.
 	 */
 	if (error) {
-		if (!xfs_is_shutdown(target->bt_mount))
+		if (!xlog_is_shutdown(target->bt_mount->m_log))
 			xfs_buf_ioerror_alert(bp, fa);
 
 		bp->b_flags &= ~XBF_DONE;
@@ -1177,10 +1178,10 @@ xfs_buf_ioend_handle_error(
 	struct xfs_error_cfg	*cfg;
 
 	/*
-	 * If we've already decided to shutdown the filesystem because of I/O
+	 * If we've already decided to shutdown the journal because of I/O
 	 * errors, there's no point in giving this a retry.
 	 */
-	if (xfs_is_shutdown(mp))
+	if (xlog_is_shutdown(mp->m_log))
 		goto out_stale;
 
 	xfs_buf_ioerror_alert_ratelimited(bp);
@@ -1593,8 +1594,23 @@ __xfs_buf_submit(
 
 	ASSERT(!(bp->b_flags & _XBF_DELWRI_Q));
 
-	/* on shutdown we stale and complete the buffer immediately */
-	if (xfs_is_shutdown(bp->b_mount)) {
+	/*
+	 * On log shutdown we stale and complete the buffer immediately. We can
+	 * be called to read the superblock before the log has been set up, so
+	 * be careful checking the log state.
+	 *
+	 * Checking the mount shutdown state here can result in the log tail
+	 * moving inappropriately on disk as the log may not yet be shut down.
+	 * Hence failing this buffer on mount shutdown can remove it from the
+	 * AIL and move the tail of the log forwards without having written
+	 * this buffer to disk. This corrupts the log tail state in memory, and
+	 * because the log isn't yet shut down, it can then be propagated to
+	 * disk before the log is shutdown. Hence we check log shutdown state
+	 * here rather than mount state to avoid corrupting the log tail on
+	 * shutdown.
+	 */
+	if (bp->b_mount->m_log &&
+	    xlog_is_shutdown(bp->b_mount->m_log)) {
 		xfs_buf_ioend_fail(bp);
 		return -EIO;
 	}
@@ -1808,10 +1824,10 @@ xfs_buftarg_drain(
 	 * If one or more failed buffers were freed, that means dirty metadata
 	 * was thrown away. This should only ever happen after I/O completion
 	 * handling has elevated I/O error(s) to permanent failures and shuts
-	 * down the fs.
+	 * down the journal.
 	 */
 	if (write_fail) {
-		ASSERT(xfs_is_shutdown(btp->bt_mount));
+		ASSERT(xlog_is_shutdown(btp->bt_mount->m_log));
 		xfs_alert(btp->bt_mount,
 	      "Please run xfs_repair to determine the extent of the problem.");
 	}
diff --git a/fs/xfs/xfs_icache.c b/fs/xfs/xfs_icache.c
index 9644f938990c..57ebf6ceef30 100644
--- a/fs/xfs/xfs_icache.c
+++ b/fs/xfs/xfs_icache.c
@@ -23,6 +23,7 @@
 #include "xfs_reflink.h"
 #include "xfs_ialloc.h"
 #include "xfs_ag.h"
+#include "xfs_log_priv.h"
 
 #include <linux/iversion.h>
 
@@ -873,7 +874,7 @@ xfs_reclaim_inode(
 	if (xfs_iflags_test_and_set(ip, XFS_IFLUSHING))
 		goto out_iunlock;
 
-	if (xfs_is_shutdown(ip->i_mount)) {
+	if (xlog_is_shutdown(ip->i_mount->m_log)) {
 		xfs_iunpin_wait(ip);
 		xfs_iflush_abort(ip);
 		goto reclaim;
diff --git a/fs/xfs/xfs_inode.c b/fs/xfs/xfs_inode.c
index 04bf467b1090..aab55a06ece7 100644
--- a/fs/xfs/xfs_inode.c
+++ b/fs/xfs/xfs_inode.c
@@ -35,6 +35,7 @@
 #include "xfs_bmap_btree.h"
 #include "xfs_reflink.h"
 #include "xfs_ag.h"
+#include "xfs_log_priv.h"
 
 struct kmem_cache *xfs_inode_cache;
 
@@ -3659,7 +3660,7 @@ xfs_iflush_cluster(
 		 * AIL, leaving a dirty/unpinned inode attached to the buffer
 		 * that otherwise looks like it should be flushed.
 		 */
-		if (xfs_is_shutdown(mp)) {
+		if (xlog_is_shutdown(mp->m_log)) {
 			xfs_iunpin_wait(ip);
 			xfs_iflush_abort(ip);
 			xfs_iunlock(ip, XFS_ILOCK_SHARED);
@@ -3685,9 +3686,19 @@ xfs_iflush_cluster(
 	}
 
 	if (error) {
+		/*
+		 * Shutdown first so we kill the log before we release this
+		 * buffer. If it is an INODE_ALLOC buffer and pins the tail
+		 * of the log, failing it before the _log_ is shut down can
+		 * result in the log tail being moved forward in the journal
+		 * on disk because log writes can still be taking place. Hence
+		 * unpinning the tail will allow the ICREATE intent to be
+		 * removed from the log an recovery will fail with uninitialised
+		 * inode cluster buffers.
+		 */
+		xfs_force_shutdown(mp, SHUTDOWN_CORRUPT_INCORE);
 		bp->b_flags |= XBF_ASYNC;
 		xfs_buf_ioend_fail(bp);
-		xfs_force_shutdown(mp, SHUTDOWN_CORRUPT_INCORE);
 		return error;
 	}
 
diff --git a/fs/xfs/xfs_inode_item.c b/fs/xfs/xfs_inode_item.c
index 19dc3e37bb4d..308b30f35a71 100644
--- a/fs/xfs/xfs_inode_item.c
+++ b/fs/xfs/xfs_inode_item.c
@@ -17,6 +17,7 @@
 #include "xfs_trans_priv.h"
 #include "xfs_buf_item.h"
 #include "xfs_log.h"
+#include "xfs_log_priv.h"
 #include "xfs_error.h"
 
 #include <linux/iversion.h>
@@ -709,6 +710,17 @@ xfs_iflush_ail_updates(
 		if (INODE_ITEM(lip)->ili_flush_lsn != lip->li_lsn)
 			continue;
 
+		/*
+		 * dgc: Not sure how this happens, but it happens very
+		 * occassionaly via generic/388.  xfs_iflush_abort() also
+		 * silently handles this same "under writeback but not in AIL at
+		 * shutdown" condition via xfs_trans_ail_delete().
+		 */
+		if (!test_bit(XFS_LI_IN_AIL, &lip->li_flags)) {
+			ASSERT(xlog_is_shutdown(lip->li_log));
+			continue;
+		}
+
 		lsn = xfs_ail_delete_one(ailp, lip);
 		if (!tail_lsn && lsn)
 			tail_lsn = lsn;
diff --git a/fs/xfs/xfs_qm.c b/fs/xfs/xfs_qm.c
index 32ac8d9c8940..f165d1a3de1d 100644
--- a/fs/xfs/xfs_qm.c
+++ b/fs/xfs/xfs_qm.c
@@ -25,6 +25,7 @@
 #include "xfs_error.h"
 #include "xfs_ag.h"
 #include "xfs_ialloc.h"
+#include "xfs_log_priv.h"
 
 /*
  * The global quota manager. There is only one of these for the entire
@@ -121,8 +122,7 @@ xfs_qm_dqpurge(
 	struct xfs_dquot	*dqp,
 	void			*data)
 {
-	struct xfs_mount	*mp = dqp->q_mount;
-	struct xfs_quotainfo	*qi = mp->m_quotainfo;
+	struct xfs_quotainfo	*qi = dqp->q_mount->m_quotainfo;
 	int			error = -EAGAIN;
 
 	xfs_dqlock(dqp);
@@ -157,7 +157,7 @@ xfs_qm_dqpurge(
 	}
 
 	ASSERT(atomic_read(&dqp->q_pincount) == 0);
-	ASSERT(xfs_is_shutdown(mp) ||
+	ASSERT(xlog_is_shutdown(dqp->q_logitem.qli_item.li_log) ||
 		!test_bit(XFS_LI_IN_AIL, &dqp->q_logitem.qli_item.li_flags));
 
 	xfs_dqfunlock(dqp);
@@ -172,7 +172,7 @@ xfs_qm_dqpurge(
 	 */
 	ASSERT(!list_empty(&dqp->q_lru));
 	list_lru_del(&qi->qi_lru, &dqp->q_lru);
-	XFS_STATS_DEC(mp, xs_qm_dquot_unused);
+	XFS_STATS_DEC(dqp->q_mount, xs_qm_dquot_unused);
 
 	xfs_qm_dqdestroy(dqp);
 	return 0;
-- 
2.35.1


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

* Re: [PATCH 1/7] xfs: log worker needs to start before intent/unlink recovery
  2022-03-15  6:42 ` [PATCH 1/7] xfs: log worker needs to start before intent/unlink recovery Dave Chinner
@ 2022-03-15  9:14   ` Chandan Babu R
  0 siblings, 0 replies; 29+ messages in thread
From: Chandan Babu R @ 2022-03-15  9:14 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs

On 15 Mar 2022 at 12:12, Dave Chinner wrote:
> From: Dave Chinner <dchinner@redhat.com>
>
> After 963 iterations of generic/530, it deadlocked during recovery
> on a pinned inode cluster buffer like so:
>
> XFS (pmem1): Starting recovery (logdev: internal)
> INFO: task kworker/8:0:306037 blocked for more than 122 seconds.
>       Not tainted 5.17.0-rc6-dgc+ #975
> "echo 0 > /proc/sys/kernel/hung_task_timeout_secs" disables this message.
> task:kworker/8:0     state:D stack:13024 pid:306037 ppid:     2 flags:0x00004000
> Workqueue: xfs-inodegc/pmem1 xfs_inodegc_worker
> Call Trace:
>  <TASK>
>  __schedule+0x30d/0x9e0
>  schedule+0x55/0xd0
>  schedule_timeout+0x114/0x160
>  __down+0x99/0xf0
>  down+0x5e/0x70
>  xfs_buf_lock+0x36/0xf0
>  xfs_buf_find+0x418/0x850
>  xfs_buf_get_map+0x47/0x380
>  xfs_buf_read_map+0x54/0x240
>  xfs_trans_read_buf_map+0x1bd/0x490
>  xfs_imap_to_bp+0x4f/0x70
>  xfs_iunlink_map_ino+0x66/0xd0
>  xfs_iunlink_map_prev.constprop.0+0x148/0x2f0
>  xfs_iunlink_remove_inode+0xf2/0x1d0
>  xfs_inactive_ifree+0x1a3/0x900
>  xfs_inode_unlink+0xcc/0x210
>  xfs_inodegc_worker+0x1ac/0x2f0
>  process_one_work+0x1ac/0x390
>  worker_thread+0x56/0x3c0
>  kthread+0xf6/0x120
>  ret_from_fork+0x1f/0x30
>  </TASK>
> task:mount           state:D stack:13248 pid:324509 ppid:324233 flags:0x00004000
> Call Trace:
>  <TASK>
>  __schedule+0x30d/0x9e0
>  schedule+0x55/0xd0
>  schedule_timeout+0x114/0x160
>  __down+0x99/0xf0
>  down+0x5e/0x70
>  xfs_buf_lock+0x36/0xf0
>  xfs_buf_find+0x418/0x850
>  xfs_buf_get_map+0x47/0x380
>  xfs_buf_read_map+0x54/0x240
>  xfs_trans_read_buf_map+0x1bd/0x490
>  xfs_imap_to_bp+0x4f/0x70
>  xfs_iget+0x300/0xb40
>  xlog_recover_process_one_iunlink+0x4c/0x170
>  xlog_recover_process_iunlinks.isra.0+0xee/0x130
>  xlog_recover_finish+0x57/0x110
>  xfs_log_mount_finish+0xfc/0x1e0
>  xfs_mountfs+0x540/0x910
>  xfs_fs_fill_super+0x495/0x850
>  get_tree_bdev+0x171/0x270
>  xfs_fs_get_tree+0x15/0x20
>  vfs_get_tree+0x24/0xc0
>  path_mount+0x304/0xba0
>  __x64_sys_mount+0x108/0x140
>  do_syscall_64+0x35/0x80
>  entry_SYSCALL_64_after_hwframe+0x44/0xae
>  </TASK>
> task:xfsaild/pmem1   state:D stack:14544 pid:324525 ppid:     2 flags:0x00004000
> Call Trace:
>  <TASK>
>  __schedule+0x30d/0x9e0
>  schedule+0x55/0xd0
>  io_schedule+0x4b/0x80
>  xfs_buf_wait_unpin+0x9e/0xf0
>  __xfs_buf_submit+0x14a/0x230
>  xfs_buf_delwri_submit_buffers+0x107/0x280
>  xfs_buf_delwri_submit_nowait+0x10/0x20
>  xfsaild+0x27e/0x9d0
>  kthread+0xf6/0x120
>  ret_from_fork+0x1f/0x30
>
> We have the mount process waiting on an inode cluster buffer read,
> inodegc doing unlink waiting on the same inode cluster buffer, and
> the AIL push thread blocked in writeback waiting for the inode to
> become unpinned.

... "waiting for the inode cluster buffer to become unpinned"?.

Periodic flushing of the current CIL context by the log worker task will
clearly break the deadlock described in this patch. I don't see any negative
side effects of the change presented here. Hence,

Reviewed-by: Chandan Babu R <chandan.babu@oracle.com>

>
> What has happened here is that the AIL push thread has raced with
> the inodegc process modifying, committing and pinning the inode
> cluster buffer here in xfs_buf_delwri_submit_buffers() here:
>
> 	blk_start_plug(&plug);
> 	list_for_each_entry_safe(bp, n, buffer_list, b_list) {
> 		if (!wait_list) {
> 			if (xfs_buf_ispinned(bp)) {
> 				pinned++;
> 				continue;
> 			}
> Here >>>>>>
> 			if (!xfs_buf_trylock(bp))
> 				continue;
>
> Basically, the AIL has found the buffer wasn't pinned and got the
> lock without blocking, but then the buffer was pinned. This implies
> the processing here was pre-empted between the pin check and the
> lock, because the pin count can only be increased while holding the
> buffer locked. Hence when it has gone to submit the IO, it has
> blocked waiting for the buffer to be unpinned.
>
> With all executing threads now waiting on the buffer to be unpinned,
> we normally get out of situations like this via the background log
> worker issuing a log force which will unpinned stuck buffers like
> this. But at this point in recovery, we haven't started the log
> worker. In fact, the first thing we do after processing intents and
> unlinked inodes is *start the log worker*. IOWs, we start it too
> late to have it break deadlocks like this.
>
> Avoid this and any other similar deadlock vectors in intent and
> unlinked inode recovery by starting the log worker before we recover
> intents and unlinked inodes. This part of recovery runs as though
> the filesystem is fully active, so we really should have the same
> infrastructure running as we normally do at runtime.
>
> Signed-off-by: Dave Chinner <dchinner@redhat.com>
> Reviewed-by: Darrick J. Wong <djwong@kernel.org>
> ---
>  fs/xfs/xfs_log.c | 3 +--
>  1 file changed, 1 insertion(+), 2 deletions(-)
>
> diff --git a/fs/xfs/xfs_log.c b/fs/xfs/xfs_log.c
> index 9a49acd94516..b0e05fa902d4 100644
> --- a/fs/xfs/xfs_log.c
> +++ b/fs/xfs/xfs_log.c
> @@ -833,10 +833,9 @@ xfs_log_mount_finish(
>  	 * mount failure occurs.
>  	 */
>  	mp->m_super->s_flags |= SB_ACTIVE;
> +	xfs_log_work_queue(mp);
>  	if (xlog_recovery_needed(log))
>  		error = xlog_recover_finish(log);
> -	if (!error)
> -		xfs_log_work_queue(mp);
>  	mp->m_super->s_flags &= ~SB_ACTIVE;
>  	evict_inodes(mp->m_super);


-- 
chandan

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

* Re: [PATCH 2/7] xfs: check buffer pin state after locking in delwri_submit
  2022-03-15  6:42 ` [PATCH 2/7] xfs: check buffer pin state after locking in delwri_submit Dave Chinner
@ 2022-03-15 10:04   ` Chandan Babu R
  2022-03-15 19:13   ` Darrick J. Wong
  1 sibling, 0 replies; 29+ messages in thread
From: Chandan Babu R @ 2022-03-15 10:04 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs

On 15 Mar 2022 at 12:12, Dave Chinner wrote:
> From: Dave Chinner <dchinner@redhat.com>
>
> AIL flushing can get stuck here:
>
> [316649.005769] INFO: task xfsaild/pmem1:324525 blocked for more than 123 seconds.
> [316649.007807]       Not tainted 5.17.0-rc6-dgc+ #975
> [316649.009186] "echo 0 > /proc/sys/kernel/hung_task_timeout_secs" disables this message.
> [316649.011720] task:xfsaild/pmem1   state:D stack:14544 pid:324525 ppid:     2 flags:0x00004000
> [316649.014112] Call Trace:
> [316649.014841]  <TASK>
> [316649.015492]  __schedule+0x30d/0x9e0
> [316649.017745]  schedule+0x55/0xd0
> [316649.018681]  io_schedule+0x4b/0x80
> [316649.019683]  xfs_buf_wait_unpin+0x9e/0xf0
> [316649.021850]  __xfs_buf_submit+0x14a/0x230
> [316649.023033]  xfs_buf_delwri_submit_buffers+0x107/0x280
> [316649.024511]  xfs_buf_delwri_submit_nowait+0x10/0x20
> [316649.025931]  xfsaild+0x27e/0x9d0
> [316649.028283]  kthread+0xf6/0x120
> [316649.030602]  ret_from_fork+0x1f/0x30
>
> in the situation where flushing gets preempted between the unpin
> check and the buffer trylock under nowait conditions:
>
> 	blk_start_plug(&plug);
> 	list_for_each_entry_safe(bp, n, buffer_list, b_list) {
> 		if (!wait_list) {
> 			if (xfs_buf_ispinned(bp)) {
> 				pinned++;
> 				continue;
> 			}
> Here >>>>>>
> 			if (!xfs_buf_trylock(bp))
> 				continue;
>
> This means submission is stuck until something else triggers a log
> force to unpin the buffer.
>
> To get onto the delwri list to begin with, the buffer pin state has
> already been checked, and hence it's relatively rare we get a race
> between flushing and encountering a pinned buffer in delwri
> submission to begin with. Further, to increase the pin count the
> buffer has to be locked, so the only way we can hit this race
> without failing the trylock is to be preempted between the pincount
> check seeing zero and the trylock being run.
>
> Hence to avoid this problem, just invert the order of trylock vs
> pin check. We shouldn't hit that many pinned buffers here, so
> optimising away the trylock for pinned buffers should not matter for
> performance at all.

Looks good to me from the perspective of logical correctness.

Reviewed-by: Chandan Babu R <chandan.babu@oracle.com>

>
> Signed-off-by: Dave Chinner <dchinner@redhat.com>
> ---
>  fs/xfs/xfs_buf.c | 5 +++--
>  1 file changed, 3 insertions(+), 2 deletions(-)
>
> diff --git a/fs/xfs/xfs_buf.c b/fs/xfs/xfs_buf.c
> index b45e0d50a405..8867f143598e 100644
> --- a/fs/xfs/xfs_buf.c
> +++ b/fs/xfs/xfs_buf.c
> @@ -2094,12 +2094,13 @@ xfs_buf_delwri_submit_buffers(
>  	blk_start_plug(&plug);
>  	list_for_each_entry_safe(bp, n, buffer_list, b_list) {
>  		if (!wait_list) {
> +			if (!xfs_buf_trylock(bp))
> +				continue;
>  			if (xfs_buf_ispinned(bp)) {
> +				xfs_buf_unlock(bp);
>  				pinned++;
>  				continue;
>  			}
> -			if (!xfs_buf_trylock(bp))
> -				continue;
>  		} else {
>  			xfs_buf_lock(bp);
>  		}


-- 
chandan

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

* Re: [PATCH 3/7] xfs: xfs_ail_push_all_sync() stalls when racing with updates
  2022-03-15  6:42 ` [PATCH 3/7] xfs: xfs_ail_push_all_sync() stalls when racing with updates Dave Chinner
@ 2022-03-15 15:14   ` Chandan Babu R
  2022-03-15 19:17   ` Darrick J. Wong
  1 sibling, 0 replies; 29+ messages in thread
From: Chandan Babu R @ 2022-03-15 15:14 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs

On 15 Mar 2022 at 12:12, Dave Chinner wrote:
> From: Dave Chinner <dchinner@redhat.com>
>
> xfs_ail_push_all_sync() has a loop like this:
>
> while max_ail_lsn {
> 	prepare_to_wait(ail_empty)
> 	target = max_ail_lsn
> 	wake_up(ail_task);
> 	schedule()
> }
>
> Which is designed to sleep until the AIL is emptied. When
> xfs_ail_finish_update() moves the tail of the log, it does:

... xfs_ail_update_finish() ...

>
> 	if (list_empty(&ailp->ail_head))
> 		wake_up_all(&ailp->ail_empty);
>
> So it will only wake up the sync push waiter when the AIL goes
> empty. If, by the time the push waiter has woken, the AIL has more
> in it, it will reset the target, wake the push task and go back to
> sleep.
>
> The problem here is that if the AIL is having items added to it
> when xfs_ail_push_all_sync() is called, then they may get inserted
> into the AIL at a LSN higher than the target LSN. At this point,
> xfsaild_push() will see that the target is X, the item LSNs are
> (X+N) and skip over them, hence never pushing the out.
>
> The result of this the AIL will not get emptied by the AIL push
> thread, hence xfs_ail_finish_update() will never see the AIL being
> empty even if it moves the tail. Hence xfs_ail_push_all_sync() never
> gets woken and hence cannot update the push target to capture the
> items beyond the current target on the LSN.
>
> This is a TOCTOU type of issue so the way to avoid it is to not
> use the push target at all for sync pushes. We know that a sync push
> is being requested by the fact the ail_empty wait queue is active,
> hence the xfsaild can just set the target to max_ail_lsn on every
> push that we see the wait queue active. Hence we no longer will
> leave items on the AIL that are beyond the LSN sampled at the start
> of a sync push.
>

The fix seems to logically correct.

Reviewed-by: Chandan Babu R <chandan.babu@oracle.com>

> Signed-off-by: Dave Chinner <dchinner@redhat.com>
> ---
>  fs/xfs/xfs_trans_ail.c | 21 ++++++++++++++++-----
>  1 file changed, 16 insertions(+), 5 deletions(-)
>
> diff --git a/fs/xfs/xfs_trans_ail.c b/fs/xfs/xfs_trans_ail.c
> index 2a8c8dc54c95..1b52952097c1 100644
> --- a/fs/xfs/xfs_trans_ail.c
> +++ b/fs/xfs/xfs_trans_ail.c
> @@ -448,10 +448,22 @@ xfsaild_push(
>  
>  	spin_lock(&ailp->ail_lock);
>  
> -	/* barrier matches the ail_target update in xfs_ail_push() */
> -	smp_rmb();
> -	target = ailp->ail_target;
> -	ailp->ail_target_prev = target;
> +	/*
> +	 * If we have a sync push waiter, we always have to push till the AIL is
> +	 * empty. Update the target to point to the end of the AIL so that
> +	 * capture updates that occur after the sync push waiter has gone to
> +	 * sleep.
> +	 */
> +	if (waitqueue_active(&ailp->ail_empty)) {
> +		lip = xfs_ail_max(ailp);
> +		if (lip)
> +			target = lip->li_lsn;
> +	} else {
> +		/* barrier matches the ail_target update in xfs_ail_push() */
> +		smp_rmb();
> +		target = ailp->ail_target;
> +		ailp->ail_target_prev = target;
> +	}
>  
>  	/* we're done if the AIL is empty or our push has reached the end */
>  	lip = xfs_trans_ail_cursor_first(ailp, &cur, ailp->ail_last_pushed_lsn);
> @@ -724,7 +736,6 @@ xfs_ail_push_all_sync(
>  	spin_lock(&ailp->ail_lock);
>  	while ((lip = xfs_ail_max(ailp)) != NULL) {
>  		prepare_to_wait(&ailp->ail_empty, &wait, TASK_UNINTERRUPTIBLE);
> -		ailp->ail_target = lip->li_lsn;
>  		wake_up_process(ailp->ail_task);
>  		spin_unlock(&ailp->ail_lock);
>  		schedule();


-- 
chandan

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

* Re: [PATCH 2/7] xfs: check buffer pin state after locking in delwri_submit
  2022-03-15  6:42 ` [PATCH 2/7] xfs: check buffer pin state after locking in delwri_submit Dave Chinner
  2022-03-15 10:04   ` Chandan Babu R
@ 2022-03-15 19:13   ` Darrick J. Wong
  2022-03-15 21:11     ` Dave Chinner
  1 sibling, 1 reply; 29+ messages in thread
From: Darrick J. Wong @ 2022-03-15 19:13 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs

On Tue, Mar 15, 2022 at 05:42:36PM +1100, Dave Chinner wrote:
> From: Dave Chinner <dchinner@redhat.com>
> 
> AIL flushing can get stuck here:
> 
> [316649.005769] INFO: task xfsaild/pmem1:324525 blocked for more than 123 seconds.
> [316649.007807]       Not tainted 5.17.0-rc6-dgc+ #975
> [316649.009186] "echo 0 > /proc/sys/kernel/hung_task_timeout_secs" disables this message.
> [316649.011720] task:xfsaild/pmem1   state:D stack:14544 pid:324525 ppid:     2 flags:0x00004000
> [316649.014112] Call Trace:
> [316649.014841]  <TASK>
> [316649.015492]  __schedule+0x30d/0x9e0
> [316649.017745]  schedule+0x55/0xd0
> [316649.018681]  io_schedule+0x4b/0x80
> [316649.019683]  xfs_buf_wait_unpin+0x9e/0xf0
> [316649.021850]  __xfs_buf_submit+0x14a/0x230
> [316649.023033]  xfs_buf_delwri_submit_buffers+0x107/0x280
> [316649.024511]  xfs_buf_delwri_submit_nowait+0x10/0x20
> [316649.025931]  xfsaild+0x27e/0x9d0
> [316649.028283]  kthread+0xf6/0x120
> [316649.030602]  ret_from_fork+0x1f/0x30
> 
> in the situation where flushing gets preempted between the unpin
> check and the buffer trylock under nowait conditions:
> 
> 	blk_start_plug(&plug);
> 	list_for_each_entry_safe(bp, n, buffer_list, b_list) {
> 		if (!wait_list) {
> 			if (xfs_buf_ispinned(bp)) {
> 				pinned++;
> 				continue;
> 			}
> Here >>>>>>
> 			if (!xfs_buf_trylock(bp))
> 				continue;
> 
> This means submission is stuck until something else triggers a log
> force to unpin the buffer.
> 
> To get onto the delwri list to begin with, the buffer pin state has
> already been checked, and hence it's relatively rare we get a race
> between flushing and encountering a pinned buffer in delwri
> submission to begin with. Further, to increase the pin count the
> buffer has to be locked, so the only way we can hit this race
> without failing the trylock is to be preempted between the pincount
> check seeing zero and the trylock being run.
> 
> Hence to avoid this problem, just invert the order of trylock vs
> pin check. We shouldn't hit that many pinned buffers here, so
> optimising away the trylock for pinned buffers should not matter for
> performance at all.
> 
> Signed-off-by: Dave Chinner <dchinner@redhat.com>
> ---
>  fs/xfs/xfs_buf.c | 5 +++--
>  1 file changed, 3 insertions(+), 2 deletions(-)
> 
> diff --git a/fs/xfs/xfs_buf.c b/fs/xfs/xfs_buf.c
> index b45e0d50a405..8867f143598e 100644
> --- a/fs/xfs/xfs_buf.c
> +++ b/fs/xfs/xfs_buf.c
> @@ -2094,12 +2094,13 @@ xfs_buf_delwri_submit_buffers(
>  	blk_start_plug(&plug);
>  	list_for_each_entry_safe(bp, n, buffer_list, b_list) {
>  		if (!wait_list) {
> +			if (!xfs_buf_trylock(bp))
 +				continue;
>  			if (xfs_buf_ispinned(bp)) {
> +				xfs_buf_unlock(bp);
>  				pinned++;
>  				continue;

Hmm.  So I think this means that this function willl skip buffers that
are locked or pinned.  The only way that the AIL would encounter this
situation is when a buffer on its list is now locked by a reader thread
or is participating in a transaction.  In the reader case this is (one
hopes) ok because the reader won't block on the AIL.

The tx case is trickier -- transaction allocation can result in an AIL
push if the head is too close to the tail, right?  Ordinarily, the AIL
won't find itself unable to write a buffer that's pinning the log
because a transaction holds that buffer -- eventually that tx should
commit, which will unlock the buffer and allow the AIL to make some
progress.

But -- what if the frontend is running a chained transaction, and it
bjoin'd the buffer to the transaction, tried to roll the transaction,
and the chain runs out of permanent log reservation (because we've
rolled more than logcount times) and we have to wait for more log grant
space?  The regrant for the successor tx happens before the commit of
the old tx, so can we livelock the log in this way?

And doesn't this potential exist regardless of this patch?

I suspect the answers are 'yes' and 'yes', which means this patch is ok
to move forward, but this has been bugging me since the V1 of this
patch, which is where I got stuck. :/

--D

>  			}
> -			if (!xfs_buf_trylock(bp))
> -				continue;
>  		} else {
>  			xfs_buf_lock(bp);
>  		}
> -- 
> 2.35.1
> 

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

* Re: [PATCH 3/7] xfs: xfs_ail_push_all_sync() stalls when racing with updates
  2022-03-15  6:42 ` [PATCH 3/7] xfs: xfs_ail_push_all_sync() stalls when racing with updates Dave Chinner
  2022-03-15 15:14   ` Chandan Babu R
@ 2022-03-15 19:17   ` Darrick J. Wong
  2022-03-15 21:29     ` Dave Chinner
  1 sibling, 1 reply; 29+ messages in thread
From: Darrick J. Wong @ 2022-03-15 19:17 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs

On Tue, Mar 15, 2022 at 05:42:37PM +1100, Dave Chinner wrote:
> From: Dave Chinner <dchinner@redhat.com>
> 
> xfs_ail_push_all_sync() has a loop like this:
> 
> while max_ail_lsn {
> 	prepare_to_wait(ail_empty)
> 	target = max_ail_lsn
> 	wake_up(ail_task);
> 	schedule()
> }
> 
> Which is designed to sleep until the AIL is emptied. When
> xfs_ail_finish_update() moves the tail of the log, it does:
> 
> 	if (list_empty(&ailp->ail_head))
> 		wake_up_all(&ailp->ail_empty);
> 
> So it will only wake up the sync push waiter when the AIL goes
> empty. If, by the time the push waiter has woken, the AIL has more
> in it, it will reset the target, wake the push task and go back to
> sleep.
> 
> The problem here is that if the AIL is having items added to it
> when xfs_ail_push_all_sync() is called, then they may get inserted
> into the AIL at a LSN higher than the target LSN. At this point,
> xfsaild_push() will see that the target is X, the item LSNs are
> (X+N) and skip over them, hence never pushing the out.
> 
> The result of this the AIL will not get emptied by the AIL push
> thread, hence xfs_ail_finish_update() will never see the AIL being
> empty even if it moves the tail. Hence xfs_ail_push_all_sync() never
> gets woken and hence cannot update the push target to capture the
> items beyond the current target on the LSN.
> 
> This is a TOCTOU type of issue so the way to avoid it is to not
> use the push target at all for sync pushes. We know that a sync push
> is being requested by the fact the ail_empty wait queue is active,
> hence the xfsaild can just set the target to max_ail_lsn on every
> push that we see the wait queue active. Hence we no longer will
> leave items on the AIL that are beyond the LSN sampled at the start
> of a sync push.
> 
> Signed-off-by: Dave Chinner <dchinner@redhat.com>
> ---
>  fs/xfs/xfs_trans_ail.c | 21 ++++++++++++++++-----
>  1 file changed, 16 insertions(+), 5 deletions(-)
> 
> diff --git a/fs/xfs/xfs_trans_ail.c b/fs/xfs/xfs_trans_ail.c
> index 2a8c8dc54c95..1b52952097c1 100644
> --- a/fs/xfs/xfs_trans_ail.c
> +++ b/fs/xfs/xfs_trans_ail.c
> @@ -448,10 +448,22 @@ xfsaild_push(
>  
>  	spin_lock(&ailp->ail_lock);
>  
> -	/* barrier matches the ail_target update in xfs_ail_push() */
> -	smp_rmb();
> -	target = ailp->ail_target;
> -	ailp->ail_target_prev = target;
> +	/*
> +	 * If we have a sync push waiter, we always have to push till the AIL is
> +	 * empty. Update the target to point to the end of the AIL so that
> +	 * capture updates that occur after the sync push waiter has gone to
> +	 * sleep.
> +	 */
> +	if (waitqueue_active(&ailp->ail_empty)) {
> +		lip = xfs_ail_max(ailp);
> +		if (lip)
> +			target = lip->li_lsn;
> +	} else {
> +		/* barrier matches the ail_target update in xfs_ail_push() */
> +		smp_rmb();

Doesn't the spin_lock provide the required rmb?  I think it's
unnecessary given that, but I also don't think it hurts anything, so:

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

--D

> +		target = ailp->ail_target;
> +		ailp->ail_target_prev = target;
> +	}
>  
>  	/* we're done if the AIL is empty or our push has reached the end */
>  	lip = xfs_trans_ail_cursor_first(ailp, &cur, ailp->ail_last_pushed_lsn);
> @@ -724,7 +736,6 @@ xfs_ail_push_all_sync(
>  	spin_lock(&ailp->ail_lock);
>  	while ((lip = xfs_ail_max(ailp)) != NULL) {
>  		prepare_to_wait(&ailp->ail_empty, &wait, TASK_UNINTERRUPTIBLE);
> -		ailp->ail_target = lip->li_lsn;
>  		wake_up_process(ailp->ail_task);
>  		spin_unlock(&ailp->ail_lock);
>  		schedule();
> -- 
> 2.35.1
> 

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

* Re: [PATCH 4/7] xfs: async CIL flushes need pending pushes to be made stable
  2022-03-15  6:42 ` [PATCH 4/7] xfs: async CIL flushes need pending pushes to be made stable Dave Chinner
@ 2022-03-15 19:36   ` Darrick J. Wong
  2022-03-15 21:47     ` Dave Chinner
  2022-03-16 10:34   ` Chandan Babu R
  1 sibling, 1 reply; 29+ messages in thread
From: Darrick J. Wong @ 2022-03-15 19:36 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs

On Tue, Mar 15, 2022 at 05:42:38PM +1100, Dave Chinner wrote:
> From: Dave Chinner <dchinner@redhat.com>
> 
> When the AIL tries to flush the CIL, it relies on the CIL push
> ending up on stable storage without having to wait for and
> manipulate iclog state directly. However, if there is already a
> pending CIL push when the AIL tries to flush the CIL, it won't set
> the cil->xc_push_commit_stable flag and so the CIL push will not
> actively flush the commit record iclog.
> 
> generic/530 when run on a single CPU test VM can trigger this fairly
> reliably. This test exercises unlinked inode recovery, and can
> result in inodes being pinned in memory by ongoing modifications to
> the inode cluster buffer to record unlinked list modifications. As a
> result, the first inode unlinked in a buffer can pin the tail of the
> log whilst the inode cluster buffer is pinned by the current
> checkpoint that has been pushed but isn't on stable storage because
> because the cil->xc_push_commit_stable was not set. This results in
> the log/AIL effectively deadlocking until something triggers the
> commit record iclog to be pushed to stable storage (i.e. the
> periodic log worker calling xfs_log_force()).
> 
> The fix is two-fold - first we should always set the
> cil->xc_push_commit_stable when xlog_cil_flush() is called,
> regardless of whether there is already a pending push or not.
> 
> Second, if the CIL is empty, we should trigger an iclog flush to
> ensure that the iclogs of the last checkpoint have actually been
> submitted to disk as that checkpoint may not have been run under
> stable completion constraints.

Can it ever be the case that the CIL is not empty but the last
checkpoint wasn't committed to disk?  Let's say someone else
commits a transaction after the worker samples xc_push_commit_stable?

IOWs, why does a not-empty CIL mean that the last checkpoint is on disk?

> Reported-and-tested-by: Matthew Wilcox <willy@infradead.org>

MMMM testing. :D

--D

> Fixes: 0020a190cf3e ("xfs: AIL needs asynchronous CIL forcing")
> Signed-off-by: Dave Chinner <dchinner@redhat.com>
> ---
>  fs/xfs/xfs_log_cil.c | 22 +++++++++++++++++++---
>  1 file changed, 19 insertions(+), 3 deletions(-)
> 
> diff --git a/fs/xfs/xfs_log_cil.c b/fs/xfs/xfs_log_cil.c
> index 3d8ebf2a1e55..48b16a5feb27 100644
> --- a/fs/xfs/xfs_log_cil.c
> +++ b/fs/xfs/xfs_log_cil.c
> @@ -1369,18 +1369,27 @@ xlog_cil_push_now(
>  	if (!async)
>  		flush_workqueue(cil->xc_push_wq);
>  
> +	spin_lock(&cil->xc_push_lock);
> +
> +	/*
> +	 * If this is an async flush request, we always need to set the
> +	 * xc_push_commit_stable flag even if something else has already queued
> +	 * a push. The flush caller is asking for the CIL to be on stable
> +	 * storage when the next push completes, so regardless of who has queued
> +	 * the push, the flush requires stable semantics from it.
> +	 */
> +	cil->xc_push_commit_stable = async;
> +
>  	/*
>  	 * If the CIL is empty or we've already pushed the sequence then
> -	 * there's no work we need to do.
> +	 * there's no more work that we need to do.
>  	 */
> -	spin_lock(&cil->xc_push_lock);
>  	if (list_empty(&cil->xc_cil) || push_seq <= cil->xc_push_seq) {
>  		spin_unlock(&cil->xc_push_lock);
>  		return;
>  	}
>  
>  	cil->xc_push_seq = push_seq;
> -	cil->xc_push_commit_stable = async;
>  	queue_work(cil->xc_push_wq, &cil->xc_ctx->push_work);
>  	spin_unlock(&cil->xc_push_lock);
>  }
> @@ -1520,6 +1529,13 @@ xlog_cil_flush(
>  
>  	trace_xfs_log_force(log->l_mp, seq, _RET_IP_);
>  	xlog_cil_push_now(log, seq, true);
> +
> +	/*
> +	 * If the CIL is empty, make sure that any previous checkpoint that may
> +	 * still be in an active iclog is pushed to stable storage.
> +	 */
> +	if (list_empty(&log->l_cilp->xc_cil))
> +		xfs_log_force(log->l_mp, 0);
>  }
>  
>  /*
> -- 
> 2.35.1
> 

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

* Re: [PATCH 5/7] xfs: log items should have a xlog pointer, not a mount
  2022-03-15  6:42 ` [PATCH 5/7] xfs: log items should have a xlog pointer, not a mount Dave Chinner
@ 2022-03-15 19:37   ` Darrick J. Wong
  2022-03-16 11:06   ` Chandan Babu R
  1 sibling, 0 replies; 29+ messages in thread
From: Darrick J. Wong @ 2022-03-15 19:37 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs

On Tue, Mar 15, 2022 at 05:42:39PM +1100, Dave Chinner wrote:
> From: Dave Chinner <dchinner@redhat.com>
> 
> Log items belong to the log, not the xfs_mount. Convert the mount
> pointer in the log item to a xlog pointer in preparation for
> upcoming log centric changes to the log items.
> 
> Signed-off-by: Dave Chinner <dchinner@redhat.com>

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

--D

> ---
>  fs/xfs/xfs_bmap_item.c     | 2 +-
>  fs/xfs/xfs_buf_item.c      | 5 +++--
>  fs/xfs/xfs_extfree_item.c  | 2 +-
>  fs/xfs/xfs_log.c           | 2 +-
>  fs/xfs/xfs_log_cil.c       | 2 +-
>  fs/xfs/xfs_refcount_item.c | 2 +-
>  fs/xfs/xfs_rmap_item.c     | 2 +-
>  fs/xfs/xfs_trace.h         | 4 ++--
>  fs/xfs/xfs_trans.c         | 2 +-
>  fs/xfs/xfs_trans.h         | 3 ++-
>  10 files changed, 14 insertions(+), 12 deletions(-)
> 
> diff --git a/fs/xfs/xfs_bmap_item.c b/fs/xfs/xfs_bmap_item.c
> index fa710067aac2..65ac261b3b28 100644
> --- a/fs/xfs/xfs_bmap_item.c
> +++ b/fs/xfs/xfs_bmap_item.c
> @@ -476,7 +476,7 @@ xfs_bui_item_recover(
>  	struct xfs_bui_log_item		*buip = BUI_ITEM(lip);
>  	struct xfs_trans		*tp;
>  	struct xfs_inode		*ip = NULL;
> -	struct xfs_mount		*mp = lip->li_mountp;
> +	struct xfs_mount		*mp = lip->li_log->l_mp;
>  	struct xfs_map_extent		*bmap;
>  	struct xfs_bud_log_item		*budp;
>  	xfs_filblks_t			count;
> diff --git a/fs/xfs/xfs_buf_item.c b/fs/xfs/xfs_buf_item.c
> index a7a8e4528881..522d450a94b1 100644
> --- a/fs/xfs/xfs_buf_item.c
> +++ b/fs/xfs/xfs_buf_item.c
> @@ -21,6 +21,7 @@
>  #include "xfs_dquot.h"
>  #include "xfs_trace.h"
>  #include "xfs_log.h"
> +#include "xfs_log_priv.h"
>  
>  
>  struct kmem_cache	*xfs_buf_item_cache;
> @@ -428,7 +429,7 @@ xfs_buf_item_format(
>  	 * occurs during recovery.
>  	 */
>  	if (bip->bli_flags & XFS_BLI_INODE_BUF) {
> -		if (xfs_has_v3inodes(lip->li_mountp) ||
> +		if (xfs_has_v3inodes(lip->li_log->l_mp) ||
>  		    !((bip->bli_flags & XFS_BLI_INODE_ALLOC_BUF) &&
>  		      xfs_log_item_in_current_chkpt(lip)))
>  			bip->__bli_format.blf_flags |= XFS_BLF_INODE_BUF;
> @@ -616,7 +617,7 @@ xfs_buf_item_put(
>  	 * that case, the bli is freed on buffer writeback completion.
>  	 */
>  	aborted = test_bit(XFS_LI_ABORTED, &lip->li_flags) ||
> -		  xfs_is_shutdown(lip->li_mountp);
> +			xlog_is_shutdown(lip->li_log);
>  	dirty = bip->bli_flags & XFS_BLI_DIRTY;
>  	if (dirty && !aborted)
>  		return false;
> diff --git a/fs/xfs/xfs_extfree_item.c b/fs/xfs/xfs_extfree_item.c
> index 36eeac9413f5..893a7dd15cbb 100644
> --- a/fs/xfs/xfs_extfree_item.c
> +++ b/fs/xfs/xfs_extfree_item.c
> @@ -615,7 +615,7 @@ xfs_efi_item_recover(
>  	struct list_head		*capture_list)
>  {
>  	struct xfs_efi_log_item		*efip = EFI_ITEM(lip);
> -	struct xfs_mount		*mp = lip->li_mountp;
> +	struct xfs_mount		*mp = lip->li_log->l_mp;
>  	struct xfs_efd_log_item		*efdp;
>  	struct xfs_trans		*tp;
>  	struct xfs_extent		*extp;
> diff --git a/fs/xfs/xfs_log.c b/fs/xfs/xfs_log.c
> index b0e05fa902d4..5c4ef45f42d2 100644
> --- a/fs/xfs/xfs_log.c
> +++ b/fs/xfs/xfs_log.c
> @@ -1136,7 +1136,7 @@ xfs_log_item_init(
>  	int			type,
>  	const struct xfs_item_ops *ops)
>  {
> -	item->li_mountp = mp;
> +	item->li_log = mp->m_log;
>  	item->li_ailp = mp->m_ail;
>  	item->li_type = type;
>  	item->li_ops = ops;
> diff --git a/fs/xfs/xfs_log_cil.c b/fs/xfs/xfs_log_cil.c
> index 48b16a5feb27..e9b80036268a 100644
> --- a/fs/xfs/xfs_log_cil.c
> +++ b/fs/xfs/xfs_log_cil.c
> @@ -76,7 +76,7 @@ bool
>  xfs_log_item_in_current_chkpt(
>  	struct xfs_log_item *lip)
>  {
> -	return xlog_item_in_current_chkpt(lip->li_mountp->m_log->l_cilp, lip);
> +	return xlog_item_in_current_chkpt(lip->li_log->l_cilp, lip);
>  }
>  
>  /*
> diff --git a/fs/xfs/xfs_refcount_item.c b/fs/xfs/xfs_refcount_item.c
> index d4632f2ceb89..1b82b818f515 100644
> --- a/fs/xfs/xfs_refcount_item.c
> +++ b/fs/xfs/xfs_refcount_item.c
> @@ -468,7 +468,7 @@ xfs_cui_item_recover(
>  	struct xfs_cud_log_item		*cudp;
>  	struct xfs_trans		*tp;
>  	struct xfs_btree_cur		*rcur = NULL;
> -	struct xfs_mount		*mp = lip->li_mountp;
> +	struct xfs_mount		*mp = lip->li_log->l_mp;
>  	xfs_fsblock_t			new_fsb;
>  	xfs_extlen_t			new_len;
>  	unsigned int			refc_type;
> diff --git a/fs/xfs/xfs_rmap_item.c b/fs/xfs/xfs_rmap_item.c
> index cb0490919b2c..546bd824cdf7 100644
> --- a/fs/xfs/xfs_rmap_item.c
> +++ b/fs/xfs/xfs_rmap_item.c
> @@ -523,7 +523,7 @@ xfs_rui_item_recover(
>  	struct xfs_rud_log_item		*rudp;
>  	struct xfs_trans		*tp;
>  	struct xfs_btree_cur		*rcur = NULL;
> -	struct xfs_mount		*mp = lip->li_mountp;
> +	struct xfs_mount		*mp = lip->li_log->l_mp;
>  	enum xfs_rmap_intent_type	type;
>  	xfs_exntst_t			state;
>  	int				i;
> diff --git a/fs/xfs/xfs_trace.h b/fs/xfs/xfs_trace.h
> index 585bd9853b6b..cc69b7c066e8 100644
> --- a/fs/xfs/xfs_trace.h
> +++ b/fs/xfs/xfs_trace.h
> @@ -1308,7 +1308,7 @@ DECLARE_EVENT_CLASS(xfs_log_item_class,
>  		__field(xfs_lsn_t, lsn)
>  	),
>  	TP_fast_assign(
> -		__entry->dev = lip->li_mountp->m_super->s_dev;
> +		__entry->dev = lip->li_log->l_mp->m_super->s_dev;
>  		__entry->lip = lip;
>  		__entry->type = lip->li_type;
>  		__entry->flags = lip->li_flags;
> @@ -1364,7 +1364,7 @@ DECLARE_EVENT_CLASS(xfs_ail_class,
>  		__field(xfs_lsn_t, new_lsn)
>  	),
>  	TP_fast_assign(
> -		__entry->dev = lip->li_mountp->m_super->s_dev;
> +		__entry->dev = lip->li_log->l_mp->m_super->s_dev;
>  		__entry->lip = lip;
>  		__entry->type = lip->li_type;
>  		__entry->flags = lip->li_flags;
> diff --git a/fs/xfs/xfs_trans.c b/fs/xfs/xfs_trans.c
> index 82590007e6c5..de87fb136b51 100644
> --- a/fs/xfs/xfs_trans.c
> +++ b/fs/xfs/xfs_trans.c
> @@ -646,7 +646,7 @@ xfs_trans_add_item(
>  	struct xfs_trans	*tp,
>  	struct xfs_log_item	*lip)
>  {
> -	ASSERT(lip->li_mountp == tp->t_mountp);
> +	ASSERT(lip->li_log == tp->t_mountp->m_log);
>  	ASSERT(lip->li_ailp == tp->t_mountp->m_ail);
>  	ASSERT(list_empty(&lip->li_trans));
>  	ASSERT(!test_bit(XFS_LI_DIRTY, &lip->li_flags));
> diff --git a/fs/xfs/xfs_trans.h b/fs/xfs/xfs_trans.h
> index 85dca2c9b559..1c5c5d7f522f 100644
> --- a/fs/xfs/xfs_trans.h
> +++ b/fs/xfs/xfs_trans.h
> @@ -8,6 +8,7 @@
>  
>  /* kernel only transaction subsystem defines */
>  
> +struct xlog;
>  struct xfs_buf;
>  struct xfs_buftarg;
>  struct xfs_efd_log_item;
> @@ -31,7 +32,7 @@ struct xfs_log_item {
>  	struct list_head		li_ail;		/* AIL pointers */
>  	struct list_head		li_trans;	/* transaction list */
>  	xfs_lsn_t			li_lsn;		/* last on-disk lsn */
> -	struct xfs_mount		*li_mountp;	/* ptr to fs mount */
> +	struct xlog			*li_log;
>  	struct xfs_ail			*li_ailp;	/* ptr to AIL */
>  	uint				li_type;	/* item type */
>  	unsigned long			li_flags;	/* misc flags */
> -- 
> 2.35.1
> 

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

* Re: [PATCH 6/7] xfs: AIL should be log centric
  2022-03-15  6:42 ` [PATCH 6/7] xfs: AIL should be log centric Dave Chinner
@ 2022-03-15 19:39   ` Darrick J. Wong
  2022-03-16 11:12   ` Chandan Babu R
  1 sibling, 0 replies; 29+ messages in thread
From: Darrick J. Wong @ 2022-03-15 19:39 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs

On Tue, Mar 15, 2022 at 05:42:40PM +1100, Dave Chinner wrote:
> From: Dave Chinner <dchinner@redhat.com>
> 
> The AIL operates purely on log items, so it is a log centric
> subsystem. Divorce it from the xfs_mount and instead have it pass
> around xlog pointers.
> 
> Signed-off-by: Dave Chinner <dchinner@redhat.com>

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

--D

> ---
>  fs/xfs/xfs_trans.c      |  2 +-
>  fs/xfs/xfs_trans_ail.c  | 26 +++++++++++++-------------
>  fs/xfs/xfs_trans_priv.h |  3 ++-
>  3 files changed, 16 insertions(+), 15 deletions(-)
> 
> diff --git a/fs/xfs/xfs_trans.c b/fs/xfs/xfs_trans.c
> index de87fb136b51..831d355c3258 100644
> --- a/fs/xfs/xfs_trans.c
> +++ b/fs/xfs/xfs_trans.c
> @@ -773,7 +773,7 @@ xfs_trans_committed_bulk(
>  		 * object into the AIL as we are in a shutdown situation.
>  		 */
>  		if (aborted) {
> -			ASSERT(xfs_is_shutdown(ailp->ail_mount));
> +			ASSERT(xlog_is_shutdown(ailp->ail_log));
>  			if (lip->li_ops->iop_unpin)
>  				lip->li_ops->iop_unpin(lip, 1);
>  			continue;
> diff --git a/fs/xfs/xfs_trans_ail.c b/fs/xfs/xfs_trans_ail.c
> index 1b52952097c1..c2ccb98c7bcd 100644
> --- a/fs/xfs/xfs_trans_ail.c
> +++ b/fs/xfs/xfs_trans_ail.c
> @@ -398,7 +398,7 @@ xfsaild_push_item(
>  	 * If log item pinning is enabled, skip the push and track the item as
>  	 * pinned. This can help induce head-behind-tail conditions.
>  	 */
> -	if (XFS_TEST_ERROR(false, ailp->ail_mount, XFS_ERRTAG_LOG_ITEM_PIN))
> +	if (XFS_TEST_ERROR(false, ailp->ail_log->l_mp, XFS_ERRTAG_LOG_ITEM_PIN))
>  		return XFS_ITEM_PINNED;
>  
>  	/*
> @@ -418,7 +418,7 @@ static long
>  xfsaild_push(
>  	struct xfs_ail		*ailp)
>  {
> -	xfs_mount_t		*mp = ailp->ail_mount;
> +	struct xfs_mount	*mp = ailp->ail_log->l_mp;
>  	struct xfs_ail_cursor	cur;
>  	struct xfs_log_item	*lip;
>  	xfs_lsn_t		lsn;
> @@ -443,7 +443,7 @@ xfsaild_push(
>  		ailp->ail_log_flush = 0;
>  
>  		XFS_STATS_INC(mp, xs_push_ail_flush);
> -		xlog_cil_flush(mp->m_log);
> +		xlog_cil_flush(ailp->ail_log);
>  	}
>  
>  	spin_lock(&ailp->ail_lock);
> @@ -632,7 +632,7 @@ xfsaild(
>  			 * opportunity to release such buffers from the queue.
>  			 */
>  			ASSERT(list_empty(&ailp->ail_buf_list) ||
> -			       xfs_is_shutdown(ailp->ail_mount));
> +			       xlog_is_shutdown(ailp->ail_log));
>  			xfs_buf_delwri_cancel(&ailp->ail_buf_list);
>  			break;
>  		}
> @@ -695,7 +695,7 @@ xfs_ail_push(
>  	struct xfs_log_item	*lip;
>  
>  	lip = xfs_ail_min(ailp);
> -	if (!lip || xfs_is_shutdown(ailp->ail_mount) ||
> +	if (!lip || xlog_is_shutdown(ailp->ail_log) ||
>  	    XFS_LSN_CMP(threshold_lsn, ailp->ail_target) <= 0)
>  		return;
>  
> @@ -751,7 +751,7 @@ xfs_ail_update_finish(
>  	struct xfs_ail		*ailp,
>  	xfs_lsn_t		old_lsn) __releases(ailp->ail_lock)
>  {
> -	struct xfs_mount	*mp = ailp->ail_mount;
> +	struct xlog		*log = ailp->ail_log;
>  
>  	/* if the tail lsn hasn't changed, don't do updates or wakeups. */
>  	if (!old_lsn || old_lsn == __xfs_ail_min_lsn(ailp)) {
> @@ -759,13 +759,13 @@ xfs_ail_update_finish(
>  		return;
>  	}
>  
> -	if (!xfs_is_shutdown(mp))
> -		xlog_assign_tail_lsn_locked(mp);
> +	if (!xlog_is_shutdown(log))
> +		xlog_assign_tail_lsn_locked(log->l_mp);
>  
>  	if (list_empty(&ailp->ail_head))
>  		wake_up_all(&ailp->ail_empty);
>  	spin_unlock(&ailp->ail_lock);
> -	xfs_log_space_wake(mp);
> +	xfs_log_space_wake(log->l_mp);
>  }
>  
>  /*
> @@ -873,13 +873,13 @@ xfs_trans_ail_delete(
>  	int			shutdown_type)
>  {
>  	struct xfs_ail		*ailp = lip->li_ailp;
> -	struct xfs_mount	*mp = ailp->ail_mount;
> +	struct xfs_mount	*mp = ailp->ail_log->l_mp;
>  	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 && !xfs_is_shutdown(mp)) {
> +		if (shutdown_type && !xlog_is_shutdown(ailp->ail_log)) {
>  			xfs_alert_tag(mp, XFS_PTAG_AILDELETE,
>  	"%s: attempting to delete a log item that is not in the AIL",
>  					__func__);
> @@ -904,7 +904,7 @@ xfs_trans_ail_init(
>  	if (!ailp)
>  		return -ENOMEM;
>  
> -	ailp->ail_mount = mp;
> +	ailp->ail_log = mp->m_log;
>  	INIT_LIST_HEAD(&ailp->ail_head);
>  	INIT_LIST_HEAD(&ailp->ail_cursors);
>  	spin_lock_init(&ailp->ail_lock);
> @@ -912,7 +912,7 @@ xfs_trans_ail_init(
>  	init_waitqueue_head(&ailp->ail_empty);
>  
>  	ailp->ail_task = kthread_run(xfsaild, ailp, "xfsaild/%s",
> -			ailp->ail_mount->m_super->s_id);
> +				mp->m_super->s_id);
>  	if (IS_ERR(ailp->ail_task))
>  		goto out_free_ailp;
>  
> diff --git a/fs/xfs/xfs_trans_priv.h b/fs/xfs/xfs_trans_priv.h
> index 3004aeac9110..f0d79a9050ba 100644
> --- a/fs/xfs/xfs_trans_priv.h
> +++ b/fs/xfs/xfs_trans_priv.h
> @@ -6,6 +6,7 @@
>  #ifndef __XFS_TRANS_PRIV_H__
>  #define	__XFS_TRANS_PRIV_H__
>  
> +struct xlog;
>  struct xfs_log_item;
>  struct xfs_mount;
>  struct xfs_trans;
> @@ -50,7 +51,7 @@ struct xfs_ail_cursor {
>   * Eventually we need to drive the locking in here as well.
>   */
>  struct xfs_ail {
> -	struct xfs_mount	*ail_mount;
> +	struct xlog		*ail_log;
>  	struct task_struct	*ail_task;
>  	struct list_head	ail_head;
>  	xfs_lsn_t		ail_target;
> -- 
> 2.35.1
> 

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

* Re: [PATCH 7/7] xfs: xfs_is_shutdown vs xlog_is_shutdown cage fight
  2022-03-15  6:42 ` [PATCH 7/7] xfs: xfs_is_shutdown vs xlog_is_shutdown cage fight Dave Chinner
@ 2022-03-15 20:03   ` Darrick J. Wong
  2022-03-15 22:20     ` Dave Chinner
  0 siblings, 1 reply; 29+ messages in thread
From: Darrick J. Wong @ 2022-03-15 20:03 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs

On Tue, Mar 15, 2022 at 05:42:41PM +1100, Dave Chinner wrote:
> From: Dave Chinner <dchinner@redhat.com>
> 
> I've been chasing a recent resurgence in generic/388 recovery
> failure and/or corruption events. The events have largely been

recoveryloop, the gift that keeps on giving...

> uninitialised inode chunks being tripped over in log recovery
> such as:
> 
>  XFS (pmem1): User initiated shutdown received.
>  pmem1: writeback error on inode 12621949, offset 1019904, sector 12968096
>  XFS (pmem1): Log I/O Error (0x6) detected at xfs_fs_goingdown+0xa3/0xf0 (fs/xfs/xfs_fsops.c:500).  Shutting down filesystem.
>  XFS (pmem1): Please unmount the filesystem and rectify the problem(s)
>  XFS (pmem1): Unmounting Filesystem
>  XFS (pmem1): Mounting V5 Filesystem
>  XFS (pmem1): Starting recovery (logdev: internal)
>  XFS (pmem1): bad inode magic/vsn daddr 8723584 #0 (magic=1818)
>  XFS (pmem1): Metadata corruption detected at xfs_inode_buf_verify+0x180/0x190, xfs_inode block 0x851c80 xfs_inode_buf_verify
>  XFS (pmem1): Unmount and run xfs_repair
>  XFS (pmem1): First 128 bytes of corrupted metadata buffer:
>  00000000: 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18  ................
>  00000010: 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18  ................
>  00000020: 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18  ................
>  00000030: 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18  ................
>  00000040: 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18  ................
>  00000050: 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18  ................
>  00000060: 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18  ................
>  00000070: 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18 18  ................
>  XFS (pmem1): metadata I/O error in "xlog_recover_items_pass2+0x52/0xc0" at daddr 0x851c80 len 32 error 117
>  XFS (pmem1): log mount/recovery failed: error -117
>  XFS (pmem1): log mount failed
> 
> There have been isolated random other issues, too - xfs_repair fails
> because it finds some corruption in symlink blocks, rmap
> inconsistencies, etc - but they are nowhere near as common as the
> uninitialised inode chunk failure.
> 
> The problem has clearly happened at runtime before recovery has run;
> I can see the ICREATE log item in the log shortly before the
> actively recovered range of the log. This means the ICREATE was
> definitely created and written to the log, but for some reason the
> tail of the log has been moved past the ordered buffer log item that
> tracks INODE_ALLOC buffers and, supposedly, prevents the tail of the
> log moving past the ICREATE log item before the inode chunk buffer
> is written to disk.
> 
> Tracing the fsstress processes that are running when the filesystem
> shut down immediately pin-pointed the problem:
> 
> user shutdown marks xfs_mount as shutdown
> 
>          godown-213341 [008]  6398.022871: console:              [ 6397.915392] XFS (pmem1): User initiated shutdown received.
> .....
> 
> aild tries to push ordered inode cluster buffer
> 
>   xfsaild/pmem1-213314 [001]  6398.022974: xfs_buf_trylock:      dev 259:1 daddr 0x851c80 bbcount 0x20 hold 16 pincount 0 lock 0 flags DONE|INODES|PAGES caller xfs_inode_item_push+0x8e
>   xfsaild/pmem1-213314 [001]  6398.022976: xfs_ilock_nowait:     dev 259:1 ino 0x851c80 flags ILOCK_SHARED caller xfs_iflush_cluster+0xae
> 
> xfs_iflush_cluster() checks xfs_is_shutdown(), returns true,
> calls xfs_iflush_abort() to kill writeback of the inode.
> Inode is removed from AIL, drops cluster buffer reference.
> 
>   xfsaild/pmem1-213314 [001]  6398.022977: xfs_ail_delete:       dev 259:1 lip 0xffff88880247ed80 old lsn 7/20344 new lsn 7/21000 type XFS_LI_INODE flags IN_AIL
>   xfsaild/pmem1-213314 [001]  6398.022978: xfs_buf_rele:         dev 259:1 daddr 0x851c80 bbcount 0x20 hold 17 pincount 0 lock 0 flags DONE|INODES|PAGES caller xfs_iflush_abort+0xd7
> 
> .....
> 
> All inodes on cluster buffer are aborted, then the cluster buffer
> itself is aborted and removed from the AIL *without writeback*:
> 
> xfsaild/pmem1-213314 [001]  6398.023011: xfs_buf_error_relse:  dev 259:1 daddr 0x851c80 bbcount 0x20 hold 2 pincount 0 lock 0 flags ASYNC|DONE|STALE|INODES|PAGES caller xfs_buf_ioend_fail+0x33
>    xfsaild/pmem1-213314 [001]  6398.023012: xfs_ail_delete:       dev 259:1 lip 0xffff8888053efde8 old lsn 7/20344 new lsn 7/20344 type XFS_LI_BUF flags IN_AIL
> 
> The inode buffer was at 7/20344 when it was removed from the AIL.
> 
>    xfsaild/pmem1-213314 [001]  6398.023012: xfs_buf_item_relse:   dev 259:1 daddr 0x851c80 bbcount 0x20 hold 2 pincount 0 lock 0 flags ASYNC|DONE|STALE|INODES|PAGES caller xfs_buf_item_done+0x31
>    xfsaild/pmem1-213314 [001]  6398.023012: xfs_buf_rele:         dev 259:1 daddr 0x851c80 bbcount 0x20 hold 2 pincount 0 lock 0 flags ASYNC|DONE|STALE|INODES|PAGES caller xfs_buf_item_relse+0x39
> 
> .....
> 
> Userspace is still running, doing stuff. an fsstress process runs
> syncfs() or sync() and we end up in sync_fs_one_sb() which issues
> a log force. This pushes on the CIL:
> 
>         fsstress-213322 [001]  6398.024430: xfs_fs_sync_fs:       dev 259:1 m_features 0x20000000019ff6e9 opstate (clean|shutdown|inodegc|blockgc) s_flags 0x70810000 caller sync_fs_one_sb+0x26
>         fsstress-213322 [001]  6398.024430: xfs_log_force:        dev 259:1 lsn 0x0 caller xfs_fs_sync_fs+0x82
>         fsstress-213322 [001]  6398.024430: xfs_log_force:        dev 259:1 lsn 0x5f caller xfs_log_force+0x7c
>            <...>-194402 [001]  6398.024467: kmem_alloc:           size 176 flags 0x14 caller xlog_cil_push_work+0x9f
> 
> And the CIL fills up iclogs with pending changes. This picks up
> the current tail from the AIL:
> 
>            <...>-194402 [001]  6398.024497: xlog_iclog_get_space: dev 259:1 state XLOG_STATE_ACTIVE refcnt 1 offset 0 lsn 0x0 flags  caller xlog_write+0x149
>            <...>-194402 [001]  6398.024498: xlog_iclog_switch:    dev 259:1 state XLOG_STATE_ACTIVE refcnt 1 offset 0 lsn 0x700005408 flags  caller xlog_state_get_iclog_space+0x37e
>            <...>-194402 [001]  6398.024521: xlog_iclog_release:   dev 259:1 state XLOG_STATE_WANT_SYNC refcnt 1 offset 32256 lsn 0x700005408 flags  caller xlog_write+0x5f9
>            <...>-194402 [001]  6398.024522: xfs_log_assign_tail_lsn: dev 259:1 new tail lsn 7/21000, old lsn 7/20344, last sync 7/21448
> 
> And it moves the tail of the log to 7/21000 from 7/20344. This
> *moves the tail of the log beyond the ICREATE transaction* that was
> at 7/20344 and pinned by the inode cluster buffer that was cancelled
> above.
> 
> ....
> 
>          godown-213341 [008]  6398.027005: xfs_force_shutdown:   dev 259:1 tag logerror flags log_io|force_umount file fs/xfs/xfs_fsops.c line_num 500
>           godown-213341 [008]  6398.027022: console:              [ 6397.915406] pmem1: writeback error on inode 12621949, offset 1019904, sector 12968096
>           godown-213341 [008]  6398.030551: console:              [ 6397.919546] XFS (pmem1): Log I/O Error (0x6) detected at xfs_fs_goingdown+0xa3/0xf0 (fs/
> 
> And finally the log itself is now shutdown, stopping all further
> writes to the log. But this is too late to prevent the corruption
> that moving the tail of the log forwards after we start cancelling
> writeback causes.
> 
> The fundamental problem here is that we are using the wrong shutdown
> checks for log items. We've long conflated mount shutdown with log
> shutdown state, and I started separating that recently with the
> atomic shutdown state changes in commit b36d4651e165 ("xfs: make
> forced shutdown processing atomic"). The changes in that commit
> series are directly responsible for being able to diagnose this
> issue because it clearly separated mount shutdown from log shutdown.
> 
> Essentially, once we start cancelling writeback of log items and
> removing them from the AIL because the filesystem is shut down, we
> *cannot* update the journal because we may have cancelled the items
> that pin the tail of the log. That moves the tail of the log
> forwards without having written the metadata back, hence we have
> corrupt in memory state and writing to the journal propagates that
> to the on-disk state.
> 
> What commit b36d4651e165 makes clear is that log item state needs to
> change relative to log shutdown, not mount shutdown. IOWs, anything
> that aborts metadata writeback needs to check log shutdown state
> because log items directly affect log consistency. Having them check
> mount shutdown state introduces the above race condition where we
> cancel metadata writeback before the log shuts down.
> 
> To fix this, this patch works through all log items and converts
> shutdown checks to use xlog_is_shutdown() rather than
> xfs_is_shutdown(), so that we don't start aborting metadata
> writeback before we shut off journal writes.

Once the log has shut down, is there any reason we shouldn't consider
the filesystem shut down too?

IOWs, should xfs_is_shutdown be doing something like this:

bool
xfs_is_shutdown(struct xfs_mount *mp)
{
	return test_bit(XFS_OPSTATE_SHUTDOWN, &mp->m_opstate) ||
		xlog_is_shutdown(mp->m_log);
}

I could very easily envision myself reintroducing bugs w.r.t.
{xfs,xlog}_is_shutdown because it's not immediately obvious to me
(particularly in xfs_buf.c) which one I ought to use.

Another way to put this is: what needs to succeed between the point
where we set OPSTATE_SHUTDOWN and XLOG_IO_ERROR?  Is the answer to that
"any log IO that was initiated right up until we actually set
XLOG_IO_ERROR"?  Which means random parts of the buffer cache, and the
inode/dquot flush code?

IOWs the log flush and any AIL writeback that was in progress?

> AFAICT, this race condition is a zero day IO error handling bug in
> XFS that dates back to the introduction of XLOG_IO_ERROR,
> XLOG_STATE_IOERROR and XFS_FORCED_SHUTDOWN back in January 1997.
> 
> Signed-off-by: Dave Chinner <dchinner@redhat.com>
> ---
>  fs/xfs/xfs_buf.c        | 30 +++++++++++++++++++++++-------
>  fs/xfs/xfs_icache.c     |  3 ++-
>  fs/xfs/xfs_inode.c      | 15 +++++++++++++--
>  fs/xfs/xfs_inode_item.c | 12 ++++++++++++
>  fs/xfs/xfs_qm.c         |  8 ++++----
>  5 files changed, 54 insertions(+), 14 deletions(-)
> 
> diff --git a/fs/xfs/xfs_buf.c b/fs/xfs/xfs_buf.c
> index 8867f143598e..b6073b5a990a 100644
> --- a/fs/xfs/xfs_buf.c
> +++ b/fs/xfs/xfs_buf.c
> @@ -14,6 +14,7 @@
>  #include "xfs_trace.h"
>  #include "xfs_log.h"
>  #include "xfs_log_recover.h"
> +#include "xfs_log_priv.h"
>  #include "xfs_trans.h"
>  #include "xfs_buf_item.h"
>  #include "xfs_errortag.h"
> @@ -813,7 +814,7 @@ xfs_buf_read_map(
>  	 * buffer.
>  	 */
>  	if (error) {
> -		if (!xfs_is_shutdown(target->bt_mount))
> +		if (!xlog_is_shutdown(target->bt_mount->m_log))
>  			xfs_buf_ioerror_alert(bp, fa);
>  
>  		bp->b_flags &= ~XBF_DONE;
> @@ -1177,10 +1178,10 @@ xfs_buf_ioend_handle_error(
>  	struct xfs_error_cfg	*cfg;
>  
>  	/*
> -	 * If we've already decided to shutdown the filesystem because of I/O
> +	 * If we've already decided to shutdown the journal because of I/O
>  	 * errors, there's no point in giving this a retry.
>  	 */
> -	if (xfs_is_shutdown(mp))
> +	if (xlog_is_shutdown(mp->m_log))
>  		goto out_stale;
>  
>  	xfs_buf_ioerror_alert_ratelimited(bp);
> @@ -1593,8 +1594,23 @@ __xfs_buf_submit(
>  
>  	ASSERT(!(bp->b_flags & _XBF_DELWRI_Q));
>  
> -	/* on shutdown we stale and complete the buffer immediately */
> -	if (xfs_is_shutdown(bp->b_mount)) {
> +	/*
> +	 * On log shutdown we stale and complete the buffer immediately. We can
> +	 * be called to read the superblock before the log has been set up, so
> +	 * be careful checking the log state.
> +	 *
> +	 * Checking the mount shutdown state here can result in the log tail
> +	 * moving inappropriately on disk as the log may not yet be shut down.
> +	 * Hence failing this buffer on mount shutdown can remove it from the
> +	 * AIL and move the tail of the log forwards without having written
> +	 * this buffer to disk. This corrupts the log tail state in memory, and
> +	 * because the log isn't yet shut down, it can then be propagated to
> +	 * disk before the log is shutdown. Hence we check log shutdown state
> +	 * here rather than mount state to avoid corrupting the log tail on
> +	 * shutdown.
> +	 */
> +	if (bp->b_mount->m_log &&
> +	    xlog_is_shutdown(bp->b_mount->m_log)) {
>  		xfs_buf_ioend_fail(bp);
>  		return -EIO;
>  	}
> @@ -1808,10 +1824,10 @@ xfs_buftarg_drain(
>  	 * If one or more failed buffers were freed, that means dirty metadata
>  	 * was thrown away. This should only ever happen after I/O completion
>  	 * handling has elevated I/O error(s) to permanent failures and shuts
> -	 * down the fs.
> +	 * down the journal.
>  	 */
>  	if (write_fail) {
> -		ASSERT(xfs_is_shutdown(btp->bt_mount));
> +		ASSERT(xlog_is_shutdown(btp->bt_mount->m_log));
>  		xfs_alert(btp->bt_mount,
>  	      "Please run xfs_repair to determine the extent of the problem.");
>  	}
> diff --git a/fs/xfs/xfs_icache.c b/fs/xfs/xfs_icache.c
> index 9644f938990c..57ebf6ceef30 100644
> --- a/fs/xfs/xfs_icache.c
> +++ b/fs/xfs/xfs_icache.c
> @@ -23,6 +23,7 @@
>  #include "xfs_reflink.h"
>  #include "xfs_ialloc.h"
>  #include "xfs_ag.h"
> +#include "xfs_log_priv.h"
>  
>  #include <linux/iversion.h>
>  
> @@ -873,7 +874,7 @@ xfs_reclaim_inode(
>  	if (xfs_iflags_test_and_set(ip, XFS_IFLUSHING))
>  		goto out_iunlock;
>  
> -	if (xfs_is_shutdown(ip->i_mount)) {
> +	if (xlog_is_shutdown(ip->i_mount->m_log)) {
>  		xfs_iunpin_wait(ip);
>  		xfs_iflush_abort(ip);
>  		goto reclaim;
> diff --git a/fs/xfs/xfs_inode.c b/fs/xfs/xfs_inode.c
> index 04bf467b1090..aab55a06ece7 100644
> --- a/fs/xfs/xfs_inode.c
> +++ b/fs/xfs/xfs_inode.c
> @@ -35,6 +35,7 @@
>  #include "xfs_bmap_btree.h"
>  #include "xfs_reflink.h"
>  #include "xfs_ag.h"
> +#include "xfs_log_priv.h"
>  
>  struct kmem_cache *xfs_inode_cache;
>  
> @@ -3659,7 +3660,7 @@ xfs_iflush_cluster(
>  		 * AIL, leaving a dirty/unpinned inode attached to the buffer
>  		 * that otherwise looks like it should be flushed.
>  		 */
> -		if (xfs_is_shutdown(mp)) {
> +		if (xlog_is_shutdown(mp->m_log)) {
>  			xfs_iunpin_wait(ip);
>  			xfs_iflush_abort(ip);
>  			xfs_iunlock(ip, XFS_ILOCK_SHARED);
> @@ -3685,9 +3686,19 @@ xfs_iflush_cluster(
>  	}
>  
>  	if (error) {
> +		/*
> +		 * Shutdown first so we kill the log before we release this
> +		 * buffer. If it is an INODE_ALLOC buffer and pins the tail

Does inode flush failure leading to immediate shutdown need to happen
with the dquot code too?  I /think/ we don't?  Because all we do is
remove the dirty flag on the dquot and kill the log?

Ok, lunch time, I'm going to push send so you can see the progress I've
made so far.

--D

> +		 * of the log, failing it before the _log_ is shut down can
> +		 * result in the log tail being moved forward in the journal
> +		 * on disk because log writes can still be taking place. Hence
> +		 * unpinning the tail will allow the ICREATE intent to be
> +		 * removed from the log an recovery will fail with uninitialised
> +		 * inode cluster buffers.
> +		 */
> +		xfs_force_shutdown(mp, SHUTDOWN_CORRUPT_INCORE);
>  		bp->b_flags |= XBF_ASYNC;
>  		xfs_buf_ioend_fail(bp);
> -		xfs_force_shutdown(mp, SHUTDOWN_CORRUPT_INCORE);
>  		return error;
>  	}
>  
> diff --git a/fs/xfs/xfs_inode_item.c b/fs/xfs/xfs_inode_item.c
> index 19dc3e37bb4d..308b30f35a71 100644
> --- a/fs/xfs/xfs_inode_item.c
> +++ b/fs/xfs/xfs_inode_item.c
> @@ -17,6 +17,7 @@
>  #include "xfs_trans_priv.h"
>  #include "xfs_buf_item.h"
>  #include "xfs_log.h"
> +#include "xfs_log_priv.h"
>  #include "xfs_error.h"
>  
>  #include <linux/iversion.h>
> @@ -709,6 +710,17 @@ xfs_iflush_ail_updates(
>  		if (INODE_ITEM(lip)->ili_flush_lsn != lip->li_lsn)
>  			continue;
>  
> +		/*
> +		 * dgc: Not sure how this happens, but it happens very
> +		 * occassionaly via generic/388.  xfs_iflush_abort() also
> +		 * silently handles this same "under writeback but not in AIL at
> +		 * shutdown" condition via xfs_trans_ail_delete().
> +		 */
> +		if (!test_bit(XFS_LI_IN_AIL, &lip->li_flags)) {
> +			ASSERT(xlog_is_shutdown(lip->li_log));
> +			continue;
> +		}
> +
>  		lsn = xfs_ail_delete_one(ailp, lip);
>  		if (!tail_lsn && lsn)
>  			tail_lsn = lsn;
> diff --git a/fs/xfs/xfs_qm.c b/fs/xfs/xfs_qm.c
> index 32ac8d9c8940..f165d1a3de1d 100644
> --- a/fs/xfs/xfs_qm.c
> +++ b/fs/xfs/xfs_qm.c
> @@ -25,6 +25,7 @@
>  #include "xfs_error.h"
>  #include "xfs_ag.h"
>  #include "xfs_ialloc.h"
> +#include "xfs_log_priv.h"
>  
>  /*
>   * The global quota manager. There is only one of these for the entire
> @@ -121,8 +122,7 @@ xfs_qm_dqpurge(
>  	struct xfs_dquot	*dqp,
>  	void			*data)
>  {
> -	struct xfs_mount	*mp = dqp->q_mount;
> -	struct xfs_quotainfo	*qi = mp->m_quotainfo;
> +	struct xfs_quotainfo	*qi = dqp->q_mount->m_quotainfo;
>  	int			error = -EAGAIN;
>  
>  	xfs_dqlock(dqp);
> @@ -157,7 +157,7 @@ xfs_qm_dqpurge(
>  	}
>  
>  	ASSERT(atomic_read(&dqp->q_pincount) == 0);
> -	ASSERT(xfs_is_shutdown(mp) ||
> +	ASSERT(xlog_is_shutdown(dqp->q_logitem.qli_item.li_log) ||
>  		!test_bit(XFS_LI_IN_AIL, &dqp->q_logitem.qli_item.li_flags));
>  
>  	xfs_dqfunlock(dqp);
> @@ -172,7 +172,7 @@ xfs_qm_dqpurge(
>  	 */
>  	ASSERT(!list_empty(&dqp->q_lru));
>  	list_lru_del(&qi->qi_lru, &dqp->q_lru);
> -	XFS_STATS_DEC(mp, xs_qm_dquot_unused);
> +	XFS_STATS_DEC(dqp->q_mount, xs_qm_dquot_unused);
>  
>  	xfs_qm_dqdestroy(dqp);
>  	return 0;
> -- 
> 2.35.1
> 

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

* Re: [PATCH 2/7] xfs: check buffer pin state after locking in delwri_submit
  2022-03-15 19:13   ` Darrick J. Wong
@ 2022-03-15 21:11     ` Dave Chinner
  2022-03-15 22:42       ` Darrick J. Wong
  0 siblings, 1 reply; 29+ messages in thread
From: Dave Chinner @ 2022-03-15 21:11 UTC (permalink / raw)
  To: Darrick J. Wong; +Cc: linux-xfs

On Tue, Mar 15, 2022 at 12:13:20PM -0700, Darrick J. Wong wrote:
> On Tue, Mar 15, 2022 at 05:42:36PM +1100, Dave Chinner wrote:
> > From: Dave Chinner <dchinner@redhat.com>
> > 
> > AIL flushing can get stuck here:
> > 
> > [316649.005769] INFO: task xfsaild/pmem1:324525 blocked for more than 123 seconds.
> > [316649.007807]       Not tainted 5.17.0-rc6-dgc+ #975
> > [316649.009186] "echo 0 > /proc/sys/kernel/hung_task_timeout_secs" disables this message.
> > [316649.011720] task:xfsaild/pmem1   state:D stack:14544 pid:324525 ppid:     2 flags:0x00004000
> > [316649.014112] Call Trace:
> > [316649.014841]  <TASK>
> > [316649.015492]  __schedule+0x30d/0x9e0
> > [316649.017745]  schedule+0x55/0xd0
> > [316649.018681]  io_schedule+0x4b/0x80
> > [316649.019683]  xfs_buf_wait_unpin+0x9e/0xf0
> > [316649.021850]  __xfs_buf_submit+0x14a/0x230
> > [316649.023033]  xfs_buf_delwri_submit_buffers+0x107/0x280
> > [316649.024511]  xfs_buf_delwri_submit_nowait+0x10/0x20
> > [316649.025931]  xfsaild+0x27e/0x9d0
> > [316649.028283]  kthread+0xf6/0x120
> > [316649.030602]  ret_from_fork+0x1f/0x30
> > 
> > in the situation where flushing gets preempted between the unpin
> > check and the buffer trylock under nowait conditions:
> > 
> > 	blk_start_plug(&plug);
> > 	list_for_each_entry_safe(bp, n, buffer_list, b_list) {
> > 		if (!wait_list) {
> > 			if (xfs_buf_ispinned(bp)) {
> > 				pinned++;
> > 				continue;
> > 			}
> > Here >>>>>>
> > 			if (!xfs_buf_trylock(bp))
> > 				continue;
> > 
> > This means submission is stuck until something else triggers a log
> > force to unpin the buffer.
> > 
> > To get onto the delwri list to begin with, the buffer pin state has
> > already been checked, and hence it's relatively rare we get a race
> > between flushing and encountering a pinned buffer in delwri
> > submission to begin with. Further, to increase the pin count the
> > buffer has to be locked, so the only way we can hit this race
> > without failing the trylock is to be preempted between the pincount
> > check seeing zero and the trylock being run.
> > 
> > Hence to avoid this problem, just invert the order of trylock vs
> > pin check. We shouldn't hit that many pinned buffers here, so
> > optimising away the trylock for pinned buffers should not matter for
> > performance at all.
> > 
> > Signed-off-by: Dave Chinner <dchinner@redhat.com>
> > ---
> >  fs/xfs/xfs_buf.c | 5 +++--
> >  1 file changed, 3 insertions(+), 2 deletions(-)
> > 
> > diff --git a/fs/xfs/xfs_buf.c b/fs/xfs/xfs_buf.c
> > index b45e0d50a405..8867f143598e 100644
> > --- a/fs/xfs/xfs_buf.c
> > +++ b/fs/xfs/xfs_buf.c
> > @@ -2094,12 +2094,13 @@ xfs_buf_delwri_submit_buffers(
> >  	blk_start_plug(&plug);
> >  	list_for_each_entry_safe(bp, n, buffer_list, b_list) {
> >  		if (!wait_list) {
> > +			if (!xfs_buf_trylock(bp))
>  +				continue;
> >  			if (xfs_buf_ispinned(bp)) {
> > +				xfs_buf_unlock(bp);
> >  				pinned++;
> >  				continue;
> 
> Hmm.  So I think this means that this function willl skip buffers that
> are locked or pinned.  The only way that the AIL would encounter this
> situation is when a buffer on its list is now locked by a reader thread
> or is participating in a transaction.  In the reader case this is (one
> hopes) ok because the reader won't block on the AIL.
> 
> The tx case is trickier -- transaction allocation can result in an AIL
> push if the head is too close to the tail, right?  Ordinarily, the AIL
> won't find itself unable to write a buffer that's pinning the log
> because a transaction holds that buffer -- eventually that tx should
> commit, which will unlock the buffer and allow the AIL to make some
> progress.
> 
> But -- what if the frontend is running a chained transaction, and it
> bjoin'd the buffer to the transaction, tried to roll the transaction,
> and the chain runs out of permanent log reservation (because we've
> rolled more than logcount times) and we have to wait for more log grant
> space?  The regrant for the successor tx happens before the commit of
> the old tx, so can we livelock the log in this way?
> 
> And doesn't this potential exist regardless of this patch?
> 
> I suspect the answers are 'yes' and 'yes',

The answer is yes and yes.

The transaction case you talk about is the same as an inode we are
running a long tx chain on. Say extent removal on an inode with a
few million extents - thinking about this case is somewhat easier to
reason about(*) - the inode stays locked across tx commit and is
re-joined to the next transaction so the extent removal is atomic
from the perspective of the user (i.e.  ftruncate() completes before
any concurrent IO can make progress).

This works from a tx and log perspective because the inode is logged
in *every* transaction of the chain, which has the effect of
continually moving it forward in the log and AIL as the CIL commits in the
background and updates the LSN of the latest modification of the
item in the AIL. Hence the item never needs writeback to unpin the
tail of the log - the act of committing the latest transaction in
the chain will always move it to the head of the log.

IOWs, relogging items that remain locked across transaction commits
is a requirement of permanent transactions to prevent the deadlock
you mention. It's also one of the reasons why we must be able to fit
at least two whole checkpoints in the log - so that items that have
been relogged can unpin the tail of the log when the second
checkpoint completes without requiring writeback of the metadata.
There's some more detail in "Introduction to Re-logging in XFS" in
Documentation/filesystems/xfs-delayed-logging-design.rst", but the
gist of it is above...

(*) Buffers have a couple of extra cases where we do have to be
*really* careful about rolling transactions. The primary one is
INODE_ALLOC buffers, which have to remain pinned in the AIL to their
original LSN even when they are relogged (e.g. for unlinked list
updates) because we cannot move the tail of the log past the LSN
where the inode chunk is initialised on disk without actually
initialising the inode chunk on disk. Hence INODE_ALLOC buffers
cannot be used as the basis of long running atomic TX chains because
they require writeback instead of relogging to unpin the tail of the
log.

Cheers,

Dave.
-- 
Dave Chinner
david@fromorbit.com

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

* Re: [PATCH 3/7] xfs: xfs_ail_push_all_sync() stalls when racing with updates
  2022-03-15 19:17   ` Darrick J. Wong
@ 2022-03-15 21:29     ` Dave Chinner
  0 siblings, 0 replies; 29+ messages in thread
From: Dave Chinner @ 2022-03-15 21:29 UTC (permalink / raw)
  To: Darrick J. Wong; +Cc: linux-xfs

On Tue, Mar 15, 2022 at 12:17:35PM -0700, Darrick J. Wong wrote:
> On Tue, Mar 15, 2022 at 05:42:37PM +1100, Dave Chinner wrote:
> > From: Dave Chinner <dchinner@redhat.com>
> > 
> > xfs_ail_push_all_sync() has a loop like this:
> > 
> > while max_ail_lsn {
> > 	prepare_to_wait(ail_empty)
> > 	target = max_ail_lsn
> > 	wake_up(ail_task);
> > 	schedule()
> > }
> > 
> > Which is designed to sleep until the AIL is emptied. When
> > xfs_ail_finish_update() moves the tail of the log, it does:
> > 
> > 	if (list_empty(&ailp->ail_head))
> > 		wake_up_all(&ailp->ail_empty);
> > 
> > So it will only wake up the sync push waiter when the AIL goes
> > empty. If, by the time the push waiter has woken, the AIL has more
> > in it, it will reset the target, wake the push task and go back to
> > sleep.
> > 
> > The problem here is that if the AIL is having items added to it
> > when xfs_ail_push_all_sync() is called, then they may get inserted
> > into the AIL at a LSN higher than the target LSN. At this point,
> > xfsaild_push() will see that the target is X, the item LSNs are
> > (X+N) and skip over them, hence never pushing the out.
> > 
> > The result of this the AIL will not get emptied by the AIL push
> > thread, hence xfs_ail_finish_update() will never see the AIL being
> > empty even if it moves the tail. Hence xfs_ail_push_all_sync() never
> > gets woken and hence cannot update the push target to capture the
> > items beyond the current target on the LSN.
> > 
> > This is a TOCTOU type of issue so the way to avoid it is to not
> > use the push target at all for sync pushes. We know that a sync push
> > is being requested by the fact the ail_empty wait queue is active,
> > hence the xfsaild can just set the target to max_ail_lsn on every
> > push that we see the wait queue active. Hence we no longer will
> > leave items on the AIL that are beyond the LSN sampled at the start
> > of a sync push.
> > 
> > Signed-off-by: Dave Chinner <dchinner@redhat.com>
> > ---
> >  fs/xfs/xfs_trans_ail.c | 21 ++++++++++++++++-----
> >  1 file changed, 16 insertions(+), 5 deletions(-)
> > 
> > diff --git a/fs/xfs/xfs_trans_ail.c b/fs/xfs/xfs_trans_ail.c
> > index 2a8c8dc54c95..1b52952097c1 100644
> > --- a/fs/xfs/xfs_trans_ail.c
> > +++ b/fs/xfs/xfs_trans_ail.c
> > @@ -448,10 +448,22 @@ xfsaild_push(
> >  
> >  	spin_lock(&ailp->ail_lock);
> >  
> > -	/* barrier matches the ail_target update in xfs_ail_push() */
> > -	smp_rmb();
> > -	target = ailp->ail_target;
> > -	ailp->ail_target_prev = target;
> > +	/*
> > +	 * If we have a sync push waiter, we always have to push till the AIL is
> > +	 * empty. Update the target to point to the end of the AIL so that
> > +	 * capture updates that occur after the sync push waiter has gone to
> > +	 * sleep.
> > +	 */
> > +	if (waitqueue_active(&ailp->ail_empty)) {
> > +		lip = xfs_ail_max(ailp);
> > +		if (lip)
> > +			target = lip->li_lsn;
> > +	} else {
> > +		/* barrier matches the ail_target update in xfs_ail_push() */
> > +		smp_rmb();
> 
> Doesn't the spin_lock provide the required rmb?  I think it's
> unnecessary given that, but I also don't think it hurts anything, so:

No. xfs_ail_push() does not take the ail_lock to update
ail->ail_target on 64 bit systems(*). Spin locks only provide memory
barriers between critical sections within the lock/unlock calls, and
even then the barrier is in the unlock -> lock direction only.  i.e.
what is written before the unlock in one critical section is
guaranteed to be read after the lock that starts the next critical
section.

Instead, xfs_ail_push() has smp_wmb() calls around setting the
target to ensure that all ail state updates done -before the wmb- are
seen by reads done -after the rmb- above. These memory barriers
could probably be replaced with a smp_store_release() and
smp_load_acquire() pair, because that is effectively what they are
implementing but the implementation predates those primitives.

OTOH, we don't need a rmb before the new waitqueue_active check
because all the waitqueue manipulations are done under the ail_lock.
Hence the ail_lock provides the memory barriers for that branch.

IOWs, the smp_rmb() is still necessary for the lockless
xfs_ail_push() update path, just like it was before this patch.

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

Thanks!

Cheers,

Dave.
-- 
Dave Chinner
david@fromorbit.com

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

* Re: [PATCH 4/7] xfs: async CIL flushes need pending pushes to be made stable
  2022-03-15 19:36   ` Darrick J. Wong
@ 2022-03-15 21:47     ` Dave Chinner
  2022-03-16  2:00       ` Darrick J. Wong
  0 siblings, 1 reply; 29+ messages in thread
From: Dave Chinner @ 2022-03-15 21:47 UTC (permalink / raw)
  To: Darrick J. Wong; +Cc: linux-xfs

On Tue, Mar 15, 2022 at 12:36:24PM -0700, Darrick J. Wong wrote:
> On Tue, Mar 15, 2022 at 05:42:38PM +1100, Dave Chinner wrote:
> > From: Dave Chinner <dchinner@redhat.com>
> > 
> > When the AIL tries to flush the CIL, it relies on the CIL push
> > ending up on stable storage without having to wait for and
> > manipulate iclog state directly. However, if there is already a
> > pending CIL push when the AIL tries to flush the CIL, it won't set
> > the cil->xc_push_commit_stable flag and so the CIL push will not
> > actively flush the commit record iclog.
> > 
> > generic/530 when run on a single CPU test VM can trigger this fairly
> > reliably. This test exercises unlinked inode recovery, and can
> > result in inodes being pinned in memory by ongoing modifications to
> > the inode cluster buffer to record unlinked list modifications. As a
> > result, the first inode unlinked in a buffer can pin the tail of the
> > log whilst the inode cluster buffer is pinned by the current
> > checkpoint that has been pushed but isn't on stable storage because
> > because the cil->xc_push_commit_stable was not set. This results in
> > the log/AIL effectively deadlocking until something triggers the
> > commit record iclog to be pushed to stable storage (i.e. the
> > periodic log worker calling xfs_log_force()).
> > 
> > The fix is two-fold - first we should always set the
> > cil->xc_push_commit_stable when xlog_cil_flush() is called,
> > regardless of whether there is already a pending push or not.
> > 
> > Second, if the CIL is empty, we should trigger an iclog flush to
> > ensure that the iclogs of the last checkpoint have actually been
> > submitted to disk as that checkpoint may not have been run under
> > stable completion constraints.
> 
> Can it ever be the case that the CIL is not empty but the last
> checkpoint wasn't committed to disk?

Yes. But xlog_cil_push_now() will capture that, queue it and mark
it as xc_push_commit_stable. 

Remember that the push_now() code updates the push seq/stable
state under down_read(ctx lock) + spin_lock(push lock) context. The
push seq/stable state is cleared by the push worker under
down_write(ctx lock) + spin_lock(push lock) conditions when it
atomically swaps in the new empty CIL context. Hence the push worker
will always see the stable flag if it has been set for that push
sequence.

> Let's say someone else
> commits a transaction after the worker samples xc_push_commit_stable?

If we race with commits between the xlog_cil_push_now(log, seq,
true) and the CIL list_empty check in xlog_cil_flush(), there are
two posibilities:

1. the CIL push worker hasn't run and atomically switched in a new
CIL context.
2. the CIL push worker has run and switched contexts

In the first case, the commit will end up in the same context that
xlog_cil_flush() pushed, and it will be stable. That will result in
an empty CIL after the CIL push worker runs, but the racing commit
will be stable as per the xc_push_commit_stable flag. This can also
lead to the CIL being empty by the time the list_empty check is done
(because pre-empt), in which case the log force will be a no-op
because none of the iclogs need flushing.

> IOWs, why does a not-empty CIL mean that the last checkpoint is on disk?

In the second case, the CIL push triggered by xlog_cil_push_now()
will be stable because xc_push_commit_stable says it must be, and
the racing commit will end up in the new CIL context and the CIL
won't be empty. We don't need a log force in this case because the
previous sequence that was flushed with stable semantics as required.

In the case of AIL pushing, we don't actually care about racing CIL
commits because we are trying to get pinned AIL items unpinned so we
can move the tail of the log forwards. If those pinned items are
relogged by racing transactions, then the next call to
xlog_cil_flush() from the AIL will get them unpinned and that will
move them forward in the log, anyway.

Cheers,

Dave.

-- 
Dave Chinner
david@fromorbit.com

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

* Re: [PATCH 7/7] xfs: xfs_is_shutdown vs xlog_is_shutdown cage fight
  2022-03-15 20:03   ` Darrick J. Wong
@ 2022-03-15 22:20     ` Dave Chinner
  2022-03-16  1:22       ` Darrick J. Wong
  0 siblings, 1 reply; 29+ messages in thread
From: Dave Chinner @ 2022-03-15 22:20 UTC (permalink / raw)
  To: Darrick J. Wong; +Cc: linux-xfs

On Tue, Mar 15, 2022 at 01:03:21PM -0700, Darrick J. Wong wrote:
> On Tue, Mar 15, 2022 at 05:42:41PM +1100, Dave Chinner wrote:
> > From: Dave Chinner <dchinner@redhat.com>
> > 
> > I've been chasing a recent resurgence in generic/388 recovery
> > failure and/or corruption events. The events have largely been
> 
> recoveryloop, the gift that keeps on giving...

*nod*

> > The fundamental problem here is that we are using the wrong shutdown
> > checks for log items. We've long conflated mount shutdown with log
> > shutdown state, and I started separating that recently with the
> > atomic shutdown state changes in commit b36d4651e165 ("xfs: make
> > forced shutdown processing atomic"). The changes in that commit
> > series are directly responsible for being able to diagnose this
> > issue because it clearly separated mount shutdown from log shutdown.
> > 
> > Essentially, once we start cancelling writeback of log items and
> > removing them from the AIL because the filesystem is shut down, we
> > *cannot* update the journal because we may have cancelled the items
> > that pin the tail of the log. That moves the tail of the log
> > forwards without having written the metadata back, hence we have
> > corrupt in memory state and writing to the journal propagates that
> > to the on-disk state.
> > 
> > What commit b36d4651e165 makes clear is that log item state needs to
> > change relative to log shutdown, not mount shutdown. IOWs, anything
> > that aborts metadata writeback needs to check log shutdown state
> > because log items directly affect log consistency. Having them check
> > mount shutdown state introduces the above race condition where we
> > cancel metadata writeback before the log shuts down.
> > 
> > To fix this, this patch works through all log items and converts
> > shutdown checks to use xlog_is_shutdown() rather than
> > xfs_is_shutdown(), so that we don't start aborting metadata
> > writeback before we shut off journal writes.
> 
> Once the log has shut down, is there any reason we shouldn't consider
> the filesystem shut down too?
> 
> IOWs, should xfs_is_shutdown be doing something like this:
> 
> bool
> xfs_is_shutdown(struct xfs_mount *mp)
> {
> 	return test_bit(XFS_OPSTATE_SHUTDOWN, &mp->m_opstate) ||
> 		xlog_is_shutdown(mp->m_log);
> }

Not necessary - the way the shutdown code runs now we guarantee
that XFS_OPSTATE_SHUTDOWN is set *before* we set XLOG_IO_ERROR.
Hence we'll never see XLOG_IO_ERROR without XFS_OPSTATE_SHUTDOWN.

> I could very easily envision myself reintroducing bugs w.r.t.
> {xfs,xlog}_is_shutdown because it's not immediately obvious to me
> (particularly in xfs_buf.c) which one I ought to use.

Yeah, I can't give you a bright line answer to that right now. We've
smeared the abstraction between log and mount for a long while now,
and the result is that it's not clear what is log and what is mount
functionality.


> Another way to put this is: what needs to succeed between the point
> where we set OPSTATE_SHUTDOWN and XLOG_IO_ERROR?  Is the answer to that
> "any log IO that was initiated right up until we actually set
> XLOG_IO_ERROR"?

That's one half - the other half is....

> Which means random parts of the buffer cache, and the
> inode/dquot flush code?
> 
> IOWs the log flush and any AIL writeback that was in progress?

... yeah, this.

Like the CIL, the AIL belongs to the log, not to the mount.
Similarly, log items belong to the log, not the transaction
subsystem. The transaction subsystem is the interface layer between
the mount and the log - code from above that interacts with
transaction knows only about mounts and so they all use
xfs_is_shutdown().

The transactions interface with the log via log tickets and log
items, which are provided by the log, not the transaction subsystem.
Anything that operates on or manages the log, log tickets or log
items should typically use xlog_is_shutdown().

This means subsystems that are used both from the mount and log
log levels (e.g. xfs_buf.c) has a difficult line to straddle.
However, it's worth noting that high level transaction buffer read side 
does mount shutdown checks (e.g. in xfs_trans_read_buf_map()) and
so that largely allows the low level buffer code to only have to
care about log level shutdowns. Hence the check in
__xfs_buf_submit() is converted to a log level check so that it
doesn't abort buffer log item writeback before the log is shut down.

Hence I think working out what the right thing to do is short term
pain while we work through re-establishing the log vs mount
abstractions correctly.

I've got various patchsets I've been working on over the past year
that clean a fair bit of this this up. However, they are kindai
intertwined through the patchsets that provide CIL scalability,
intent whiteouts, in-memory iunlink intents, log ticket cleanups,
log ticket/grant head scalability (byte tracking, not LSNs), moving
AIL push targeting into the AIL instead of setting targets from
transaction reservation, moving iclogs behind the CIL and removing
log force shenanigans, etc. because I've done cleanups as I've
touched various bits of the code...

> > @@ -3659,7 +3660,7 @@ xfs_iflush_cluster(
> >  		 * AIL, leaving a dirty/unpinned inode attached to the buffer
> >  		 * that otherwise looks like it should be flushed.
> >  		 */
> > -		if (xfs_is_shutdown(mp)) {
> > +		if (xlog_is_shutdown(mp->m_log)) {
> >  			xfs_iunpin_wait(ip);
> >  			xfs_iflush_abort(ip);
> >  			xfs_iunlock(ip, XFS_ILOCK_SHARED);
> > @@ -3685,9 +3686,19 @@ xfs_iflush_cluster(
> >  	}
> >  
> >  	if (error) {
> > +		/*
> > +		 * Shutdown first so we kill the log before we release this
> > +		 * buffer. If it is an INODE_ALLOC buffer and pins the tail
> 
> Does inode flush failure leading to immediate shutdown need to happen
> with the dquot code too?  I /think/ we don't?  Because all we do is
> remove the dirty flag on the dquot and kill the log?

The dquot flush code already does an immediate shutdown on flush
failure, too. see xfs_qm_dqflush():

out_abort:
	dqp->q_flags &= ~XFS_DQFLAG_DIRTY;
	xfs_trans_ail_delete(lip, 0);
	xfs_force_shutdown(mp, SHUTDOWN_CORRUPT_INCORE);

Cheers,

Dave.

-- 
Dave Chinner
david@fromorbit.com

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

* Re: [PATCH 2/7] xfs: check buffer pin state after locking in delwri_submit
  2022-03-15 21:11     ` Dave Chinner
@ 2022-03-15 22:42       ` Darrick J. Wong
  0 siblings, 0 replies; 29+ messages in thread
From: Darrick J. Wong @ 2022-03-15 22:42 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs

On Wed, Mar 16, 2022 at 08:11:07AM +1100, Dave Chinner wrote:
> On Tue, Mar 15, 2022 at 12:13:20PM -0700, Darrick J. Wong wrote:
> > On Tue, Mar 15, 2022 at 05:42:36PM +1100, Dave Chinner wrote:
> > > From: Dave Chinner <dchinner@redhat.com>
> > > 
> > > AIL flushing can get stuck here:
> > > 
> > > [316649.005769] INFO: task xfsaild/pmem1:324525 blocked for more than 123 seconds.
> > > [316649.007807]       Not tainted 5.17.0-rc6-dgc+ #975
> > > [316649.009186] "echo 0 > /proc/sys/kernel/hung_task_timeout_secs" disables this message.
> > > [316649.011720] task:xfsaild/pmem1   state:D stack:14544 pid:324525 ppid:     2 flags:0x00004000
> > > [316649.014112] Call Trace:
> > > [316649.014841]  <TASK>
> > > [316649.015492]  __schedule+0x30d/0x9e0
> > > [316649.017745]  schedule+0x55/0xd0
> > > [316649.018681]  io_schedule+0x4b/0x80
> > > [316649.019683]  xfs_buf_wait_unpin+0x9e/0xf0
> > > [316649.021850]  __xfs_buf_submit+0x14a/0x230
> > > [316649.023033]  xfs_buf_delwri_submit_buffers+0x107/0x280
> > > [316649.024511]  xfs_buf_delwri_submit_nowait+0x10/0x20
> > > [316649.025931]  xfsaild+0x27e/0x9d0
> > > [316649.028283]  kthread+0xf6/0x120
> > > [316649.030602]  ret_from_fork+0x1f/0x30
> > > 
> > > in the situation where flushing gets preempted between the unpin
> > > check and the buffer trylock under nowait conditions:
> > > 
> > > 	blk_start_plug(&plug);
> > > 	list_for_each_entry_safe(bp, n, buffer_list, b_list) {
> > > 		if (!wait_list) {
> > > 			if (xfs_buf_ispinned(bp)) {
> > > 				pinned++;
> > > 				continue;
> > > 			}
> > > Here >>>>>>
> > > 			if (!xfs_buf_trylock(bp))
> > > 				continue;
> > > 
> > > This means submission is stuck until something else triggers a log
> > > force to unpin the buffer.
> > > 
> > > To get onto the delwri list to begin with, the buffer pin state has
> > > already been checked, and hence it's relatively rare we get a race
> > > between flushing and encountering a pinned buffer in delwri
> > > submission to begin with. Further, to increase the pin count the
> > > buffer has to be locked, so the only way we can hit this race
> > > without failing the trylock is to be preempted between the pincount
> > > check seeing zero and the trylock being run.
> > > 
> > > Hence to avoid this problem, just invert the order of trylock vs
> > > pin check. We shouldn't hit that many pinned buffers here, so
> > > optimising away the trylock for pinned buffers should not matter for
> > > performance at all.
> > > 
> > > Signed-off-by: Dave Chinner <dchinner@redhat.com>
> > > ---
> > >  fs/xfs/xfs_buf.c | 5 +++--
> > >  1 file changed, 3 insertions(+), 2 deletions(-)
> > > 
> > > diff --git a/fs/xfs/xfs_buf.c b/fs/xfs/xfs_buf.c
> > > index b45e0d50a405..8867f143598e 100644
> > > --- a/fs/xfs/xfs_buf.c
> > > +++ b/fs/xfs/xfs_buf.c
> > > @@ -2094,12 +2094,13 @@ xfs_buf_delwri_submit_buffers(
> > >  	blk_start_plug(&plug);
> > >  	list_for_each_entry_safe(bp, n, buffer_list, b_list) {
> > >  		if (!wait_list) {
> > > +			if (!xfs_buf_trylock(bp))
> >  +				continue;
> > >  			if (xfs_buf_ispinned(bp)) {
> > > +				xfs_buf_unlock(bp);
> > >  				pinned++;
> > >  				continue;
> > 
> > Hmm.  So I think this means that this function willl skip buffers that
> > are locked or pinned.  The only way that the AIL would encounter this
> > situation is when a buffer on its list is now locked by a reader thread
> > or is participating in a transaction.  In the reader case this is (one
> > hopes) ok because the reader won't block on the AIL.
> > 
> > The tx case is trickier -- transaction allocation can result in an AIL
> > push if the head is too close to the tail, right?  Ordinarily, the AIL
> > won't find itself unable to write a buffer that's pinning the log
> > because a transaction holds that buffer -- eventually that tx should
> > commit, which will unlock the buffer and allow the AIL to make some
> > progress.
> > 
> > But -- what if the frontend is running a chained transaction, and it
> > bjoin'd the buffer to the transaction, tried to roll the transaction,
> > and the chain runs out of permanent log reservation (because we've
> > rolled more than logcount times) and we have to wait for more log grant
> > space?  The regrant for the successor tx happens before the commit of
> > the old tx, so can we livelock the log in this way?
> > 
> > And doesn't this potential exist regardless of this patch?
> > 
> > I suspect the answers are 'yes' and 'yes',
> 
> The answer is yes and yes.
> 
> The transaction case you talk about is the same as an inode we are
> running a long tx chain on. Say extent removal on an inode with a
> few million extents - thinking about this case is somewhat easier to
> reason about(*) - the inode stays locked across tx commit and is
> re-joined to the next transaction so the extent removal is atomic
> from the perspective of the user (i.e.  ftruncate() completes before
> any concurrent IO can make progress).

When I wrote the question, I was actually thinking about online repair,
which repeatedly relogs the AGI and AGF buffers or inodes while rolling
transactions every time we take a step towards committing a repair
action.  I haven't hit a log livelock in months now, fortunately.

> This works from a tx and log perspective because the inode is logged
> in *every* transaction of the chain, which has the effect of
> continually moving it forward in the log and AIL as the CIL commits in the
> background and updates the LSN of the latest modification of the
> item in the AIL. Hence the item never needs writeback to unpin the
> tail of the log - the act of committing the latest transaction in
> the chain will always move it to the head of the log.
> 
> IOWs, relogging items that remain locked across transaction commits
> is a requirement of permanent transactions to prevent the deadlock
> you mention. It's also one of the reasons why we must be able to fit
> at least two whole checkpoints in the log - so that items that have
> been relogged can unpin the tail of the log when the second
> checkpoint completes without requiring writeback of the metadata.
> There's some more detail in "Introduction to Re-logging in XFS" in
> Documentation/filesystems/xfs-delayed-logging-design.rst", but the
> gist of it is above...
> 
> (*) Buffers have a couple of extra cases where we do have to be
> *really* careful about rolling transactions. The primary one is
> INODE_ALLOC buffers, which have to remain pinned in the AIL to their
> original LSN even when they are relogged (e.g. for unlinked list
> updates) because we cannot move the tail of the log past the LSN
> where the inode chunk is initialised on disk without actually
> initialising the inode chunk on disk. Hence INODE_ALLOC buffers
> cannot be used as the basis of long running atomic TX chains because
> they require writeback instead of relogging to unpin the tail of the
> log.

<nod>  Inode allocation is indeed one of the murkier bits that I haven't
dealt with in any great detail with.

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

--D

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

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

* Re: [PATCH 7/7] xfs: xfs_is_shutdown vs xlog_is_shutdown cage fight
  2022-03-15 22:20     ` Dave Chinner
@ 2022-03-16  1:22       ` Darrick J. Wong
  0 siblings, 0 replies; 29+ messages in thread
From: Darrick J. Wong @ 2022-03-16  1:22 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs

On Wed, Mar 16, 2022 at 09:20:16AM +1100, Dave Chinner wrote:
> On Tue, Mar 15, 2022 at 01:03:21PM -0700, Darrick J. Wong wrote:
> > On Tue, Mar 15, 2022 at 05:42:41PM +1100, Dave Chinner wrote:
> > > From: Dave Chinner <dchinner@redhat.com>
> > > 
> > > I've been chasing a recent resurgence in generic/388 recovery
> > > failure and/or corruption events. The events have largely been
> > 
> > recoveryloop, the gift that keeps on giving...
> 
> *nod*
> 
> > > The fundamental problem here is that we are using the wrong shutdown
> > > checks for log items. We've long conflated mount shutdown with log
> > > shutdown state, and I started separating that recently with the
> > > atomic shutdown state changes in commit b36d4651e165 ("xfs: make
> > > forced shutdown processing atomic"). The changes in that commit
> > > series are directly responsible for being able to diagnose this
> > > issue because it clearly separated mount shutdown from log shutdown.
> > > 
> > > Essentially, once we start cancelling writeback of log items and
> > > removing them from the AIL because the filesystem is shut down, we
> > > *cannot* update the journal because we may have cancelled the items
> > > that pin the tail of the log. That moves the tail of the log
> > > forwards without having written the metadata back, hence we have
> > > corrupt in memory state and writing to the journal propagates that
> > > to the on-disk state.
> > > 
> > > What commit b36d4651e165 makes clear is that log item state needs to
> > > change relative to log shutdown, not mount shutdown. IOWs, anything
> > > that aborts metadata writeback needs to check log shutdown state
> > > because log items directly affect log consistency. Having them check
> > > mount shutdown state introduces the above race condition where we
> > > cancel metadata writeback before the log shuts down.
> > > 
> > > To fix this, this patch works through all log items and converts
> > > shutdown checks to use xlog_is_shutdown() rather than
> > > xfs_is_shutdown(), so that we don't start aborting metadata
> > > writeback before we shut off journal writes.
> > 
> > Once the log has shut down, is there any reason we shouldn't consider
> > the filesystem shut down too?
> > 
> > IOWs, should xfs_is_shutdown be doing something like this:
> > 
> > bool
> > xfs_is_shutdown(struct xfs_mount *mp)
> > {
> > 	return test_bit(XFS_OPSTATE_SHUTDOWN, &mp->m_opstate) ||
> > 		xlog_is_shutdown(mp->m_log);
> > }
> 
> Not necessary - the way the shutdown code runs now we guarantee
> that XFS_OPSTATE_SHUTDOWN is set *before* we set XLOG_IO_ERROR.
> Hence we'll never see XLOG_IO_ERROR without XFS_OPSTATE_SHUTDOWN.
> 
> > I could very easily envision myself reintroducing bugs w.r.t.
> > {xfs,xlog}_is_shutdown because it's not immediately obvious to me
> > (particularly in xfs_buf.c) which one I ought to use.
> 
> Yeah, I can't give you a bright line answer to that right now. We've
> smeared the abstraction between log and mount for a long while now,
> and the result is that it's not clear what is log and what is mount
> functionality.

Ugh, that's going to be messy.

The first silly idea to pop in my head was "Just pass the log into the
code paths that need the log", but that was trivially wrong because
there are things (like the buffer read path) where we actually need it
to keep working so that the AIL can push buffers out to disk after we
set OPSTATE_SHUTDOWN but before XLOG_IO_ERROR...

> 
> > Another way to put this is: what needs to succeed between the point
> > where we set OPSTATE_SHUTDOWN and XLOG_IO_ERROR?  Is the answer to that
> > "any log IO that was initiated right up until we actually set
> > XLOG_IO_ERROR"?
> 
> That's one half - the other half is....
> 
> > Which means random parts of the buffer cache, and the
> > inode/dquot flush code?
> > 
> > IOWs the log flush and any AIL writeback that was in progress?
> 
> ... yeah, this.
> 
> Like the CIL, the AIL belongs to the log, not to the mount.
> Similarly, log items belong to the log, not the transaction
> subsystem. The transaction subsystem is the interface layer between
> the mount and the log - code from above that interacts with
> transaction knows only about mounts and so they all use
> xfs_is_shutdown().
> 
> The transactions interface with the log via log tickets and log
> items, which are provided by the log, not the transaction subsystem.
> Anything that operates on or manages the log, log tickets or log
> items should typically use xlog_is_shutdown().
> 
> This means subsystems that are used both from the mount and log
> log levels (e.g. xfs_buf.c) has a difficult line to straddle.
> However, it's worth noting that high level transaction buffer read side 
> does mount shutdown checks (e.g. in xfs_trans_read_buf_map()) and
> so that largely allows the low level buffer code to only have to
> care about log level shutdowns. Hence the check in
> __xfs_buf_submit() is converted to a log level check so that it
> doesn't abort buffer log item writeback before the log is shut down.

...which I agree is why we can't make bright line statements about which
one you're supposed to use.  There's no convenient hierarchy to make it
obvious which is which, since xfs and its log are rather <cough>
codependent.

> Hence I think working out what the right thing to do is short term
> pain while we work through re-establishing the log vs mount
> abstractions correctly.

Hm.  In the /really/ short term, for each callsite you switch to
xlog_is_shutdown, can you make sure there's a comment saying why we need
to query the log and not the fs shutdown state?  I think you've done
that for /most/ of the sites where it really matters, but
xfs_buf_read_map and xfs_reclaim_inode were a bit subtle.

As I said on IRC just now, the guideline I'm thinking of is "xfs_*
functions use xfs_is_shutdown; functions called under xlog_* use
xlog_is_shutdown; and anywhere those two rules aren't apply should
probably have a comment".

My goal here is to give the rest of us enough breadcrumbs that we don't
mess up log recovery while you work on getting the rest of your stuff
merged:

> I've got various patchsets I've been working on over the past year
> that clean a fair bit of this this up. However, they are kindai
> intertwined through the patchsets that provide CIL scalability,
> intent whiteouts, in-memory iunlink intents, log ticket cleanups,
> log ticket/grant head scalability (byte tracking, not LSNs), moving
> AIL push targeting into the AIL instead of setting targets from
> transaction reservation, moving iclogs behind the CIL and removing
> log force shenanigans, etc. because I've done cleanups as I've
> touched various bits of the code...

...while not making you make so many non-documentation changes such that
rebasing your dev branch becomes a huge nightmare because you had to
extricate/rearrange a ton of cleanups.

(Heck, even a terse comment that merely affirms that we're checking for
log death and this isn't the usual "checking for fs death, urrrk" would
be enough to make me think carefully about touching such a line... :))

> 
> > > @@ -3659,7 +3660,7 @@ xfs_iflush_cluster(
> > >  		 * AIL, leaving a dirty/unpinned inode attached to the buffer
> > >  		 * that otherwise looks like it should be flushed.
> > >  		 */
> > > -		if (xfs_is_shutdown(mp)) {
> > > +		if (xlog_is_shutdown(mp->m_log)) {
> > >  			xfs_iunpin_wait(ip);
> > >  			xfs_iflush_abort(ip);
> > >  			xfs_iunlock(ip, XFS_ILOCK_SHARED);
> > > @@ -3685,9 +3686,19 @@ xfs_iflush_cluster(
> > >  	}
> > >  
> > >  	if (error) {
> > > +		/*
> > > +		 * Shutdown first so we kill the log before we release this
> > > +		 * buffer. If it is an INODE_ALLOC buffer and pins the tail
> > 
> > Does inode flush failure leading to immediate shutdown need to happen
> > with the dquot code too?  I /think/ we don't?  Because all we do is
> > remove the dirty flag on the dquot and kill the log?
> 
> The dquot flush code already does an immediate shutdown on flush
> failure, too. see xfs_qm_dqflush():
> 
> out_abort:
> 	dqp->q_flags &= ~XFS_DQFLAG_DIRTY;
> 	xfs_trans_ail_delete(lip, 0);
> 	xfs_force_shutdown(mp, SHUTDOWN_CORRUPT_INCORE);

Ah right.  So I think the logic in this patch looks ok, but a little
more commenting would help, esp. given how much I've already tripped
over this on IRC. ;)

--D

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

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

* Re: [PATCH 4/7] xfs: async CIL flushes need pending pushes to be made stable
  2022-03-15 21:47     ` Dave Chinner
@ 2022-03-16  2:00       ` Darrick J. Wong
  0 siblings, 0 replies; 29+ messages in thread
From: Darrick J. Wong @ 2022-03-16  2:00 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs

On Wed, Mar 16, 2022 at 08:47:45AM +1100, Dave Chinner wrote:
> On Tue, Mar 15, 2022 at 12:36:24PM -0700, Darrick J. Wong wrote:
> > On Tue, Mar 15, 2022 at 05:42:38PM +1100, Dave Chinner wrote:
> > > From: Dave Chinner <dchinner@redhat.com>
> > > 
> > > When the AIL tries to flush the CIL, it relies on the CIL push
> > > ending up on stable storage without having to wait for and
> > > manipulate iclog state directly. However, if there is already a
> > > pending CIL push when the AIL tries to flush the CIL, it won't set
> > > the cil->xc_push_commit_stable flag and so the CIL push will not
> > > actively flush the commit record iclog.
> > > 
> > > generic/530 when run on a single CPU test VM can trigger this fairly
> > > reliably. This test exercises unlinked inode recovery, and can
> > > result in inodes being pinned in memory by ongoing modifications to
> > > the inode cluster buffer to record unlinked list modifications. As a
> > > result, the first inode unlinked in a buffer can pin the tail of the
> > > log whilst the inode cluster buffer is pinned by the current
> > > checkpoint that has been pushed but isn't on stable storage because
> > > because the cil->xc_push_commit_stable was not set. This results in
> > > the log/AIL effectively deadlocking until something triggers the
> > > commit record iclog to be pushed to stable storage (i.e. the
> > > periodic log worker calling xfs_log_force()).
> > > 
> > > The fix is two-fold - first we should always set the
> > > cil->xc_push_commit_stable when xlog_cil_flush() is called,
> > > regardless of whether there is already a pending push or not.
> > > 
> > > Second, if the CIL is empty, we should trigger an iclog flush to
> > > ensure that the iclogs of the last checkpoint have actually been
> > > submitted to disk as that checkpoint may not have been run under
> > > stable completion constraints.
> > 
> > Can it ever be the case that the CIL is not empty but the last
> > checkpoint wasn't committed to disk?
> 
> Yes. But xlog_cil_push_now() will capture that, queue it and mark
> it as xc_push_commit_stable. 
> 
> Remember that the push_now() code updates the push seq/stable
> state under down_read(ctx lock) + spin_lock(push lock) context. The
> push seq/stable state is cleared by the push worker under
> down_write(ctx lock) + spin_lock(push lock) conditions when it
> atomically swaps in the new empty CIL context. Hence the push worker
> will always see the stable flag if it has been set for that push
> sequence.
> 
> > Let's say someone else
> > commits a transaction after the worker samples xc_push_commit_stable?
> 
> If we race with commits between the xlog_cil_push_now(log, seq,
> true) and the CIL list_empty check in xlog_cil_flush(), there are
> two posibilities:
> 
> 1. the CIL push worker hasn't run and atomically switched in a new
> CIL context.
> 2. the CIL push worker has run and switched contexts
> 
> In the first case, the commit will end up in the same context that
> xlog_cil_flush() pushed, and it will be stable. That will result in
> an empty CIL after the CIL push worker runs, but the racing commit
> will be stable as per the xc_push_commit_stable flag. This can also
> lead to the CIL being empty by the time the list_empty check is done
> (because pre-empt), in which case the log force will be a no-op
> because none of the iclogs need flushing.
> 
> > IOWs, why does a not-empty CIL mean that the last checkpoint is on disk?
> 
> In the second case, the CIL push triggered by xlog_cil_push_now()
> will be stable because xc_push_commit_stable says it must be, and
> the racing commit will end up in the new CIL context and the CIL
> won't be empty. We don't need a log force in this case because the
> previous sequence that was flushed with stable semantics as required.
> 
> In the case of AIL pushing, we don't actually care about racing CIL
> commits because we are trying to get pinned AIL items unpinned so we
> can move the tail of the log forwards. If those pinned items are
> relogged by racing transactions, then the next call to
> xlog_cil_flush() from the AIL will get them unpinned and that will
> move them forward in the log, anyway.

Ok, that's kinda what I was thinking, but wasn't sure there wasn't some
other weird subtlety that I hadn't figured out.

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

--D


--D

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

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

* Re: [PATCH 4/7] xfs: async CIL flushes need pending pushes to be made stable
  2022-03-15  6:42 ` [PATCH 4/7] xfs: async CIL flushes need pending pushes to be made stable Dave Chinner
  2022-03-15 19:36   ` Darrick J. Wong
@ 2022-03-16 10:34   ` Chandan Babu R
  2022-03-16 23:24     ` Dave Chinner
  1 sibling, 1 reply; 29+ messages in thread
From: Chandan Babu R @ 2022-03-16 10:34 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs

On 15 Mar 2022 at 12:12, Dave Chinner wrote:
> From: Dave Chinner <dchinner@redhat.com>
>
> When the AIL tries to flush the CIL, it relies on the CIL push
> ending up on stable storage without having to wait for and
> manipulate iclog state directly. However, if there is already a
> pending CIL push when the AIL tries to flush the CIL, it won't set
> the cil->xc_push_commit_stable flag and so the CIL push will not
> actively flush the commit record iclog.

I think the above sentence maps to the following snippet from
xlog_cil_push_now(),

	if (list_empty(&cil->xc_cil) || push_seq <= cil->xc_push_seq) {
		spin_unlock(&cil->xc_push_lock);
		return;
	}

i.e. if the CIL sequence that we are trying to push is already being pushed
then xlog_cil_push_now() returns without queuing work on cil->xc_push_wq.

However, the push_seq could have been previously pushed by,
1. xfsaild_push()
   In this case, cil->xc_push_commit_stable is set to true. Hence,
   xlog_cil_push_work() will definitely make sure to submit the commit record
   iclog for write I/O.
2. xfs_log_force_seq() => xlog_cil_force_seq()
   xfs_log_force_seq() invokes xlog_force_lsn() after executing
   xlog_cil_force_seq(). Here, A partially filled iclog will be in
   XLOG_STATE_ACTIVE state. This will cause xlog_force_and_check_iclog() to be
   invoked and hence the iclog is submitted for write I/O.

In both the cases listed above, iclog is guaranteed to be submitted for I/O
without any help from the log worker task.

Looks like I am missing something obvious here.

>
> generic/530 when run on a single CPU test VM can trigger this fairly
> reliably. This test exercises unlinked inode recovery, and can
> result in inodes being pinned in memory by ongoing modifications to
> the inode cluster buffer to record unlinked list modifications. As a
> result, the first inode unlinked in a buffer can pin the tail of the
> log whilst the inode cluster buffer is pinned by the current
> checkpoint that has been pushed but isn't on stable storage because
> because the cil->xc_push_commit_stable was not set. This results in
> the log/AIL effectively deadlocking until something triggers the
> commit record iclog to be pushed to stable storage (i.e. the
> periodic log worker calling xfs_log_force()).
>
> The fix is two-fold - first we should always set the
> cil->xc_push_commit_stable when xlog_cil_flush() is called,
> regardless of whether there is already a pending push or not.
>
> Second, if the CIL is empty, we should trigger an iclog flush to
> ensure that the iclogs of the last checkpoint have actually been
> submitted to disk as that checkpoint may not have been run under
> stable completion constraints.
>
> Reported-and-tested-by: Matthew Wilcox <willy@infradead.org>
> Fixes: 0020a190cf3e ("xfs: AIL needs asynchronous CIL forcing")
> Signed-off-by: Dave Chinner <dchinner@redhat.com>
> ---
>  fs/xfs/xfs_log_cil.c | 22 +++++++++++++++++++---
>  1 file changed, 19 insertions(+), 3 deletions(-)
>
> diff --git a/fs/xfs/xfs_log_cil.c b/fs/xfs/xfs_log_cil.c
> index 3d8ebf2a1e55..48b16a5feb27 100644
> --- a/fs/xfs/xfs_log_cil.c
> +++ b/fs/xfs/xfs_log_cil.c
> @@ -1369,18 +1369,27 @@ xlog_cil_push_now(
>  	if (!async)
>  		flush_workqueue(cil->xc_push_wq);
>  
> +	spin_lock(&cil->xc_push_lock);
> +
> +	/*
> +	 * If this is an async flush request, we always need to set the
> +	 * xc_push_commit_stable flag even if something else has already queued
> +	 * a push. The flush caller is asking for the CIL to be on stable
> +	 * storage when the next push completes, so regardless of who has queued
> +	 * the push, the flush requires stable semantics from it.
> +	 */
> +	cil->xc_push_commit_stable = async;
> +
>  	/*
>  	 * If the CIL is empty or we've already pushed the sequence then
> -	 * there's no work we need to do.
> +	 * there's no more work that we need to do.
>  	 */
> -	spin_lock(&cil->xc_push_lock);
>  	if (list_empty(&cil->xc_cil) || push_seq <= cil->xc_push_seq) {
>  		spin_unlock(&cil->xc_push_lock);
>  		return;
>  	}
>  
>  	cil->xc_push_seq = push_seq;
> -	cil->xc_push_commit_stable = async;
>  	queue_work(cil->xc_push_wq, &cil->xc_ctx->push_work);
>  	spin_unlock(&cil->xc_push_lock);
>  }
> @@ -1520,6 +1529,13 @@ xlog_cil_flush(
>  
>  	trace_xfs_log_force(log->l_mp, seq, _RET_IP_);
>  	xlog_cil_push_now(log, seq, true);
> +
> +	/*
> +	 * If the CIL is empty, make sure that any previous checkpoint that may
> +	 * still be in an active iclog is pushed to stable storage.
> +	 */
> +	if (list_empty(&log->l_cilp->xc_cil))
> +		xfs_log_force(log->l_mp, 0);
>  }
>  
>  /*

-- 
chandan

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

* Re: [PATCH 5/7] xfs: log items should have a xlog pointer, not a mount
  2022-03-15  6:42 ` [PATCH 5/7] xfs: log items should have a xlog pointer, not a mount Dave Chinner
  2022-03-15 19:37   ` Darrick J. Wong
@ 2022-03-16 11:06   ` Chandan Babu R
  1 sibling, 0 replies; 29+ messages in thread
From: Chandan Babu R @ 2022-03-16 11:06 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs

On 15 Mar 2022 at 12:12, Dave Chinner wrote:
> From: Dave Chinner <dchinner@redhat.com>
>
> Log items belong to the log, not the xfs_mount. Convert the mount
> pointer in the log item to a xlog pointer in preparation for
> upcoming log centric changes to the log items.
>

This patch didn't cleanly apply on top of v5.17-rc8 + Previous 4 patches. But
the changes look to be quite simple.

Reviewed-by: Chandan Babu R <chandan.babu@oracle.com>

> Signed-off-by: Dave Chinner <dchinner@redhat.com>
> ---
>  fs/xfs/xfs_bmap_item.c     | 2 +-
>  fs/xfs/xfs_buf_item.c      | 5 +++--
>  fs/xfs/xfs_extfree_item.c  | 2 +-
>  fs/xfs/xfs_log.c           | 2 +-
>  fs/xfs/xfs_log_cil.c       | 2 +-
>  fs/xfs/xfs_refcount_item.c | 2 +-
>  fs/xfs/xfs_rmap_item.c     | 2 +-
>  fs/xfs/xfs_trace.h         | 4 ++--
>  fs/xfs/xfs_trans.c         | 2 +-
>  fs/xfs/xfs_trans.h         | 3 ++-
>  10 files changed, 14 insertions(+), 12 deletions(-)
>
> diff --git a/fs/xfs/xfs_bmap_item.c b/fs/xfs/xfs_bmap_item.c
> index fa710067aac2..65ac261b3b28 100644
> --- a/fs/xfs/xfs_bmap_item.c
> +++ b/fs/xfs/xfs_bmap_item.c
> @@ -476,7 +476,7 @@ xfs_bui_item_recover(
>  	struct xfs_bui_log_item		*buip = BUI_ITEM(lip);
>  	struct xfs_trans		*tp;
>  	struct xfs_inode		*ip = NULL;
> -	struct xfs_mount		*mp = lip->li_mountp;
> +	struct xfs_mount		*mp = lip->li_log->l_mp;
>  	struct xfs_map_extent		*bmap;
>  	struct xfs_bud_log_item		*budp;
>  	xfs_filblks_t			count;
> diff --git a/fs/xfs/xfs_buf_item.c b/fs/xfs/xfs_buf_item.c
> index a7a8e4528881..522d450a94b1 100644
> --- a/fs/xfs/xfs_buf_item.c
> +++ b/fs/xfs/xfs_buf_item.c
> @@ -21,6 +21,7 @@
>  #include "xfs_dquot.h"
>  #include "xfs_trace.h"
>  #include "xfs_log.h"
> +#include "xfs_log_priv.h"
>  
>  
>  struct kmem_cache	*xfs_buf_item_cache;
> @@ -428,7 +429,7 @@ xfs_buf_item_format(
>  	 * occurs during recovery.
>  	 */
>  	if (bip->bli_flags & XFS_BLI_INODE_BUF) {
> -		if (xfs_has_v3inodes(lip->li_mountp) ||
> +		if (xfs_has_v3inodes(lip->li_log->l_mp) ||
>  		    !((bip->bli_flags & XFS_BLI_INODE_ALLOC_BUF) &&
>  		      xfs_log_item_in_current_chkpt(lip)))
>  			bip->__bli_format.blf_flags |= XFS_BLF_INODE_BUF;
> @@ -616,7 +617,7 @@ xfs_buf_item_put(
>  	 * that case, the bli is freed on buffer writeback completion.
>  	 */
>  	aborted = test_bit(XFS_LI_ABORTED, &lip->li_flags) ||
> -		  xfs_is_shutdown(lip->li_mountp);
> +			xlog_is_shutdown(lip->li_log);
>  	dirty = bip->bli_flags & XFS_BLI_DIRTY;
>  	if (dirty && !aborted)
>  		return false;
> diff --git a/fs/xfs/xfs_extfree_item.c b/fs/xfs/xfs_extfree_item.c
> index 36eeac9413f5..893a7dd15cbb 100644
> --- a/fs/xfs/xfs_extfree_item.c
> +++ b/fs/xfs/xfs_extfree_item.c
> @@ -615,7 +615,7 @@ xfs_efi_item_recover(
>  	struct list_head		*capture_list)
>  {
>  	struct xfs_efi_log_item		*efip = EFI_ITEM(lip);
> -	struct xfs_mount		*mp = lip->li_mountp;
> +	struct xfs_mount		*mp = lip->li_log->l_mp;
>  	struct xfs_efd_log_item		*efdp;
>  	struct xfs_trans		*tp;
>  	struct xfs_extent		*extp;
> diff --git a/fs/xfs/xfs_log.c b/fs/xfs/xfs_log.c
> index b0e05fa902d4..5c4ef45f42d2 100644
> --- a/fs/xfs/xfs_log.c
> +++ b/fs/xfs/xfs_log.c
> @@ -1136,7 +1136,7 @@ xfs_log_item_init(
>  	int			type,
>  	const struct xfs_item_ops *ops)
>  {
> -	item->li_mountp = mp;
> +	item->li_log = mp->m_log;
>  	item->li_ailp = mp->m_ail;
>  	item->li_type = type;
>  	item->li_ops = ops;
> diff --git a/fs/xfs/xfs_log_cil.c b/fs/xfs/xfs_log_cil.c
> index 48b16a5feb27..e9b80036268a 100644
> --- a/fs/xfs/xfs_log_cil.c
> +++ b/fs/xfs/xfs_log_cil.c
> @@ -76,7 +76,7 @@ bool
>  xfs_log_item_in_current_chkpt(
>  	struct xfs_log_item *lip)
>  {
> -	return xlog_item_in_current_chkpt(lip->li_mountp->m_log->l_cilp, lip);
> +	return xlog_item_in_current_chkpt(lip->li_log->l_cilp, lip);
>  }
>  
>  /*
> diff --git a/fs/xfs/xfs_refcount_item.c b/fs/xfs/xfs_refcount_item.c
> index d4632f2ceb89..1b82b818f515 100644
> --- a/fs/xfs/xfs_refcount_item.c
> +++ b/fs/xfs/xfs_refcount_item.c
> @@ -468,7 +468,7 @@ xfs_cui_item_recover(
>  	struct xfs_cud_log_item		*cudp;
>  	struct xfs_trans		*tp;
>  	struct xfs_btree_cur		*rcur = NULL;
> -	struct xfs_mount		*mp = lip->li_mountp;
> +	struct xfs_mount		*mp = lip->li_log->l_mp;
>  	xfs_fsblock_t			new_fsb;
>  	xfs_extlen_t			new_len;
>  	unsigned int			refc_type;
> diff --git a/fs/xfs/xfs_rmap_item.c b/fs/xfs/xfs_rmap_item.c
> index cb0490919b2c..546bd824cdf7 100644
> --- a/fs/xfs/xfs_rmap_item.c
> +++ b/fs/xfs/xfs_rmap_item.c
> @@ -523,7 +523,7 @@ xfs_rui_item_recover(
>  	struct xfs_rud_log_item		*rudp;
>  	struct xfs_trans		*tp;
>  	struct xfs_btree_cur		*rcur = NULL;
> -	struct xfs_mount		*mp = lip->li_mountp;
> +	struct xfs_mount		*mp = lip->li_log->l_mp;
>  	enum xfs_rmap_intent_type	type;
>  	xfs_exntst_t			state;
>  	int				i;
> diff --git a/fs/xfs/xfs_trace.h b/fs/xfs/xfs_trace.h
> index 585bd9853b6b..cc69b7c066e8 100644
> --- a/fs/xfs/xfs_trace.h
> +++ b/fs/xfs/xfs_trace.h
> @@ -1308,7 +1308,7 @@ DECLARE_EVENT_CLASS(xfs_log_item_class,
>  		__field(xfs_lsn_t, lsn)
>  	),
>  	TP_fast_assign(
> -		__entry->dev = lip->li_mountp->m_super->s_dev;
> +		__entry->dev = lip->li_log->l_mp->m_super->s_dev;
>  		__entry->lip = lip;
>  		__entry->type = lip->li_type;
>  		__entry->flags = lip->li_flags;
> @@ -1364,7 +1364,7 @@ DECLARE_EVENT_CLASS(xfs_ail_class,
>  		__field(xfs_lsn_t, new_lsn)
>  	),
>  	TP_fast_assign(
> -		__entry->dev = lip->li_mountp->m_super->s_dev;
> +		__entry->dev = lip->li_log->l_mp->m_super->s_dev;
>  		__entry->lip = lip;
>  		__entry->type = lip->li_type;
>  		__entry->flags = lip->li_flags;
> diff --git a/fs/xfs/xfs_trans.c b/fs/xfs/xfs_trans.c
> index 82590007e6c5..de87fb136b51 100644
> --- a/fs/xfs/xfs_trans.c
> +++ b/fs/xfs/xfs_trans.c
> @@ -646,7 +646,7 @@ xfs_trans_add_item(
>  	struct xfs_trans	*tp,
>  	struct xfs_log_item	*lip)
>  {
> -	ASSERT(lip->li_mountp == tp->t_mountp);
> +	ASSERT(lip->li_log == tp->t_mountp->m_log);
>  	ASSERT(lip->li_ailp == tp->t_mountp->m_ail);
>  	ASSERT(list_empty(&lip->li_trans));
>  	ASSERT(!test_bit(XFS_LI_DIRTY, &lip->li_flags));
> diff --git a/fs/xfs/xfs_trans.h b/fs/xfs/xfs_trans.h
> index 85dca2c9b559..1c5c5d7f522f 100644
> --- a/fs/xfs/xfs_trans.h
> +++ b/fs/xfs/xfs_trans.h
> @@ -8,6 +8,7 @@
>  
>  /* kernel only transaction subsystem defines */
>  
> +struct xlog;
>  struct xfs_buf;
>  struct xfs_buftarg;
>  struct xfs_efd_log_item;
> @@ -31,7 +32,7 @@ struct xfs_log_item {
>  	struct list_head		li_ail;		/* AIL pointers */
>  	struct list_head		li_trans;	/* transaction list */
>  	xfs_lsn_t			li_lsn;		/* last on-disk lsn */
> -	struct xfs_mount		*li_mountp;	/* ptr to fs mount */
> +	struct xlog			*li_log;
>  	struct xfs_ail			*li_ailp;	/* ptr to AIL */
>  	uint				li_type;	/* item type */
>  	unsigned long			li_flags;	/* misc flags */


-- 
chandan

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

* Re: [PATCH 6/7] xfs: AIL should be log centric
  2022-03-15  6:42 ` [PATCH 6/7] xfs: AIL should be log centric Dave Chinner
  2022-03-15 19:39   ` Darrick J. Wong
@ 2022-03-16 11:12   ` Chandan Babu R
  1 sibling, 0 replies; 29+ messages in thread
From: Chandan Babu R @ 2022-03-16 11:12 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs

On 15 Mar 2022 at 12:12, Dave Chinner wrote:
> From: Dave Chinner <dchinner@redhat.com>
>
> The AIL operates purely on log items, so it is a log centric
> subsystem. Divorce it from the xfs_mount and instead have it pass
> around xlog pointers.
>

Looks good to me.

Reviewed-by: Chandan Babu R <chandan.babu@oracle.com>

> Signed-off-by: Dave Chinner <dchinner@redhat.com>
> ---
>  fs/xfs/xfs_trans.c      |  2 +-
>  fs/xfs/xfs_trans_ail.c  | 26 +++++++++++++-------------
>  fs/xfs/xfs_trans_priv.h |  3 ++-
>  3 files changed, 16 insertions(+), 15 deletions(-)
>
> diff --git a/fs/xfs/xfs_trans.c b/fs/xfs/xfs_trans.c
> index de87fb136b51..831d355c3258 100644
> --- a/fs/xfs/xfs_trans.c
> +++ b/fs/xfs/xfs_trans.c
> @@ -773,7 +773,7 @@ xfs_trans_committed_bulk(
>  		 * object into the AIL as we are in a shutdown situation.
>  		 */
>  		if (aborted) {
> -			ASSERT(xfs_is_shutdown(ailp->ail_mount));
> +			ASSERT(xlog_is_shutdown(ailp->ail_log));
>  			if (lip->li_ops->iop_unpin)
>  				lip->li_ops->iop_unpin(lip, 1);
>  			continue;
> diff --git a/fs/xfs/xfs_trans_ail.c b/fs/xfs/xfs_trans_ail.c
> index 1b52952097c1..c2ccb98c7bcd 100644
> --- a/fs/xfs/xfs_trans_ail.c
> +++ b/fs/xfs/xfs_trans_ail.c
> @@ -398,7 +398,7 @@ xfsaild_push_item(
>  	 * If log item pinning is enabled, skip the push and track the item as
>  	 * pinned. This can help induce head-behind-tail conditions.
>  	 */
> -	if (XFS_TEST_ERROR(false, ailp->ail_mount, XFS_ERRTAG_LOG_ITEM_PIN))
> +	if (XFS_TEST_ERROR(false, ailp->ail_log->l_mp, XFS_ERRTAG_LOG_ITEM_PIN))
>  		return XFS_ITEM_PINNED;
>  
>  	/*
> @@ -418,7 +418,7 @@ static long
>  xfsaild_push(
>  	struct xfs_ail		*ailp)
>  {
> -	xfs_mount_t		*mp = ailp->ail_mount;
> +	struct xfs_mount	*mp = ailp->ail_log->l_mp;
>  	struct xfs_ail_cursor	cur;
>  	struct xfs_log_item	*lip;
>  	xfs_lsn_t		lsn;
> @@ -443,7 +443,7 @@ xfsaild_push(
>  		ailp->ail_log_flush = 0;
>  
>  		XFS_STATS_INC(mp, xs_push_ail_flush);
> -		xlog_cil_flush(mp->m_log);
> +		xlog_cil_flush(ailp->ail_log);
>  	}
>  
>  	spin_lock(&ailp->ail_lock);
> @@ -632,7 +632,7 @@ xfsaild(
>  			 * opportunity to release such buffers from the queue.
>  			 */
>  			ASSERT(list_empty(&ailp->ail_buf_list) ||
> -			       xfs_is_shutdown(ailp->ail_mount));
> +			       xlog_is_shutdown(ailp->ail_log));
>  			xfs_buf_delwri_cancel(&ailp->ail_buf_list);
>  			break;
>  		}
> @@ -695,7 +695,7 @@ xfs_ail_push(
>  	struct xfs_log_item	*lip;
>  
>  	lip = xfs_ail_min(ailp);
> -	if (!lip || xfs_is_shutdown(ailp->ail_mount) ||
> +	if (!lip || xlog_is_shutdown(ailp->ail_log) ||
>  	    XFS_LSN_CMP(threshold_lsn, ailp->ail_target) <= 0)
>  		return;
>  
> @@ -751,7 +751,7 @@ xfs_ail_update_finish(
>  	struct xfs_ail		*ailp,
>  	xfs_lsn_t		old_lsn) __releases(ailp->ail_lock)
>  {
> -	struct xfs_mount	*mp = ailp->ail_mount;
> +	struct xlog		*log = ailp->ail_log;
>  
>  	/* if the tail lsn hasn't changed, don't do updates or wakeups. */
>  	if (!old_lsn || old_lsn == __xfs_ail_min_lsn(ailp)) {
> @@ -759,13 +759,13 @@ xfs_ail_update_finish(
>  		return;
>  	}
>  
> -	if (!xfs_is_shutdown(mp))
> -		xlog_assign_tail_lsn_locked(mp);
> +	if (!xlog_is_shutdown(log))
> +		xlog_assign_tail_lsn_locked(log->l_mp);
>  
>  	if (list_empty(&ailp->ail_head))
>  		wake_up_all(&ailp->ail_empty);
>  	spin_unlock(&ailp->ail_lock);
> -	xfs_log_space_wake(mp);
> +	xfs_log_space_wake(log->l_mp);
>  }
>  
>  /*
> @@ -873,13 +873,13 @@ xfs_trans_ail_delete(
>  	int			shutdown_type)
>  {
>  	struct xfs_ail		*ailp = lip->li_ailp;
> -	struct xfs_mount	*mp = ailp->ail_mount;
> +	struct xfs_mount	*mp = ailp->ail_log->l_mp;
>  	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 && !xfs_is_shutdown(mp)) {
> +		if (shutdown_type && !xlog_is_shutdown(ailp->ail_log)) {
>  			xfs_alert_tag(mp, XFS_PTAG_AILDELETE,
>  	"%s: attempting to delete a log item that is not in the AIL",
>  					__func__);
> @@ -904,7 +904,7 @@ xfs_trans_ail_init(
>  	if (!ailp)
>  		return -ENOMEM;
>  
> -	ailp->ail_mount = mp;
> +	ailp->ail_log = mp->m_log;
>  	INIT_LIST_HEAD(&ailp->ail_head);
>  	INIT_LIST_HEAD(&ailp->ail_cursors);
>  	spin_lock_init(&ailp->ail_lock);
> @@ -912,7 +912,7 @@ xfs_trans_ail_init(
>  	init_waitqueue_head(&ailp->ail_empty);
>  
>  	ailp->ail_task = kthread_run(xfsaild, ailp, "xfsaild/%s",
> -			ailp->ail_mount->m_super->s_id);
> +				mp->m_super->s_id);
>  	if (IS_ERR(ailp->ail_task))
>  		goto out_free_ailp;
>  
> diff --git a/fs/xfs/xfs_trans_priv.h b/fs/xfs/xfs_trans_priv.h
> index 3004aeac9110..f0d79a9050ba 100644
> --- a/fs/xfs/xfs_trans_priv.h
> +++ b/fs/xfs/xfs_trans_priv.h
> @@ -6,6 +6,7 @@
>  #ifndef __XFS_TRANS_PRIV_H__
>  #define	__XFS_TRANS_PRIV_H__
>  
> +struct xlog;
>  struct xfs_log_item;
>  struct xfs_mount;
>  struct xfs_trans;
> @@ -50,7 +51,7 @@ struct xfs_ail_cursor {
>   * Eventually we need to drive the locking in here as well.
>   */
>  struct xfs_ail {
> -	struct xfs_mount	*ail_mount;
> +	struct xlog		*ail_log;
>  	struct task_struct	*ail_task;
>  	struct list_head	ail_head;
>  	xfs_lsn_t		ail_target;


-- 
chandan

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

* Re: [PATCH 4/7] xfs: async CIL flushes need pending pushes to be made stable
  2022-03-16 10:34   ` Chandan Babu R
@ 2022-03-16 23:24     ` Dave Chinner
  2022-03-17  6:49       ` Chandan Babu R
  0 siblings, 1 reply; 29+ messages in thread
From: Dave Chinner @ 2022-03-16 23:24 UTC (permalink / raw)
  To: Chandan Babu R; +Cc: linux-xfs

On Wed, Mar 16, 2022 at 04:04:55PM +0530, Chandan Babu R wrote:
> On 15 Mar 2022 at 12:12, Dave Chinner wrote:
> > From: Dave Chinner <dchinner@redhat.com>
> >
> > When the AIL tries to flush the CIL, it relies on the CIL push
> > ending up on stable storage without having to wait for and
> > manipulate iclog state directly. However, if there is already a
> > pending CIL push when the AIL tries to flush the CIL, it won't set
> > the cil->xc_push_commit_stable flag and so the CIL push will not
> > actively flush the commit record iclog.
> 
> I think the above sentence maps to the following snippet from
> xlog_cil_push_now(),
> 
> 	if (list_empty(&cil->xc_cil) || push_seq <= cil->xc_push_seq) {
> 		spin_unlock(&cil->xc_push_lock);
> 		return;
> 	}
> 
> i.e. if the CIL sequence that we are trying to push is already being pushed
> then xlog_cil_push_now() returns without queuing work on cil->xc_push_wq.
> 
> However, the push_seq could have been previously pushed by,
> 1. xfsaild_push()
>    In this case, cil->xc_push_commit_stable is set to true. Hence,
>    xlog_cil_push_work() will definitely make sure to submit the commit record
>    iclog for write I/O.
> 2. xfs_log_force_seq() => xlog_cil_force_seq()
>    xfs_log_force_seq() invokes xlog_force_lsn() after executing
>    xlog_cil_force_seq(). Here, A partially filled iclog will be in
>    XLOG_STATE_ACTIVE state. This will cause xlog_force_and_check_iclog() to be
>    invoked and hence the iclog is submitted for write I/O.
> 
> In both the cases listed above, iclog is guaranteed to be submitted for I/O
> without any help from the log worker task.
> 
> Looks like I am missing something obvious here.

Pushes triggered by xlog_cil_push_background() can complete leaving
the partially filled iclog in ACTIVE state. Then xlog_cil_push_now()
does nothing because it doesn't trigger a new CIL push and so
setting the cil->xc_push_commit_stable flag doesn't trigger a flush
of the ACTIVE iclog.

The AIL flush does not use xfs_log_force_seq() because that blocks
waiting for the entire CIL to hit the disk before it can force the
last iclog to disk. Hence the second piece of this patch is
necessary, and that is to call xfs_log_force() if the CIL is empty
(i.e. the case where xlog_cil_push_now() is a no-op because the
CIL is empty due to background pushes).


Cheers,

Dave.
-- 
Dave Chinner
david@fromorbit.com

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

* Re: [PATCH 4/7] xfs: async CIL flushes need pending pushes to be made stable
  2022-03-16 23:24     ` Dave Chinner
@ 2022-03-17  6:49       ` Chandan Babu R
  0 siblings, 0 replies; 29+ messages in thread
From: Chandan Babu R @ 2022-03-17  6:49 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs

On 17 Mar 2022 at 04:54, Dave Chinner wrote:
> On Wed, Mar 16, 2022 at 04:04:55PM +0530, Chandan Babu R wrote:
>> On 15 Mar 2022 at 12:12, Dave Chinner wrote:
>> > From: Dave Chinner <dchinner@redhat.com>
>> >
>> > When the AIL tries to flush the CIL, it relies on the CIL push
>> > ending up on stable storage without having to wait for and
>> > manipulate iclog state directly. However, if there is already a
>> > pending CIL push when the AIL tries to flush the CIL, it won't set
>> > the cil->xc_push_commit_stable flag and so the CIL push will not
>> > actively flush the commit record iclog.
>> 
>> I think the above sentence maps to the following snippet from
>> xlog_cil_push_now(),
>> 
>> 	if (list_empty(&cil->xc_cil) || push_seq <= cil->xc_push_seq) {
>> 		spin_unlock(&cil->xc_push_lock);
>> 		return;
>> 	}
>> 
>> i.e. if the CIL sequence that we are trying to push is already being pushed
>> then xlog_cil_push_now() returns without queuing work on cil->xc_push_wq.
>> 
>> However, the push_seq could have been previously pushed by,
>> 1. xfsaild_push()
>>    In this case, cil->xc_push_commit_stable is set to true. Hence,
>>    xlog_cil_push_work() will definitely make sure to submit the commit record
>>    iclog for write I/O.
>> 2. xfs_log_force_seq() => xlog_cil_force_seq()
>>    xfs_log_force_seq() invokes xlog_force_lsn() after executing
>>    xlog_cil_force_seq(). Here, A partially filled iclog will be in
>>    XLOG_STATE_ACTIVE state. This will cause xlog_force_and_check_iclog() to be
>>    invoked and hence the iclog is submitted for write I/O.
>> 
>> In both the cases listed above, iclog is guaranteed to be submitted for I/O
>> without any help from the log worker task.
>> 
>> Looks like I am missing something obvious here.
>
> Pushes triggered by xlog_cil_push_background() can complete leaving
> the partially filled iclog in ACTIVE state. Then xlog_cil_push_now()
> does nothing because it doesn't trigger a new CIL push and so
> setting the cil->xc_push_commit_stable flag doesn't trigger a flush
> of the ACTIVE iclog.

Ah. I had missed xlog_cil_push_background().

>
> The AIL flush does not use xfs_log_force_seq() because that blocks
> waiting for the entire CIL to hit the disk before it can force the
> last iclog to disk. Hence the second piece of this patch is
> necessary, and that is to call xfs_log_force() if the CIL is empty
> (i.e. the case where xlog_cil_push_now() is a no-op because the
> CIL is empty due to background pushes).
>

Thanks for clarifying my doubts.

-- 
chandan

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

end of thread, other threads:[~2022-03-17  6:50 UTC | newest]

Thread overview: 29+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2022-03-15  6:42 [PATCH 0/7 v3] xfs: log recovery fixes Dave Chinner
2022-03-15  6:42 ` [PATCH 1/7] xfs: log worker needs to start before intent/unlink recovery Dave Chinner
2022-03-15  9:14   ` Chandan Babu R
2022-03-15  6:42 ` [PATCH 2/7] xfs: check buffer pin state after locking in delwri_submit Dave Chinner
2022-03-15 10:04   ` Chandan Babu R
2022-03-15 19:13   ` Darrick J. Wong
2022-03-15 21:11     ` Dave Chinner
2022-03-15 22:42       ` Darrick J. Wong
2022-03-15  6:42 ` [PATCH 3/7] xfs: xfs_ail_push_all_sync() stalls when racing with updates Dave Chinner
2022-03-15 15:14   ` Chandan Babu R
2022-03-15 19:17   ` Darrick J. Wong
2022-03-15 21:29     ` Dave Chinner
2022-03-15  6:42 ` [PATCH 4/7] xfs: async CIL flushes need pending pushes to be made stable Dave Chinner
2022-03-15 19:36   ` Darrick J. Wong
2022-03-15 21:47     ` Dave Chinner
2022-03-16  2:00       ` Darrick J. Wong
2022-03-16 10:34   ` Chandan Babu R
2022-03-16 23:24     ` Dave Chinner
2022-03-17  6:49       ` Chandan Babu R
2022-03-15  6:42 ` [PATCH 5/7] xfs: log items should have a xlog pointer, not a mount Dave Chinner
2022-03-15 19:37   ` Darrick J. Wong
2022-03-16 11:06   ` Chandan Babu R
2022-03-15  6:42 ` [PATCH 6/7] xfs: AIL should be log centric Dave Chinner
2022-03-15 19:39   ` Darrick J. Wong
2022-03-16 11:12   ` Chandan Babu R
2022-03-15  6:42 ` [PATCH 7/7] xfs: xfs_is_shutdown vs xlog_is_shutdown cage fight Dave Chinner
2022-03-15 20:03   ` Darrick J. Wong
2022-03-15 22:20     ` Dave Chinner
2022-03-16  1:22       ` 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.