linux-xfs.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCHSET v23.1 0/5] xfs: drain deferred work items when scrubbing
@ 2022-10-02 18:19 Darrick J. Wong
  2022-10-02 18:19 ` [PATCH 1/5] xfs: allow queued AG intents to drain before scrubbing Darrick J. Wong
                   ` (4 more replies)
  0 siblings, 5 replies; 6+ messages in thread
From: Darrick J. Wong @ 2022-10-02 18:19 UTC (permalink / raw)
  To: djwong; +Cc: linux-xfs

Hi all,

The design doc for XFS online fsck contains a long discussion of the
eventual consistency models in use for XFS metadata.  In that chapter,
we note that it is possible for scrub to collide with a chain of
deferred space metadata updates, and proposes a lightweight solution:
The use of a pending-intents counter so that scrub can wait for the
system to drain all chains.

This patchset implements that scrub drain.  The first patch implements
the basic mechanism, and the subsequent patches reduce the runtime
overhead by converting the implementation to use sloppy counters and
introducing jump labels to avoid walking into scrub hooks when it isn't
running.  This last paradigm repeats elsewhere in this megaseries.

If you're going to start using this mess, you probably ought to just
pull from my git trees, which are linked below.

This is an extraordinary way to destroy everything.  Enjoy!
Comments and questions are, as always, welcome.

--D

kernel git tree:
https://git.kernel.org/cgit/linux/kernel/git/djwong/xfs-linux.git/log/?h=scrub-drain-intents

xfsprogs git tree:
https://git.kernel.org/cgit/linux/kernel/git/djwong/xfsprogs-dev.git/log/?h=scrub-drain-intents
---
 fs/xfs/Kconfig             |    5 ++
 fs/xfs/libxfs/xfs_ag.c     |    6 ++
 fs/xfs/libxfs/xfs_ag.h     |    8 +++
 fs/xfs/libxfs/xfs_defer.c  |   10 ++-
 fs/xfs/libxfs/xfs_defer.h  |    3 +
 fs/xfs/scrub/agheader.c    |    9 +++
 fs/xfs/scrub/alloc.c       |    3 +
 fs/xfs/scrub/bmap.c        |    3 +
 fs/xfs/scrub/btree.c       |    1 
 fs/xfs/scrub/common.c      |  129 ++++++++++++++++++++++++++++++++++++++++----
 fs/xfs/scrub/common.h      |   15 +++++
 fs/xfs/scrub/dabtree.c     |    1 
 fs/xfs/scrub/fscounters.c  |    7 ++
 fs/xfs/scrub/health.c      |    2 +
 fs/xfs/scrub/ialloc.c      |    2 +
 fs/xfs/scrub/inode.c       |    3 +
 fs/xfs/scrub/quota.c       |    3 +
 fs/xfs/scrub/refcount.c    |    4 +
 fs/xfs/scrub/repair.c      |    3 +
 fs/xfs/scrub/rmap.c        |    3 +
 fs/xfs/scrub/scrub.c       |   63 ++++++++++++++++-----
 fs/xfs/scrub/scrub.h       |    6 ++
 fs/xfs/scrub/trace.h       |   34 ++++++++++++
 fs/xfs/xfs_attr_item.c     |    1 
 fs/xfs/xfs_bmap_item.c     |   48 ++++++++++++++++
 fs/xfs/xfs_extfree_item.c  |   30 ++++++++++
 fs/xfs/xfs_mount.c         |  105 ++++++++++++++++++++++++++++++++++++
 fs/xfs/xfs_mount.h         |   64 ++++++++++++++++++++++
 fs/xfs/xfs_refcount_item.c |   25 +++++++++
 fs/xfs/xfs_rmap_item.c     |   18 ++++++
 fs/xfs/xfs_trace.h         |   71 ++++++++++++++++++++++++
 31 files changed, 652 insertions(+), 33 deletions(-)


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

* [PATCH 1/5] xfs: allow queued AG intents to drain before scrubbing
  2022-10-02 18:19 [PATCHSET v23.1 0/5] xfs: drain deferred work items when scrubbing Darrick J. Wong
@ 2022-10-02 18:19 ` Darrick J. Wong
  2022-10-02 18:19 ` [PATCH 4/5] xfs: minimize overhead of drain wakeups by using jump labels Darrick J. Wong
                   ` (3 subsequent siblings)
  4 siblings, 0 replies; 6+ messages in thread
From: Darrick J. Wong @ 2022-10-02 18:19 UTC (permalink / raw)
  To: djwong; +Cc: linux-xfs

From: Darrick J. Wong <djwong@kernel.org>

When a writer thread executes a chain of log intent items, the AG header
buffer locks will cycle during a transaction roll to get from one intent
item to the next in a chain.  Although scrub takes all AG header buffer
locks, this isn't sufficient to guard against scrub checking an AG while
that writer thread is in the middle of finishing a chain because there's
no higher level locking primitive guarding allocation groups.

When there's a collision, cross-referencing between data structures
(e.g. rmapbt and refcountbt) yields false corruption events; if repair
is running, this results in incorrect repairs, which is catastrophic.

Fix this by adding to the perag structure the count of active intents
and make scrub wait until it has both AG header buffer locks and the
intent counter reaches zero.  This is a little stupid since transactions
can queue intents without taking buffer locks, but it's not the end of
the world for scrub to wait (in KILLABLE state) for those transactions.

In the next patch we'll improve on this facility, but this patch
provides the basic functionality.

Signed-off-by: Darrick J. Wong <djwong@kernel.org>
---
 fs/xfs/Kconfig             |    4 ++
 fs/xfs/libxfs/xfs_ag.c     |    4 ++
 fs/xfs/libxfs/xfs_ag.h     |    8 +++
 fs/xfs/libxfs/xfs_defer.c  |   10 +++-
 fs/xfs/libxfs/xfs_defer.h  |    3 +
 fs/xfs/scrub/common.c      |  103 +++++++++++++++++++++++++++++++++++++++-----
 fs/xfs/scrub/health.c      |    2 +
 fs/xfs/scrub/refcount.c    |    2 +
 fs/xfs/xfs_attr_item.c     |    1 
 fs/xfs/xfs_bmap_item.c     |   48 ++++++++++++++++++++-
 fs/xfs/xfs_extfree_item.c  |   30 +++++++++++++
 fs/xfs/xfs_mount.c         |   76 ++++++++++++++++++++++++++++++++
 fs/xfs/xfs_mount.h         |   52 ++++++++++++++++++++++
 fs/xfs/xfs_refcount_item.c |   25 +++++++++++
 fs/xfs/xfs_rmap_item.c     |   18 ++++++++
 fs/xfs/xfs_trace.h         |   71 ++++++++++++++++++++++++++++++
 16 files changed, 441 insertions(+), 16 deletions(-)


diff --git a/fs/xfs/Kconfig b/fs/xfs/Kconfig
index 8618ad0f84cd..a4eff514b647 100644
--- a/fs/xfs/Kconfig
+++ b/fs/xfs/Kconfig
@@ -93,10 +93,14 @@ config XFS_RT
 
 	  If unsure, say N.
 
+config XFS_DRAIN_INTENTS
+	bool
+
 config XFS_ONLINE_SCRUB
 	bool "XFS online metadata check support"
 	default n
 	depends on XFS_FS
+	select XFS_DRAIN_INTENTS
 	help
 	  If you say Y here you will be able to check metadata on a
 	  mounted XFS filesystem.  This feature is intended to reduce
diff --git a/fs/xfs/libxfs/xfs_ag.c b/fs/xfs/libxfs/xfs_ag.c
index bb0c700afe3c..27ba30f8f64b 100644
--- a/fs/xfs/libxfs/xfs_ag.c
+++ b/fs/xfs/libxfs/xfs_ag.c
@@ -192,6 +192,7 @@ xfs_free_perag(
 		spin_unlock(&mp->m_perag_lock);
 		ASSERT(pag);
 		XFS_IS_CORRUPT(pag->pag_mount, atomic_read(&pag->pag_ref) != 0);
+		xfs_drain_free(&pag->pag_intents);
 
 		cancel_delayed_work_sync(&pag->pag_blockgc_work);
 		xfs_buf_hash_destroy(pag);
@@ -313,6 +314,7 @@ xfs_initialize_perag(
 		spin_lock_init(&pag->pag_state_lock);
 		INIT_DELAYED_WORK(&pag->pag_blockgc_work, xfs_blockgc_worker);
 		INIT_RADIX_TREE(&pag->pag_ici_root, GFP_ATOMIC);
+		xfs_drain_init(&pag->pag_intents);
 		init_waitqueue_head(&pag->pagb_wait);
 		pag->pagb_count = 0;
 		pag->pagb_tree = RB_ROOT;
@@ -345,6 +347,7 @@ xfs_initialize_perag(
 	return 0;
 
 out_remove_pag:
+	xfs_drain_free(&pag->pag_intents);
 	radix_tree_delete(&mp->m_perag_tree, index);
 out_free_pag:
 	kmem_free(pag);
@@ -355,6 +358,7 @@ xfs_initialize_perag(
 		if (!pag)
 			break;
 		xfs_buf_hash_destroy(pag);
+		xfs_drain_free(&pag->pag_intents);
 		kmem_free(pag);
 	}
 	return error;
diff --git a/fs/xfs/libxfs/xfs_ag.h b/fs/xfs/libxfs/xfs_ag.h
index 517a138faa66..73dfdf2119ee 100644
--- a/fs/xfs/libxfs/xfs_ag.h
+++ b/fs/xfs/libxfs/xfs_ag.h
@@ -103,6 +103,14 @@ struct xfs_perag {
 	/* background prealloc block trimming */
 	struct delayed_work	pag_blockgc_work;
 
+	/*
+	 * We use xfs_drain to track the number of deferred log intent items
+	 * that have been queued (but not yet processed) so that waiters (e.g.
+	 * scrub) will not lock resources when other threads are in the middle
+	 * of processing a chain of intent items only to find momentary
+	 * inconsistencies.
+	 */
+	struct xfs_drain	pag_intents;
 #endif /* __KERNEL__ */
 };
 
diff --git a/fs/xfs/libxfs/xfs_defer.c b/fs/xfs/libxfs/xfs_defer.c
index 5a321b783398..7e977b1ef505 100644
--- a/fs/xfs/libxfs/xfs_defer.c
+++ b/fs/xfs/libxfs/xfs_defer.c
@@ -397,7 +397,8 @@ xfs_defer_cancel_list(
 		list_for_each_safe(pwi, n, &dfp->dfp_work) {
 			list_del(pwi);
 			dfp->dfp_count--;
-			ops->cancel_item(pwi);
+			trace_xfs_defer_cancel_item(mp, dfp, pwi);
+			ops->cancel_item(mp, pwi);
 		}
 		ASSERT(dfp->dfp_count == 0);
 		kmem_cache_free(xfs_defer_pending_cache, dfp);
@@ -476,6 +477,7 @@ xfs_defer_finish_one(
 	list_for_each_safe(li, n, &dfp->dfp_work) {
 		list_del(li);
 		dfp->dfp_count--;
+		trace_xfs_defer_finish_item(tp->t_mountp, dfp, li);
 		error = ops->finish_item(tp, dfp->dfp_done, li, &state);
 		if (error == -EAGAIN) {
 			int		ret;
@@ -623,7 +625,7 @@ xfs_defer_add(
 	struct list_head		*li)
 {
 	struct xfs_defer_pending	*dfp = NULL;
-	const struct xfs_defer_op_type	*ops;
+	const struct xfs_defer_op_type	*ops = defer_op_types[type];
 
 	ASSERT(tp->t_flags & XFS_TRANS_PERM_LOG_RES);
 	BUILD_BUG_ON(ARRAY_SIZE(defer_op_types) != XFS_DEFER_OPS_TYPE_MAX);
@@ -636,7 +638,6 @@ xfs_defer_add(
 	if (!list_empty(&tp->t_dfops)) {
 		dfp = list_last_entry(&tp->t_dfops,
 				struct xfs_defer_pending, dfp_list);
-		ops = defer_op_types[dfp->dfp_type];
 		if (dfp->dfp_type != type ||
 		    (ops->max_items && dfp->dfp_count >= ops->max_items))
 			dfp = NULL;
@@ -653,6 +654,9 @@ xfs_defer_add(
 	}
 
 	list_add_tail(li, &dfp->dfp_work);
+	trace_xfs_defer_add_item(tp->t_mountp, dfp, li);
+	if (ops->add_item)
+		ops->add_item(tp->t_mountp, li);
 	dfp->dfp_count++;
 }
 
diff --git a/fs/xfs/libxfs/xfs_defer.h b/fs/xfs/libxfs/xfs_defer.h
index 114a3a4930a3..6cce34fcefe2 100644
--- a/fs/xfs/libxfs/xfs_defer.h
+++ b/fs/xfs/libxfs/xfs_defer.h
@@ -55,7 +55,8 @@ struct xfs_defer_op_type {
 			struct list_head *item, struct xfs_btree_cur **state);
 	void (*finish_cleanup)(struct xfs_trans *tp,
 			struct xfs_btree_cur *state, int error);
-	void (*cancel_item)(struct list_head *item);
+	void (*cancel_item)(struct xfs_mount *mp, struct list_head *item);
+	void (*add_item)(struct xfs_mount *mp, const struct list_head *item);
 	unsigned int		max_items;
 };
 
diff --git a/fs/xfs/scrub/common.c b/fs/xfs/scrub/common.c
index ad70f29233c3..cc4882c0cfc2 100644
--- a/fs/xfs/scrub/common.c
+++ b/fs/xfs/scrub/common.c
@@ -396,26 +396,19 @@ want_ag_read_header_failure(
 }
 
 /*
- * Grab the perag structure and all the headers for an AG.
+ * Grab the AG header buffers for the attached perag structure.
  *
  * The headers should be released by xchk_ag_free, but as a fail safe we attach
  * all the buffers we grab to the scrub transaction so they'll all be freed
- * when we cancel it.  Returns ENOENT if we can't grab the perag structure.
+ * when we cancel it.
  */
-int
-xchk_ag_read_headers(
+static inline int
+xchk_perag_read_headers(
 	struct xfs_scrub	*sc,
-	xfs_agnumber_t		agno,
 	struct xchk_ag		*sa)
 {
-	struct xfs_mount	*mp = sc->mp;
 	int			error;
 
-	ASSERT(!sa->pag);
-	sa->pag = xfs_perag_get(mp, agno);
-	if (!sa->pag)
-		return -ENOENT;
-
 	error = xfs_ialloc_read_agi(sa->pag, sc->tp, &sa->agi_bp);
 	if (error && want_ag_read_header_failure(sc, XFS_SCRUB_TYPE_AGI))
 		return error;
@@ -427,6 +420,94 @@ xchk_ag_read_headers(
 	return 0;
 }
 
+/*
+ * Grab the AG headers for the attached perag structure and wait for pending
+ * intents to drain.
+ */
+static int
+xchk_perag_lock(
+	struct xfs_scrub	*sc)
+{
+	struct xchk_ag		*sa = &sc->sa;
+	int			error = 0;
+
+	ASSERT(sa->pag != NULL);
+	ASSERT(sa->agi_bp == NULL);
+	ASSERT(sa->agf_bp == NULL);
+
+	do {
+		if (xchk_should_terminate(sc, &error))
+			return error;
+
+		error = xchk_perag_read_headers(sc, sa);
+		if (error)
+			return error;
+
+		/*
+		 * Decide if this AG is quiet enough for all metadata to be
+		 * consistent with each other.  XFS allows the AG header buffer
+		 * locks to cycle across transaction rolls while processing
+		 * chains of deferred ops, which means that there could be
+		 * other threads in the middle of processing a chain of
+		 * deferred ops.  For regular operations we are careful about
+		 * ordering operations to prevent collisions between threads
+		 * (which is why we don't need a per-AG lock), but scrub and
+		 * repair have to serialize against chained operations.
+		 *
+		 * We just locked all the AG headers buffers; now take a look
+		 * to see if there are any intents in progress.  If there are,
+		 * drop the AG headers and wait for the intents to drain.
+		 * Since we hold all the AG header locks for the duration of
+		 * the scrub, this is the only time we have to sample the
+		 * intents counter; any threads increasing it after this point
+		 * can't possibly be in the middle of a chain of AG metadata
+		 * updates.
+		 *
+		 * Obviously, this should be slanted against scrub and in favor
+		 * of runtime threads.
+		 */
+		if (!xfs_ag_intents_busy(sa->pag))
+			return 0;
+
+		if (sa->agf_bp) {
+			xfs_trans_brelse(sc->tp, sa->agf_bp);
+			sa->agf_bp = NULL;
+		}
+
+		if (sa->agi_bp) {
+			xfs_trans_brelse(sc->tp, sa->agi_bp);
+			sa->agi_bp = NULL;
+		}
+
+		error = xfs_ag_drain_intents(sa->pag);
+		if (error == -ERESTARTSYS)
+			error = -EINTR;
+	} while (!error);
+
+	return error;
+}
+
+/*
+ * Grab the per-AG structure, grab all AG header buffers, and wait until there
+ * aren't any pending intents.  Returns -ENOENT if we can't grab the perag
+ * structure.
+ */
+int
+xchk_ag_read_headers(
+	struct xfs_scrub	*sc,
+	xfs_agnumber_t		agno,
+	struct xchk_ag		*sa)
+{
+	struct xfs_mount	*mp = sc->mp;
+
+	ASSERT(!sa->pag);
+	sa->pag = xfs_perag_get(mp, agno);
+	if (!sa->pag)
+		return -ENOENT;
+
+	return xchk_perag_lock(sc);
+}
+
 /* Release all the AG btree cursors. */
 void
 xchk_ag_btcur_free(
diff --git a/fs/xfs/scrub/health.c b/fs/xfs/scrub/health.c
index aa65ec88a0c0..f7c5a109615f 100644
--- a/fs/xfs/scrub/health.c
+++ b/fs/xfs/scrub/health.c
@@ -7,6 +7,8 @@
 #include "xfs_fs.h"
 #include "xfs_shared.h"
 #include "xfs_format.h"
+#include "xfs_trans_resv.h"
+#include "xfs_mount.h"
 #include "xfs_btree.h"
 #include "xfs_trans_resv.h"
 #include "xfs_mount.h"
diff --git a/fs/xfs/scrub/refcount.c b/fs/xfs/scrub/refcount.c
index f037e73922c1..e4a550f8d1b7 100644
--- a/fs/xfs/scrub/refcount.c
+++ b/fs/xfs/scrub/refcount.c
@@ -7,6 +7,8 @@
 #include "xfs_fs.h"
 #include "xfs_shared.h"
 #include "xfs_format.h"
+#include "xfs_trans_resv.h"
+#include "xfs_mount.h"
 #include "xfs_btree.h"
 #include "xfs_rmap.h"
 #include "xfs_refcount.h"
diff --git a/fs/xfs/xfs_attr_item.c b/fs/xfs/xfs_attr_item.c
index 5077a7ad5646..4d3722281845 100644
--- a/fs/xfs/xfs_attr_item.c
+++ b/fs/xfs/xfs_attr_item.c
@@ -504,6 +504,7 @@ xfs_attr_abort_intent(
 /* Cancel an attr */
 STATIC void
 xfs_attr_cancel_item(
+	struct xfs_mount		*mp,
 	struct list_head		*item)
 {
 	struct xfs_attr_intent		*attr;
diff --git a/fs/xfs/xfs_bmap_item.c b/fs/xfs/xfs_bmap_item.c
index 51f66e982484..996e9fd146f8 100644
--- a/fs/xfs/xfs_bmap_item.c
+++ b/fs/xfs/xfs_bmap_item.c
@@ -370,6 +370,18 @@ xfs_bmap_update_create_done(
 	return &xfs_trans_get_bud(tp, BUI_ITEM(intent))->bud_item;
 }
 
+static inline void
+xfs_bmap_drop_intents(
+	struct xfs_mount		*mp,
+	const struct xfs_bmap_intent	*bi,
+	xfs_fsblock_t			orig_startblock)
+{
+	if (!xfs_has_rmapbt(mp))
+		return;
+
+	xfs_fs_drop_intents(mp, orig_startblock);
+}
+
 /* Process a deferred rmap update. */
 STATIC int
 xfs_bmap_update_finish_item(
@@ -379,10 +391,13 @@ xfs_bmap_update_finish_item(
 	struct xfs_btree_cur		**state)
 {
 	struct xfs_bmap_intent		*bmap;
+	struct xfs_mount		*mp = tp->t_mountp;
+	xfs_fsblock_t			orig_startblock;
 	xfs_filblks_t			count;
 	int				error;
 
 	bmap = container_of(item, struct xfs_bmap_intent, bi_list);
+	orig_startblock = bmap->bi_bmap.br_startblock;
 	count = bmap->bi_bmap.br_blockcount;
 	error = xfs_trans_log_finish_bmap_update(tp, BUD_ITEM(done),
 			bmap->bi_type,
@@ -396,6 +411,14 @@ xfs_bmap_update_finish_item(
 		bmap->bi_bmap.br_blockcount = count;
 		return -EAGAIN;
 	}
+
+	/*
+	 * Drop our intent counter reference now that we've either queued a
+	 * deferred rmap intent or failed.  Be careful to use the original
+	 * startblock since the finishing functions can update the intent
+	 * state.
+	 */
+	xfs_bmap_drop_intents(mp, bmap, orig_startblock);
 	kmem_cache_free(xfs_bmap_intent_cache, bmap);
 	return error;
 }
@@ -408,17 +431,39 @@ xfs_bmap_update_abort_intent(
 	xfs_bui_release(BUI_ITEM(intent));
 }
 
-/* Cancel a deferred rmap update. */
+/* Cancel a deferred bmap update. */
 STATIC void
 xfs_bmap_update_cancel_item(
+	struct xfs_mount		*mp,
 	struct list_head		*item)
 {
 	struct xfs_bmap_intent		*bmap;
 
 	bmap = container_of(item, struct xfs_bmap_intent, bi_list);
+	xfs_bmap_drop_intents(mp, bmap, bmap->bi_bmap.br_startblock);
 	kmem_cache_free(xfs_bmap_intent_cache, bmap);
 }
 
+/* Add a deferred bmap update. */
+STATIC void
+xfs_bmap_update_add_item(
+	struct xfs_mount		*mp,
+	const struct list_head		*item)
+{
+	const struct xfs_bmap_intent	*bi;
+
+	bi = container_of(item, struct xfs_bmap_intent, bi_list);
+
+	/*
+	 * Grab an intent counter reference on behalf of the deferred rmap
+	 * intent item that we will queue when we finish this bmap work.
+	 */
+	if (!xfs_has_rmapbt(mp))
+		return;
+
+	xfs_fs_bump_intents(mp, bi->bi_bmap.br_startblock);
+}
+
 const struct xfs_defer_op_type xfs_bmap_update_defer_type = {
 	.max_items	= XFS_BUI_MAX_FAST_EXTENTS,
 	.create_intent	= xfs_bmap_update_create_intent,
@@ -426,6 +471,7 @@ const struct xfs_defer_op_type xfs_bmap_update_defer_type = {
 	.create_done	= xfs_bmap_update_create_done,
 	.finish_item	= xfs_bmap_update_finish_item,
 	.cancel_item	= xfs_bmap_update_cancel_item,
+	.add_item	= xfs_bmap_update_add_item,
 };
 
 /* Is this recovered BUI ok? */
diff --git a/fs/xfs/xfs_extfree_item.c b/fs/xfs/xfs_extfree_item.c
index 27ccfcd82f04..92032f2b7be0 100644
--- a/fs/xfs/xfs_extfree_item.c
+++ b/fs/xfs/xfs_extfree_item.c
@@ -477,6 +477,14 @@ xfs_extent_free_create_done(
 	return &xfs_trans_get_efd(tp, EFI_ITEM(intent), count)->efd_item;
 }
 
+static inline void
+xfs_extent_free_drop_intents(
+	struct xfs_mount			*mp,
+	const struct xfs_extent_free_item	*xefi)
+{
+	xfs_fs_drop_intents(mp, xefi->xefi_startblock);
+}
+
 /* Process a free extent. */
 STATIC int
 xfs_extent_free_finish_item(
@@ -486,6 +494,7 @@ xfs_extent_free_finish_item(
 	struct xfs_btree_cur		**state)
 {
 	struct xfs_owner_info		oinfo = { };
+	struct xfs_mount		*mp = tp->t_mountp;
 	struct xfs_extent_free_item	*free;
 	int				error;
 
@@ -499,6 +508,8 @@ xfs_extent_free_finish_item(
 			free->xefi_startblock,
 			free->xefi_blockcount,
 			&oinfo, free->xefi_flags & XFS_EFI_SKIP_DISCARD);
+
+	xfs_extent_free_drop_intents(mp, free);
 	kmem_cache_free(xfs_extfree_item_cache, free);
 	return error;
 }
@@ -514,14 +525,30 @@ xfs_extent_free_abort_intent(
 /* Cancel a free extent. */
 STATIC void
 xfs_extent_free_cancel_item(
+	struct xfs_mount		*mp,
 	struct list_head		*item)
 {
 	struct xfs_extent_free_item	*free;
 
 	free = container_of(item, struct xfs_extent_free_item, xefi_list);
+	xfs_extent_free_drop_intents(mp, free);
 	kmem_cache_free(xfs_extfree_item_cache, free);
 }
 
+/* Add a deferred free extent. */
+STATIC void
+xfs_extent_free_add_item(
+	struct xfs_mount		*mp,
+	const struct list_head		*item)
+{
+	const struct xfs_extent_free_item *xefi;
+
+	xefi = container_of(item, struct xfs_extent_free_item, xefi_list);
+
+	/* Grab an intent counter reference for this intent item. */
+	xfs_fs_bump_intents(mp, xefi->xefi_startblock);
+}
+
 const struct xfs_defer_op_type xfs_extent_free_defer_type = {
 	.max_items	= XFS_EFI_MAX_FAST_EXTENTS,
 	.create_intent	= xfs_extent_free_create_intent,
@@ -529,6 +556,7 @@ const struct xfs_defer_op_type xfs_extent_free_defer_type = {
 	.create_done	= xfs_extent_free_create_done,
 	.finish_item	= xfs_extent_free_finish_item,
 	.cancel_item	= xfs_extent_free_cancel_item,
+	.add_item	= xfs_extent_free_add_item,
 };
 
 /*
@@ -585,6 +613,7 @@ xfs_agfl_free_finish_item(
 	extp->ext_len = free->xefi_blockcount;
 	efdp->efd_next_extent++;
 
+	xfs_extent_free_drop_intents(mp, free);
 	kmem_cache_free(xfs_extfree_item_cache, free);
 	return error;
 }
@@ -597,6 +626,7 @@ const struct xfs_defer_op_type xfs_agfl_free_defer_type = {
 	.create_done	= xfs_extent_free_create_done,
 	.finish_item	= xfs_agfl_free_finish_item,
 	.cancel_item	= xfs_extent_free_cancel_item,
+	.add_item	= xfs_extent_free_add_item,
 };
 
 /* Is this recovered EFI ok? */
diff --git a/fs/xfs/xfs_mount.c b/fs/xfs/xfs_mount.c
index f10c88cee116..6c84c6547a0b 100644
--- a/fs/xfs/xfs_mount.c
+++ b/fs/xfs/xfs_mount.c
@@ -1385,3 +1385,79 @@ xfs_mod_delalloc(
 	percpu_counter_add_batch(&mp->m_delalloc_blks, delta,
 			XFS_DELALLOC_BATCH);
 }
+
+#ifdef CONFIG_XFS_DRAIN_INTENTS
+/* Increase the pending intent count. */
+static inline void xfs_drain_bump(struct xfs_drain *dr)
+{
+	atomic_inc(&dr->dr_count);
+}
+
+/* Decrease the pending intent count, and wake any waiters, if appropriate. */
+static inline void xfs_drain_drop(struct xfs_drain *dr)
+{
+	if (atomic_dec_and_test(&dr->dr_count) &&
+	    wq_has_sleeper(&dr->dr_waiters))
+		wake_up(&dr->dr_waiters);
+}
+
+/*
+ * Wait for the pending intent count for a drain to hit zero.
+ *
+ * Callers must not hold any locks that would prevent intents from being
+ * finished.
+ */
+static inline int xfs_drain_wait(struct xfs_drain *dr)
+{
+	return wait_event_killable(dr->dr_waiters, !xfs_drain_busy(dr));
+}
+
+/* Add an item to the pending count. */
+void
+xfs_fs_bump_intents(
+	struct xfs_mount	*mp,
+	xfs_fsblock_t		fsb)
+{
+	struct xfs_perag	*pag;
+
+	pag = xfs_perag_get(mp, XFS_FSB_TO_AGNO(mp, fsb));
+	trace_xfs_perag_bump_intents(pag, __return_address);
+	xfs_drain_bump(&pag->pag_intents);
+	xfs_perag_put(pag);
+}
+
+/* Remove an item from the pending count. */
+void
+xfs_fs_drop_intents(
+	struct xfs_mount	*mp,
+	xfs_fsblock_t		fsb)
+{
+	struct xfs_perag	*pag;
+
+	pag = xfs_perag_get(mp, XFS_FSB_TO_AGNO(mp, fsb));
+	trace_xfs_perag_drop_intents(pag, __return_address);
+	xfs_drain_drop(&pag->pag_intents);
+	xfs_perag_put(pag);
+}
+
+/*
+ * Wait for the pending intent count for AG metadata to hit zero.
+ * Callers must not hold any AG header buffers.
+ */
+int
+xfs_ag_drain_intents(
+	struct xfs_perag	*pag)
+{
+	trace_xfs_perag_wait_intents(pag, __return_address);
+	return xfs_drain_wait(&pag->pag_intents);
+}
+
+/* Might someone else be processing intents for this AG? */
+bool
+xfs_ag_intents_busy(
+	struct xfs_perag	*pag)
+{
+	return xfs_drain_busy(&pag->pag_intents);
+}
+
+#endif /* CONFIG_XFS_DRAIN_INTENTS */
diff --git a/fs/xfs/xfs_mount.h b/fs/xfs/xfs_mount.h
index 8aca2cc173ac..ddf438701022 100644
--- a/fs/xfs/xfs_mount.h
+++ b/fs/xfs/xfs_mount.h
@@ -56,6 +56,58 @@ struct xfs_error_cfg {
 	long		retry_timeout;	/* in jiffies, -1 = infinite */
 };
 
+#ifdef CONFIG_XFS_DRAIN_INTENTS
+/*
+ * Passive drain mechanism.  This data structure tracks a count of some items
+ * and contains a waitqueue for callers who would like to wake up when the
+ * count hits zero.
+ */
+struct xfs_drain {
+	/* Number of items pending in some part of the filesystem. */
+	atomic_t		dr_count;
+
+	/* Queue to wait for dri_count to go to zero */
+	struct wait_queue_head	dr_waiters;
+};
+
+int xfs_ag_drain_intents(struct xfs_perag *pag);
+bool xfs_ag_intents_busy(struct xfs_perag *pag);
+
+void xfs_fs_bump_intents(struct xfs_mount *mp, xfs_fsblock_t fsb);
+void xfs_fs_drop_intents(struct xfs_mount *mp, xfs_fsblock_t fsb);
+
+/* Are there work items pending? */
+static inline bool xfs_drain_busy(struct xfs_drain *dr)
+{
+	return atomic_read(&dr->dr_count) > 0;
+}
+
+static inline void xfs_drain_init(struct xfs_drain *dr)
+{
+	atomic_set(&dr->dr_count, 0);
+	init_waitqueue_head(&dr->dr_waiters);
+}
+
+static inline void xfs_drain_free(struct xfs_drain *dr)
+{
+	ASSERT(!xfs_drain_busy(dr));
+}
+#else
+struct xfs_drain { /* empty */ };
+
+static inline void
+xfs_fs_bump_intents(struct xfs_mount *mp, xfs_fsblock_t fsb)
+{
+}
+
+static inline void
+xfs_fs_drop_intents(struct xfs_mount *mp, xfs_fsblock_t fsb)
+{
+}
+# define xfs_drain_init(dr)	((void)0)
+# define xfs_drain_free(dr)	((void)0)
+#endif /* CONFIG_XFS_DRAIN_INTENTS */
+
 /*
  * Per-cpu deferred inode inactivation GC lists.
  */
diff --git a/fs/xfs/xfs_refcount_item.c b/fs/xfs/xfs_refcount_item.c
index 7e97bf19793d..f16208b0929d 100644
--- a/fs/xfs/xfs_refcount_item.c
+++ b/fs/xfs/xfs_refcount_item.c
@@ -379,11 +379,14 @@ xfs_refcount_update_finish_item(
 	struct xfs_btree_cur		**state)
 {
 	struct xfs_refcount_intent	*refc;
+	struct xfs_mount		*mp = tp->t_mountp;
+	xfs_fsblock_t			orig_startblock;
 	xfs_fsblock_t			new_fsb;
 	xfs_extlen_t			new_aglen;
 	int				error;
 
 	refc = container_of(item, struct xfs_refcount_intent, ri_list);
+	orig_startblock = refc->ri_startblock;
 	error = xfs_trans_log_finish_refcount_update(tp, CUD_ITEM(done),
 			refc->ri_type, refc->ri_startblock, refc->ri_blockcount,
 			&new_fsb, &new_aglen, state);
@@ -396,6 +399,13 @@ xfs_refcount_update_finish_item(
 		refc->ri_blockcount = new_aglen;
 		return -EAGAIN;
 	}
+
+	/*
+	 * Drop our intent counter reference now that we've finished all the
+	 * work or failed.  Be careful to use the original startblock because
+	 * the finishing functions can update the intent state.
+	 */
+	xfs_fs_drop_intents(mp, orig_startblock);
 	kmem_cache_free(xfs_refcount_intent_cache, refc);
 	return error;
 }
@@ -411,14 +421,28 @@ xfs_refcount_update_abort_intent(
 /* Cancel a deferred refcount update. */
 STATIC void
 xfs_refcount_update_cancel_item(
+	struct xfs_mount		*mp,
 	struct list_head		*item)
 {
 	struct xfs_refcount_intent	*refc;
 
 	refc = container_of(item, struct xfs_refcount_intent, ri_list);
+	xfs_fs_drop_intents(mp, refc->ri_startblock);
 	kmem_cache_free(xfs_refcount_intent_cache, refc);
 }
 
+/* Add a deferred refcount update. */
+STATIC void
+xfs_refcount_update_add_item(
+	struct xfs_mount		*mp,
+	const struct list_head		*item)
+{
+	const struct xfs_refcount_intent *ri;
+
+	ri = container_of(item, struct xfs_refcount_intent, ri_list);
+	xfs_fs_bump_intents(mp, ri->ri_startblock);
+}
+
 const struct xfs_defer_op_type xfs_refcount_update_defer_type = {
 	.max_items	= XFS_CUI_MAX_FAST_EXTENTS,
 	.create_intent	= xfs_refcount_update_create_intent,
@@ -427,6 +451,7 @@ const struct xfs_defer_op_type xfs_refcount_update_defer_type = {
 	.finish_item	= xfs_refcount_update_finish_item,
 	.finish_cleanup = xfs_refcount_finish_one_cleanup,
 	.cancel_item	= xfs_refcount_update_cancel_item,
+	.add_item	= xfs_refcount_update_add_item,
 };
 
 /* Is this recovered CUI ok? */
diff --git a/fs/xfs/xfs_rmap_item.c b/fs/xfs/xfs_rmap_item.c
index fef92e02f3bb..b107eb4759b1 100644
--- a/fs/xfs/xfs_rmap_item.c
+++ b/fs/xfs/xfs_rmap_item.c
@@ -431,6 +431,7 @@ xfs_rmap_update_finish_item(
 	struct xfs_btree_cur		**state)
 {
 	struct xfs_rmap_intent		*rmap;
+	struct xfs_mount		*mp = tp->t_mountp;
 	int				error;
 
 	rmap = container_of(item, struct xfs_rmap_intent, ri_list);
@@ -439,6 +440,8 @@ xfs_rmap_update_finish_item(
 			rmap->ri_bmap.br_startoff, rmap->ri_bmap.br_startblock,
 			rmap->ri_bmap.br_blockcount, rmap->ri_bmap.br_state,
 			state);
+
+	xfs_fs_drop_intents(mp, rmap->ri_bmap.br_startblock);
 	kmem_cache_free(xfs_rmap_intent_cache, rmap);
 	return error;
 }
@@ -454,14 +457,28 @@ xfs_rmap_update_abort_intent(
 /* Cancel a deferred rmap update. */
 STATIC void
 xfs_rmap_update_cancel_item(
+	struct xfs_mount		*mp,
 	struct list_head		*item)
 {
 	struct xfs_rmap_intent		*rmap;
 
 	rmap = container_of(item, struct xfs_rmap_intent, ri_list);
+	xfs_fs_drop_intents(mp, rmap->ri_bmap.br_startblock);
 	kmem_cache_free(xfs_rmap_intent_cache, rmap);
 }
 
+/* Add a deferred rmap update. */
+STATIC void
+xfs_rmap_update_add_item(
+	struct xfs_mount		*mp,
+	const struct list_head		*item)
+{
+	const struct xfs_rmap_intent	*ri;
+
+	ri = container_of(item, struct xfs_rmap_intent, ri_list);
+	xfs_fs_bump_intents(mp, ri->ri_bmap.br_startblock);
+}
+
 const struct xfs_defer_op_type xfs_rmap_update_defer_type = {
 	.max_items	= XFS_RUI_MAX_FAST_EXTENTS,
 	.create_intent	= xfs_rmap_update_create_intent,
@@ -470,6 +487,7 @@ const struct xfs_defer_op_type xfs_rmap_update_defer_type = {
 	.finish_item	= xfs_rmap_update_finish_item,
 	.finish_cleanup = xfs_rmap_finish_one_cleanup,
 	.cancel_item	= xfs_rmap_update_cancel_item,
+	.add_item	= xfs_rmap_update_add_item,
 };
 
 /* Is this recovered RUI ok? */
diff --git a/fs/xfs/xfs_trace.h b/fs/xfs/xfs_trace.h
index f9057af6e0c8..fae3b0fe0971 100644
--- a/fs/xfs/xfs_trace.h
+++ b/fs/xfs/xfs_trace.h
@@ -2675,6 +2675,44 @@ DEFINE_BMAP_FREE_DEFERRED_EVENT(xfs_bmap_free_deferred);
 DEFINE_BMAP_FREE_DEFERRED_EVENT(xfs_agfl_free_defer);
 DEFINE_BMAP_FREE_DEFERRED_EVENT(xfs_agfl_free_deferred);
 
+DECLARE_EVENT_CLASS(xfs_defer_pending_item_class,
+	TP_PROTO(struct xfs_mount *mp, struct xfs_defer_pending *dfp,
+		 void *item),
+	TP_ARGS(mp, dfp, item),
+	TP_STRUCT__entry(
+		__field(dev_t, dev)
+		__field(int, type)
+		__field(void *, intent)
+		__field(void *, item)
+		__field(char, committed)
+		__field(int, nr)
+	),
+	TP_fast_assign(
+		__entry->dev = mp ? mp->m_super->s_dev : 0;
+		__entry->type = dfp->dfp_type;
+		__entry->intent = dfp->dfp_intent;
+		__entry->item = item;
+		__entry->committed = dfp->dfp_done != NULL;
+		__entry->nr = dfp->dfp_count;
+	),
+	TP_printk("dev %d:%d optype %d intent %p item %p committed %d nr %d",
+		  MAJOR(__entry->dev), MINOR(__entry->dev),
+		  __entry->type,
+		  __entry->intent,
+		  __entry->item,
+		  __entry->committed,
+		  __entry->nr)
+)
+#define DEFINE_DEFER_PENDING_ITEM_EVENT(name) \
+DEFINE_EVENT(xfs_defer_pending_item_class, name, \
+	TP_PROTO(struct xfs_mount *mp, struct xfs_defer_pending *dfp, \
+		 void *item), \
+	TP_ARGS(mp, dfp, item))
+
+DEFINE_DEFER_PENDING_ITEM_EVENT(xfs_defer_add_item);
+DEFINE_DEFER_PENDING_ITEM_EVENT(xfs_defer_cancel_item);
+DEFINE_DEFER_PENDING_ITEM_EVENT(xfs_defer_finish_item);
+
 /* rmap tracepoints */
 DECLARE_EVENT_CLASS(xfs_rmap_class,
 	TP_PROTO(struct xfs_mount *mp, xfs_agnumber_t agno,
@@ -4208,6 +4246,39 @@ TRACE_EVENT(xfs_force_shutdown,
 		__entry->line_num)
 );
 
+#ifdef CONFIG_XFS_DRAIN_INTENTS
+DECLARE_EVENT_CLASS(xfs_perag_intents_class,
+	TP_PROTO(struct xfs_perag *pag, void *caller_ip),
+	TP_ARGS(pag, caller_ip),
+	TP_STRUCT__entry(
+		__field(dev_t, dev)
+		__field(xfs_agnumber_t, agno)
+		__field(long, nr_intents)
+		__field(void *, caller_ip)
+	),
+	TP_fast_assign(
+		__entry->dev = pag->pag_mount->m_super->s_dev;
+		__entry->agno = pag->pag_agno;
+		__entry->nr_intents = atomic_read(&pag->pag_intents.dr_count);
+		__entry->caller_ip = caller_ip;
+	),
+	TP_printk("dev %d:%d agno 0x%x intents %ld caller %pS",
+		  MAJOR(__entry->dev), MINOR(__entry->dev),
+		  __entry->agno,
+		  __entry->nr_intents,
+		  __entry->caller_ip)
+);
+
+#define DEFINE_PERAG_INTENTS_EVENT(name)	\
+DEFINE_EVENT(xfs_perag_intents_class, name,					\
+	TP_PROTO(struct xfs_perag *pag, void *caller_ip), \
+	TP_ARGS(pag, caller_ip))
+DEFINE_PERAG_INTENTS_EVENT(xfs_perag_bump_intents);
+DEFINE_PERAG_INTENTS_EVENT(xfs_perag_drop_intents);
+DEFINE_PERAG_INTENTS_EVENT(xfs_perag_wait_intents);
+
+#endif /* CONFIG_XFS_DRAIN_INTENTS */
+
 #endif /* _TRACE_XFS_H */
 
 #undef TRACE_INCLUDE_PATH


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

* [PATCH 2/5] xfs: use per-cpu counters to implement intent draining
  2022-10-02 18:19 [PATCHSET v23.1 0/5] xfs: drain deferred work items when scrubbing Darrick J. Wong
                   ` (3 preceding siblings ...)
  2022-10-02 18:19 ` [PATCH 5/5] xfs: scrub should use ECHRNG to signal that the drain is needed Darrick J. Wong
@ 2022-10-02 18:19 ` Darrick J. Wong
  4 siblings, 0 replies; 6+ messages in thread
From: Darrick J. Wong @ 2022-10-02 18:19 UTC (permalink / raw)
  To: djwong; +Cc: linux-xfs

From: Darrick J. Wong <djwong@kernel.org>

Currently, the intent draining code uses a per-AG atomic counter to keep
track of how many writer threads are currently or going to start
processing log intent items for that AG.  This isn't particularly
efficient, since every counter update will dirty the cacheline, and the
only code that cares about precise counter values is online scrub, which
shouldn't be running all that often.

Therefore, substitute the atomic_t for a per-cpu counter with a high
batch limit to avoid pingponging cache lines as long as possible.  While
updates to per-cpu counters are slower in the single-thread case (on the
author's system, 12ns vs. 8ns), this quickly reverses itself if there
are a lot of CPUs queuing intent items.

Because percpu counter summation is slow, this change shifts most of the
performance impact to code that calls xfs_drain_wait, which means that
online fsck runs a little bit slower to minimize the overhead of regular
runtime code.

Signed-off-by: Darrick J. Wong <djwong@kernel.org>
---
 fs/xfs/libxfs/xfs_ag.c |    4 +++-
 fs/xfs/xfs_mount.c     |   10 ++++++----
 fs/xfs/xfs_mount.h     |   19 ++++++++++++++-----
 fs/xfs/xfs_trace.h     |    2 +-
 4 files changed, 24 insertions(+), 11 deletions(-)


diff --git a/fs/xfs/libxfs/xfs_ag.c b/fs/xfs/libxfs/xfs_ag.c
index 27ba30f8f64b..2da2a366395c 100644
--- a/fs/xfs/libxfs/xfs_ag.c
+++ b/fs/xfs/libxfs/xfs_ag.c
@@ -314,7 +314,9 @@ xfs_initialize_perag(
 		spin_lock_init(&pag->pag_state_lock);
 		INIT_DELAYED_WORK(&pag->pag_blockgc_work, xfs_blockgc_worker);
 		INIT_RADIX_TREE(&pag->pag_ici_root, GFP_ATOMIC);
-		xfs_drain_init(&pag->pag_intents);
+		error = xfs_drain_init(&pag->pag_intents);
+		if (error)
+			goto out_remove_pag;
 		init_waitqueue_head(&pag->pagb_wait);
 		pag->pagb_count = 0;
 		pag->pagb_tree = RB_ROOT;
diff --git a/fs/xfs/xfs_mount.c b/fs/xfs/xfs_mount.c
index 6c84c6547a0b..2effc1647ce1 100644
--- a/fs/xfs/xfs_mount.c
+++ b/fs/xfs/xfs_mount.c
@@ -1390,15 +1390,14 @@ xfs_mod_delalloc(
 /* Increase the pending intent count. */
 static inline void xfs_drain_bump(struct xfs_drain *dr)
 {
-	atomic_inc(&dr->dr_count);
+	percpu_counter_add_batch(&dr->dr_count, 1, XFS_DRAIN_BATCH);
 }
 
 /* Decrease the pending intent count, and wake any waiters, if appropriate. */
 static inline void xfs_drain_drop(struct xfs_drain *dr)
 {
-	if (atomic_dec_and_test(&dr->dr_count) &&
-	    wq_has_sleeper(&dr->dr_waiters))
-		wake_up(&dr->dr_waiters);
+	percpu_counter_add_batch(&dr->dr_count, -1, XFS_DRAIN_BATCH);
+	wake_up(&dr->dr_waiters);
 }
 
 /*
@@ -1409,6 +1408,9 @@ static inline void xfs_drain_drop(struct xfs_drain *dr)
  */
 static inline int xfs_drain_wait(struct xfs_drain *dr)
 {
+	if (!xfs_drain_busy(dr))
+		return 0;
+
 	return wait_event_killable(dr->dr_waiters, !xfs_drain_busy(dr));
 }
 
diff --git a/fs/xfs/xfs_mount.h b/fs/xfs/xfs_mount.h
index ddf438701022..3e4921304442 100644
--- a/fs/xfs/xfs_mount.h
+++ b/fs/xfs/xfs_mount.h
@@ -64,7 +64,7 @@ struct xfs_error_cfg {
  */
 struct xfs_drain {
 	/* Number of items pending in some part of the filesystem. */
-	atomic_t		dr_count;
+	struct percpu_counter	dr_count;
 
 	/* Queue to wait for dri_count to go to zero */
 	struct wait_queue_head	dr_waiters;
@@ -76,21 +76,30 @@ bool xfs_ag_intents_busy(struct xfs_perag *pag);
 void xfs_fs_bump_intents(struct xfs_mount *mp, xfs_fsblock_t fsb);
 void xfs_fs_drop_intents(struct xfs_mount *mp, xfs_fsblock_t fsb);
 
+/*
+ * Use a large batch value for the drain counter so that writer threads
+ * can queue a large number of log intents before having to update the main
+ * counter.
+ */
+#define XFS_DRAIN_BATCH		(1024)
+
 /* Are there work items pending? */
 static inline bool xfs_drain_busy(struct xfs_drain *dr)
 {
-	return atomic_read(&dr->dr_count) > 0;
+	return __percpu_counter_compare(&dr->dr_count, 0, XFS_DRAIN_BATCH) > 0;
 }
 
-static inline void xfs_drain_init(struct xfs_drain *dr)
+static inline int xfs_drain_init(struct xfs_drain *dr)
 {
-	atomic_set(&dr->dr_count, 0);
 	init_waitqueue_head(&dr->dr_waiters);
+	return percpu_counter_init(&dr->dr_count, 0, GFP_KERNEL);
 }
 
 static inline void xfs_drain_free(struct xfs_drain *dr)
 {
 	ASSERT(!xfs_drain_busy(dr));
+
+	percpu_counter_destroy(&dr->dr_count);
 }
 #else
 struct xfs_drain { /* empty */ };
@@ -104,7 +113,7 @@ static inline void
 xfs_fs_drop_intents(struct xfs_mount *mp, xfs_fsblock_t fsb)
 {
 }
-# define xfs_drain_init(dr)	((void)0)
+# define xfs_drain_init(dr)	(0)
 # define xfs_drain_free(dr)	((void)0)
 #endif /* CONFIG_XFS_DRAIN_INTENTS */
 
diff --git a/fs/xfs/xfs_trace.h b/fs/xfs/xfs_trace.h
index fae3b0fe0971..c584ff52bb40 100644
--- a/fs/xfs/xfs_trace.h
+++ b/fs/xfs/xfs_trace.h
@@ -4259,7 +4259,7 @@ DECLARE_EVENT_CLASS(xfs_perag_intents_class,
 	TP_fast_assign(
 		__entry->dev = pag->pag_mount->m_super->s_dev;
 		__entry->agno = pag->pag_agno;
-		__entry->nr_intents = atomic_read(&pag->pag_intents.dr_count);
+		__entry->nr_intents = percpu_counter_sum(&pag->pag_intents.dr_count);
 		__entry->caller_ip = caller_ip;
 	),
 	TP_printk("dev %d:%d agno 0x%x intents %ld caller %pS",


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

* [PATCH 3/5] xfs: clean up scrub context if scrub setup returns -EDEADLOCK
  2022-10-02 18:19 [PATCHSET v23.1 0/5] xfs: drain deferred work items when scrubbing Darrick J. Wong
  2022-10-02 18:19 ` [PATCH 1/5] xfs: allow queued AG intents to drain before scrubbing Darrick J. Wong
  2022-10-02 18:19 ` [PATCH 4/5] xfs: minimize overhead of drain wakeups by using jump labels Darrick J. Wong
@ 2022-10-02 18:19 ` Darrick J. Wong
  2022-10-02 18:19 ` [PATCH 5/5] xfs: scrub should use ECHRNG to signal that the drain is needed Darrick J. Wong
  2022-10-02 18:19 ` [PATCH 2/5] xfs: use per-cpu counters to implement intent draining Darrick J. Wong
  4 siblings, 0 replies; 6+ messages in thread
From: Darrick J. Wong @ 2022-10-02 18:19 UTC (permalink / raw)
  To: djwong; +Cc: linux-xfs

From: Darrick J. Wong <djwong@kernel.org>

It has been a longstanding convention that online scrub and repair
functions can return -EDEADLOCK to signal that they weren't able to
obtain some necessary resource.  When this happens, the scrub framework
is supposed to release all resources attached to the scrub context, set
the TRY_HARDER flag in the scrub context flags, and try again.  In this
context, individual scrub functions are supposed to take all the
resources they (incorrectly) speculated were not necessary.

We're about to make it so that the functions that lock and wait for a
filesystem AG can also return EDEADLOCK to signal that we need to try
again with the drain waiters enabled.  Therefore, refactor
xfs_scrub_metadata to support this behavior for ->setup() functions.

Signed-off-by: Darrick J. Wong <djwong@kernel.org>
---
 fs/xfs/scrub/scrub.c |   28 ++++++++++++++++------------
 1 file changed, 16 insertions(+), 12 deletions(-)


diff --git a/fs/xfs/scrub/scrub.c b/fs/xfs/scrub/scrub.c
index 07a7a75f987f..50db13c5f626 100644
--- a/fs/xfs/scrub/scrub.c
+++ b/fs/xfs/scrub/scrub.c
@@ -491,23 +491,16 @@ xfs_scrub_metadata(
 
 	/* Set up for the operation. */
 	error = sc->ops->setup(sc);
+	if (error == -EDEADLOCK && !(sc->flags & XCHK_TRY_HARDER))
+		goto try_harder;
 	if (error)
 		goto out_teardown;
 
 	/* Scrub for errors. */
 	error = sc->ops->scrub(sc);
-	if (!(sc->flags & XCHK_TRY_HARDER) && error == -EDEADLOCK) {
-		/*
-		 * Scrubbers return -EDEADLOCK to mean 'try harder'.
-		 * Tear down everything we hold, then set up again with
-		 * preparation for worst-case scenarios.
-		 */
-		error = xchk_teardown(sc, 0);
-		if (error)
-			goto out_sc;
-		sc->flags |= XCHK_TRY_HARDER;
-		goto retry_op;
-	} else if (error || (sm->sm_flags & XFS_SCRUB_OFLAG_INCOMPLETE))
+	if (error == -EDEADLOCK && !(sc->flags & XCHK_TRY_HARDER))
+		goto try_harder;
+	if (error || (sm->sm_flags & XFS_SCRUB_OFLAG_INCOMPLETE))
 		goto out_teardown;
 
 	xchk_update_health(sc);
@@ -565,4 +558,15 @@ xfs_scrub_metadata(
 		error = 0;
 	}
 	return error;
+try_harder:
+	/*
+	 * Scrubbers return -EDEADLOCK to mean 'try harder'.  Tear down
+	 * everything we hold, then set up again with preparation for
+	 * worst-case scenarios.
+	 */
+	error = xchk_teardown(sc, 0);
+	if (error)
+		goto out_sc;
+	sc->flags |= XCHK_TRY_HARDER;
+	goto retry_op;
 }


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

* [PATCH 4/5] xfs: minimize overhead of drain wakeups by using jump labels
  2022-10-02 18:19 [PATCHSET v23.1 0/5] xfs: drain deferred work items when scrubbing Darrick J. Wong
  2022-10-02 18:19 ` [PATCH 1/5] xfs: allow queued AG intents to drain before scrubbing Darrick J. Wong
@ 2022-10-02 18:19 ` Darrick J. Wong
  2022-10-02 18:19 ` [PATCH 3/5] xfs: clean up scrub context if scrub setup returns -EDEADLOCK Darrick J. Wong
                   ` (2 subsequent siblings)
  4 siblings, 0 replies; 6+ messages in thread
From: Darrick J. Wong @ 2022-10-02 18:19 UTC (permalink / raw)
  To: djwong; +Cc: linux-xfs

From: Darrick J. Wong <djwong@kernel.org>

To reduce the runtime overhead even further when online fsck isn't
running, use a static branch key to decide if we call wake_up on the
drain.  For compilers that support jump labels, the call to wake_up is
replaced by a nop sled when nobody is waiting for intents to drain.

From my initial microbenchmarking, every transition of the static key
between the on and off states takes about 22000ns to complete; this is
paid entirely by the xfs_scrub process.  When the static key is off
(which it should be when fsck isn't running), the nop sled adds an
overhead of approximately 0.36ns to runtime code.

For the few compilers that don't support jump labels, runtime code pays
the cost of calling wake_up on an empty waitqueue, which was observed to
be about 30ns.  However, most architectures that have sufficient memory
and CPU capacity to run XFS also support jump labels, so this is not
much of a worry.

Signed-off-by: Darrick J. Wong <djwong@kernel.org>
---
 fs/xfs/Kconfig            |    1 +
 fs/xfs/scrub/agheader.c   |    9 +++++++++
 fs/xfs/scrub/alloc.c      |    3 +++
 fs/xfs/scrub/bmap.c       |    3 +++
 fs/xfs/scrub/common.c     |   24 ++++++++++++++++++++++++
 fs/xfs/scrub/common.h     |   15 +++++++++++++++
 fs/xfs/scrub/fscounters.c |    7 +++++++
 fs/xfs/scrub/ialloc.c     |    2 ++
 fs/xfs/scrub/inode.c      |    3 +++
 fs/xfs/scrub/quota.c      |    3 +++
 fs/xfs/scrub/refcount.c   |    2 ++
 fs/xfs/scrub/rmap.c       |    3 +++
 fs/xfs/scrub/scrub.c      |   25 +++++++++++++++++++++----
 fs/xfs/scrub/scrub.h      |    5 ++++-
 fs/xfs/scrub/trace.h      |   33 +++++++++++++++++++++++++++++++++
 fs/xfs/xfs_mount.c        |   29 ++++++++++++++++++++++++++++-
 fs/xfs/xfs_mount.h        |    3 +++
 17 files changed, 164 insertions(+), 6 deletions(-)


diff --git a/fs/xfs/Kconfig b/fs/xfs/Kconfig
index a4eff514b647..8c5c550355a7 100644
--- a/fs/xfs/Kconfig
+++ b/fs/xfs/Kconfig
@@ -95,6 +95,7 @@ config XFS_RT
 
 config XFS_DRAIN_INTENTS
 	bool
+	select JUMP_LABEL if HAVE_ARCH_JUMP_LABEL
 
 config XFS_ONLINE_SCRUB
 	bool "XFS online metadata check support"
diff --git a/fs/xfs/scrub/agheader.c b/fs/xfs/scrub/agheader.c
index 4dd52b15f09c..3dd9151a20ad 100644
--- a/fs/xfs/scrub/agheader.c
+++ b/fs/xfs/scrub/agheader.c
@@ -18,6 +18,15 @@
 #include "scrub/scrub.h"
 #include "scrub/common.h"
 
+int
+xchk_setup_agheader(
+	struct xfs_scrub	*sc)
+{
+	if (xchk_need_fshook_drain(sc))
+		xchk_fshooks_enable(sc, XCHK_FSHOOKS_DRAIN);
+	return xchk_setup_fs(sc);
+}
+
 /* Superblock */
 
 /* Cross-reference with the other btrees. */
diff --git a/fs/xfs/scrub/alloc.c b/fs/xfs/scrub/alloc.c
index ab427b4d7fe0..4d9ccc15b048 100644
--- a/fs/xfs/scrub/alloc.c
+++ b/fs/xfs/scrub/alloc.c
@@ -24,6 +24,9 @@ int
 xchk_setup_ag_allocbt(
 	struct xfs_scrub	*sc)
 {
+	if (xchk_need_fshook_drain(sc))
+		xchk_fshooks_enable(sc, XCHK_FSHOOKS_DRAIN);
+
 	return xchk_setup_ag_btree(sc, false);
 }
 
diff --git a/fs/xfs/scrub/bmap.c b/fs/xfs/scrub/bmap.c
index f0b9cb6506fd..576c02cc8b8f 100644
--- a/fs/xfs/scrub/bmap.c
+++ b/fs/xfs/scrub/bmap.c
@@ -31,6 +31,9 @@ xchk_setup_inode_bmap(
 {
 	int			error;
 
+	if (xchk_need_fshook_drain(sc))
+		xchk_fshooks_enable(sc, XCHK_FSHOOKS_DRAIN);
+
 	error = xchk_get_inode(sc);
 	if (error)
 		goto out;
diff --git a/fs/xfs/scrub/common.c b/fs/xfs/scrub/common.c
index cc4882c0cfc2..74ad1a211216 100644
--- a/fs/xfs/scrub/common.c
+++ b/fs/xfs/scrub/common.c
@@ -479,6 +479,8 @@ xchk_perag_lock(
 			sa->agi_bp = NULL;
 		}
 
+		if (!(sc->flags & XCHK_FSHOOKS_DRAIN))
+			return -EDEADLOCK;
 		error = xfs_ag_drain_intents(sa->pag);
 		if (error == -ERESTARTSYS)
 			error = -EINTR;
@@ -964,3 +966,25 @@ xchk_start_reaping(
 	}
 	sc->flags &= ~XCHK_REAPING_DISABLED;
 }
+
+/*
+ * Enable filesystem hooks (i.e. runtime code patching) before starting a scrub
+ * operation.  Callers must not hold any locks that intersect with the CPU
+ * hotplug lock (e.g. writeback locks) because code patching must halt the CPUs
+ * to change kernel code.
+ */
+void
+xchk_fshooks_enable(
+	struct xfs_scrub	*sc,
+	unsigned int		scrub_fshooks)
+{
+	ASSERT(!(scrub_fshooks & ~XCHK_FSHOOKS_ALL));
+	ASSERT(!(sc->flags & scrub_fshooks));
+
+	trace_xchk_fshooks_enable(sc, scrub_fshooks);
+
+	if (scrub_fshooks & XCHK_FSHOOKS_DRAIN)
+		xfs_drain_wait_enable();
+
+	sc->flags |= scrub_fshooks;
+}
diff --git a/fs/xfs/scrub/common.h b/fs/xfs/scrub/common.h
index b73648d81d23..4de5677390a4 100644
--- a/fs/xfs/scrub/common.h
+++ b/fs/xfs/scrub/common.h
@@ -72,6 +72,7 @@ bool xchk_should_check_xref(struct xfs_scrub *sc, int *error,
 			   struct xfs_btree_cur **curpp);
 
 /* Setup functions */
+int xchk_setup_agheader(struct xfs_scrub *sc);
 int xchk_setup_fs(struct xfs_scrub *sc);
 int xchk_setup_ag_allocbt(struct xfs_scrub *sc);
 int xchk_setup_ag_iallocbt(struct xfs_scrub *sc);
@@ -151,4 +152,18 @@ int xchk_ilock_inverted(struct xfs_inode *ip, uint lock_mode);
 void xchk_stop_reaping(struct xfs_scrub *sc);
 void xchk_start_reaping(struct xfs_scrub *sc);
 
+/*
+ * Setting up a hook to wait for intents to drain is costly -- we have to take
+ * the CPU hotplug lock and force an i-cache flush on all CPUs once to set it
+ * up, and again to tear it down.  These costs add up quickly, so we only want
+ * to enable the drain waiter if the drain actually detected a conflict with
+ * running intent chains.
+ */
+static inline bool xchk_need_fshook_drain(struct xfs_scrub *sc)
+{
+	return sc->flags & XCHK_TRY_HARDER;
+}
+
+void xchk_fshooks_enable(struct xfs_scrub *sc, unsigned int scrub_fshooks);
+
 #endif	/* __XFS_SCRUB_COMMON_H__ */
diff --git a/fs/xfs/scrub/fscounters.c b/fs/xfs/scrub/fscounters.c
index 3c56f5890da4..3c66523ec212 100644
--- a/fs/xfs/scrub/fscounters.c
+++ b/fs/xfs/scrub/fscounters.c
@@ -116,6 +116,13 @@ xchk_setup_fscounters(
 	struct xchk_fscounters	*fsc;
 	int			error;
 
+	/*
+	 * If the AGF doesn't track btreeblks, we have to lock the AGF to count
+	 * btree block usage by walking the actual btrees.
+	 */
+	if (!xfs_has_lazysbcount(sc->mp))
+		xchk_fshooks_enable(sc, XCHK_FSHOOKS_DRAIN);
+
 	sc->buf = kzalloc(sizeof(struct xchk_fscounters), XCHK_GFP_FLAGS);
 	if (!sc->buf)
 		return -ENOMEM;
diff --git a/fs/xfs/scrub/ialloc.c b/fs/xfs/scrub/ialloc.c
index e1026e07bf94..0b27c3520b74 100644
--- a/fs/xfs/scrub/ialloc.c
+++ b/fs/xfs/scrub/ialloc.c
@@ -32,6 +32,8 @@ int
 xchk_setup_ag_iallocbt(
 	struct xfs_scrub	*sc)
 {
+	if (xchk_need_fshook_drain(sc))
+		xchk_fshooks_enable(sc, XCHK_FSHOOKS_DRAIN);
 	return xchk_setup_ag_btree(sc, sc->flags & XCHK_TRY_HARDER);
 }
 
diff --git a/fs/xfs/scrub/inode.c b/fs/xfs/scrub/inode.c
index 51820b40ab1c..998bf06d2347 100644
--- a/fs/xfs/scrub/inode.c
+++ b/fs/xfs/scrub/inode.c
@@ -32,6 +32,9 @@ xchk_setup_inode(
 {
 	int			error;
 
+	if (xchk_need_fshook_drain(sc))
+		xchk_fshooks_enable(sc, XCHK_FSHOOKS_DRAIN);
+
 	/*
 	 * Try to get the inode.  If the verifiers fail, we try again
 	 * in raw mode.
diff --git a/fs/xfs/scrub/quota.c b/fs/xfs/scrub/quota.c
index 0b643ff32b22..d15682e2f2a3 100644
--- a/fs/xfs/scrub/quota.c
+++ b/fs/xfs/scrub/quota.c
@@ -52,6 +52,9 @@ xchk_setup_quota(
 	if (!xfs_this_quota_on(sc->mp, dqtype))
 		return -ENOENT;
 
+	if (xchk_need_fshook_drain(sc))
+		xchk_fshooks_enable(sc, XCHK_FSHOOKS_DRAIN);
+
 	error = xchk_setup_fs(sc);
 	if (error)
 		return error;
diff --git a/fs/xfs/scrub/refcount.c b/fs/xfs/scrub/refcount.c
index e4a550f8d1b7..4e70cd821b62 100644
--- a/fs/xfs/scrub/refcount.c
+++ b/fs/xfs/scrub/refcount.c
@@ -26,6 +26,8 @@ int
 xchk_setup_ag_refcountbt(
 	struct xfs_scrub	*sc)
 {
+	if (xchk_need_fshook_drain(sc))
+		xchk_fshooks_enable(sc, XCHK_FSHOOKS_DRAIN);
 	return xchk_setup_ag_btree(sc, false);
 }
 
diff --git a/fs/xfs/scrub/rmap.c b/fs/xfs/scrub/rmap.c
index 229826b2e1c0..afc4f840b6bc 100644
--- a/fs/xfs/scrub/rmap.c
+++ b/fs/xfs/scrub/rmap.c
@@ -24,6 +24,9 @@ int
 xchk_setup_ag_rmapbt(
 	struct xfs_scrub	*sc)
 {
+	if (xchk_need_fshook_drain(sc))
+		xchk_fshooks_enable(sc, XCHK_FSHOOKS_DRAIN);
+
 	return xchk_setup_ag_btree(sc, false);
 }
 
diff --git a/fs/xfs/scrub/scrub.c b/fs/xfs/scrub/scrub.c
index 50db13c5f626..8f8a4eb758ea 100644
--- a/fs/xfs/scrub/scrub.c
+++ b/fs/xfs/scrub/scrub.c
@@ -145,6 +145,21 @@ xchk_probe(
 
 /* Scrub setup and teardown */
 
+static inline void
+xchk_fshooks_disable(
+	struct xfs_scrub	*sc)
+{
+	if (!(sc->flags & XCHK_FSHOOKS_ALL))
+		return;
+
+	trace_xchk_fshooks_disable(sc, sc->flags & XCHK_FSHOOKS_ALL);
+
+	if (sc->flags & XCHK_FSHOOKS_DRAIN)
+		xfs_drain_wait_disable();
+
+	sc->flags &= ~XCHK_FSHOOKS_ALL;
+}
+
 /* Free all the resources and finish the transactions. */
 STATIC int
 xchk_teardown(
@@ -177,6 +192,8 @@ xchk_teardown(
 		kvfree(sc->buf);
 		sc->buf = NULL;
 	}
+
+	xchk_fshooks_disable(sc);
 	return error;
 }
 
@@ -191,25 +208,25 @@ static const struct xchk_meta_ops meta_scrub_ops[] = {
 	},
 	[XFS_SCRUB_TYPE_SB] = {		/* superblock */
 		.type	= ST_PERAG,
-		.setup	= xchk_setup_fs,
+		.setup	= xchk_setup_agheader,
 		.scrub	= xchk_superblock,
 		.repair	= xrep_superblock,
 	},
 	[XFS_SCRUB_TYPE_AGF] = {	/* agf */
 		.type	= ST_PERAG,
-		.setup	= xchk_setup_fs,
+		.setup	= xchk_setup_agheader,
 		.scrub	= xchk_agf,
 		.repair	= xrep_agf,
 	},
 	[XFS_SCRUB_TYPE_AGFL]= {	/* agfl */
 		.type	= ST_PERAG,
-		.setup	= xchk_setup_fs,
+		.setup	= xchk_setup_agheader,
 		.scrub	= xchk_agfl,
 		.repair	= xrep_agfl,
 	},
 	[XFS_SCRUB_TYPE_AGI] = {	/* agi */
 		.type	= ST_PERAG,
-		.setup	= xchk_setup_fs,
+		.setup	= xchk_setup_agheader,
 		.scrub	= xchk_agi,
 		.repair	= xrep_agi,
 	},
diff --git a/fs/xfs/scrub/scrub.h b/fs/xfs/scrub/scrub.h
index a0f097b8acb0..daf87c4331fd 100644
--- a/fs/xfs/scrub/scrub.h
+++ b/fs/xfs/scrub/scrub.h
@@ -96,9 +96,12 @@ struct xfs_scrub {
 
 /* XCHK state flags grow up from zero, XREP state flags grown down from 2^31 */
 #define XCHK_TRY_HARDER		(1 << 0)  /* can't get resources, try again */
-#define XCHK_REAPING_DISABLED	(1 << 2)  /* background block reaping paused */
+#define XCHK_REAPING_DISABLED	(1 << 1)  /* background block reaping paused */
+#define XCHK_FSHOOKS_DRAIN	(1 << 2)  /* defer ops draining enabled */
 #define XREP_ALREADY_FIXED	(1 << 31) /* checking our repair work */
 
+#define XCHK_FSHOOKS_ALL	(XCHK_FSHOOKS_DRAIN)
+
 /* Metadata scrubbers */
 int xchk_tester(struct xfs_scrub *sc);
 int xchk_superblock(struct xfs_scrub *sc);
diff --git a/fs/xfs/scrub/trace.h b/fs/xfs/scrub/trace.h
index 93ece6df02e3..89414eb743f1 100644
--- a/fs/xfs/scrub/trace.h
+++ b/fs/xfs/scrub/trace.h
@@ -93,6 +93,12 @@ TRACE_DEFINE_ENUM(XFS_SCRUB_TYPE_FSCOUNTERS);
 	{ XFS_SCRUB_OFLAG_WARNING,		"warning" }, \
 	{ XFS_SCRUB_OFLAG_NO_REPAIR_NEEDED,	"norepair" }
 
+#define XFS_SCRUB_STATE_STRINGS \
+	{ XCHK_TRY_HARDER,			"try_harder" }, \
+	{ XCHK_REAPING_DISABLED,		"reaping_disabled" }, \
+	{ XCHK_FSHOOKS_DRAIN,			"fshooks_drain" }, \
+	{ XREP_ALREADY_FIXED,			"already_fixed" }
+
 DECLARE_EVENT_CLASS(xchk_class,
 	TP_PROTO(struct xfs_inode *ip, struct xfs_scrub_metadata *sm,
 		 int error),
@@ -139,6 +145,33 @@ DEFINE_SCRUB_EVENT(xchk_deadlock_retry);
 DEFINE_SCRUB_EVENT(xrep_attempt);
 DEFINE_SCRUB_EVENT(xrep_done);
 
+DECLARE_EVENT_CLASS(xchk_fshook_class,
+	TP_PROTO(struct xfs_scrub *sc, unsigned int fshooks),
+	TP_ARGS(sc, fshooks),
+	TP_STRUCT__entry(
+		__field(dev_t, dev)
+		__field(unsigned int, type)
+		__field(unsigned int, fshooks)
+	),
+	TP_fast_assign(
+		__entry->dev = sc->mp->m_super->s_dev;
+		__entry->type = sc->sm->sm_type;
+		__entry->fshooks = fshooks;
+	),
+	TP_printk("dev %d:%d type %s fshooks '%s'",
+		  MAJOR(__entry->dev), MINOR(__entry->dev),
+		  __print_symbolic(__entry->type, XFS_SCRUB_TYPE_STRINGS),
+		  __print_flags(__entry->fshooks, "|", XFS_SCRUB_STATE_STRINGS))
+)
+
+#define DEFINE_SCRUB_FSHOOK_EVENT(name) \
+DEFINE_EVENT(xchk_fshook_class, name, \
+	TP_PROTO(struct xfs_scrub *sc, unsigned int fshooks), \
+	TP_ARGS(sc, fshooks))
+
+DEFINE_SCRUB_FSHOOK_EVENT(xchk_fshooks_enable);
+DEFINE_SCRUB_FSHOOK_EVENT(xchk_fshooks_disable);
+
 TRACE_EVENT(xchk_op_error,
 	TP_PROTO(struct xfs_scrub *sc, xfs_agnumber_t agno,
 		 xfs_agblock_t bno, int error, void *ret_ip),
diff --git a/fs/xfs/xfs_mount.c b/fs/xfs/xfs_mount.c
index 2effc1647ce1..e427202c3ab2 100644
--- a/fs/xfs/xfs_mount.c
+++ b/fs/xfs/xfs_mount.c
@@ -1387,6 +1387,31 @@ xfs_mod_delalloc(
 }
 
 #ifdef CONFIG_XFS_DRAIN_INTENTS
+/*
+ * Use a static key here to reduce the overhead of xfs_drain_drop.  If the
+ * compiler supports jump labels, the static branch will be replaced by a nop
+ * sled when there are no xfs_drain_wait callers.  Online fsck is currently
+ * the only caller, so this is a reasonable tradeoff.
+ *
+ * Note: Patching the kernel code requires taking the cpu hotplug lock.  Other
+ * parts of the kernel allocate memory with that lock held, which means that
+ * XFS callers cannot hold any locks that might be used by memory reclaim or
+ * writeback when calling the static_branch_{inc,dec} functions.
+ */
+static DEFINE_STATIC_KEY_FALSE(xfs_drain_waiter_hook);
+
+void
+xfs_drain_wait_disable(void)
+{
+	static_branch_dec(&xfs_drain_waiter_hook);
+}
+
+void
+xfs_drain_wait_enable(void)
+{
+	static_branch_inc(&xfs_drain_waiter_hook);
+}
+
 /* Increase the pending intent count. */
 static inline void xfs_drain_bump(struct xfs_drain *dr)
 {
@@ -1397,7 +1422,9 @@ static inline void xfs_drain_bump(struct xfs_drain *dr)
 static inline void xfs_drain_drop(struct xfs_drain *dr)
 {
 	percpu_counter_add_batch(&dr->dr_count, -1, XFS_DRAIN_BATCH);
-	wake_up(&dr->dr_waiters);
+
+	if (static_branch_unlikely(&xfs_drain_waiter_hook))
+		wake_up(&dr->dr_waiters);
 }
 
 /*
diff --git a/fs/xfs/xfs_mount.h b/fs/xfs/xfs_mount.h
index 3e4921304442..ba7888c4f650 100644
--- a/fs/xfs/xfs_mount.h
+++ b/fs/xfs/xfs_mount.h
@@ -73,6 +73,9 @@ struct xfs_drain {
 int xfs_ag_drain_intents(struct xfs_perag *pag);
 bool xfs_ag_intents_busy(struct xfs_perag *pag);
 
+void xfs_drain_wait_disable(void);
+void xfs_drain_wait_enable(void);
+
 void xfs_fs_bump_intents(struct xfs_mount *mp, xfs_fsblock_t fsb);
 void xfs_fs_drop_intents(struct xfs_mount *mp, xfs_fsblock_t fsb);
 


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

* [PATCH 5/5] xfs: scrub should use ECHRNG to signal that the drain is needed
  2022-10-02 18:19 [PATCHSET v23.1 0/5] xfs: drain deferred work items when scrubbing Darrick J. Wong
                   ` (2 preceding siblings ...)
  2022-10-02 18:19 ` [PATCH 3/5] xfs: clean up scrub context if scrub setup returns -EDEADLOCK Darrick J. Wong
@ 2022-10-02 18:19 ` Darrick J. Wong
  2022-10-02 18:19 ` [PATCH 2/5] xfs: use per-cpu counters to implement intent draining Darrick J. Wong
  4 siblings, 0 replies; 6+ messages in thread
From: Darrick J. Wong @ 2022-10-02 18:19 UTC (permalink / raw)
  To: djwong; +Cc: linux-xfs

From: Darrick J. Wong <djwong@kernel.org>

In the previous patch, we added jump labels to the intent drain code so
that regular filesystem operations need not pay the price of checking
for someone (scrub) waiting on intents to drain from some part of the
filesystem when that someone isn't running.

However, I observed that xfs/285 now spends a lot more time pushing the
AIL from the inode btree scrubber than it used to.  This is because the
inobt scrubber will try push the AIL to try to get logged inode cores
written to the filesystem when it sees a weird discrepancy between the
ondisk inode and the inobt records.  This AIL push is triggered when the
setup function sees TRY_HARDER is set; and the requisite EDEADLOCK
return is initiated when the discrepancy is seen.

The solution to this performance slow down is to use a different result
code (ECHRNG) for scrub code to signal that it needs to wait for
deferred intent work items to drain out of some part of the filesystem.
When this happens, set a new scrub state flag (XCHK_NEED_DRAIN) so that
setup functions will activate the jump label.

Signed-off-by: Darrick J. Wong <djwong@kernel.org>
---
 fs/xfs/scrub/btree.c   |    1 +
 fs/xfs/scrub/common.c  |    4 +++-
 fs/xfs/scrub/common.h  |    2 +-
 fs/xfs/scrub/dabtree.c |    1 +
 fs/xfs/scrub/repair.c  |    3 +++
 fs/xfs/scrub/scrub.c   |   10 ++++++++++
 fs/xfs/scrub/scrub.h   |    1 +
 fs/xfs/scrub/trace.h   |    1 +
 8 files changed, 21 insertions(+), 2 deletions(-)


diff --git a/fs/xfs/scrub/btree.c b/fs/xfs/scrub/btree.c
index 566a093b2cf3..47fa14311d17 100644
--- a/fs/xfs/scrub/btree.c
+++ b/fs/xfs/scrub/btree.c
@@ -36,6 +36,7 @@ __xchk_btree_process_error(
 
 	switch (*error) {
 	case -EDEADLOCK:
+	case -ECHRNG:
 		/* Used to restart an op with deadlock avoidance. */
 		trace_xchk_deadlock_retry(sc->ip, sc->sm, *error);
 		break;
diff --git a/fs/xfs/scrub/common.c b/fs/xfs/scrub/common.c
index 74ad1a211216..60c4d33fe6f5 100644
--- a/fs/xfs/scrub/common.c
+++ b/fs/xfs/scrub/common.c
@@ -75,6 +75,7 @@ __xchk_process_error(
 	case 0:
 		return true;
 	case -EDEADLOCK:
+	case -ECHRNG:
 		/* Used to restart an op with deadlock avoidance. */
 		trace_xchk_deadlock_retry(
 				sc->ip ? sc->ip : XFS_I(file_inode(sc->file)),
@@ -130,6 +131,7 @@ __xchk_fblock_process_error(
 	case 0:
 		return true;
 	case -EDEADLOCK:
+	case -ECHRNG:
 		/* Used to restart an op with deadlock avoidance. */
 		trace_xchk_deadlock_retry(sc->ip, sc->sm, *error);
 		break;
@@ -480,7 +482,7 @@ xchk_perag_lock(
 		}
 
 		if (!(sc->flags & XCHK_FSHOOKS_DRAIN))
-			return -EDEADLOCK;
+			return -ECHRNG;
 		error = xfs_ag_drain_intents(sa->pag);
 		if (error == -ERESTARTSYS)
 			error = -EINTR;
diff --git a/fs/xfs/scrub/common.h b/fs/xfs/scrub/common.h
index 4de5677390a4..0efe6b947d88 100644
--- a/fs/xfs/scrub/common.h
+++ b/fs/xfs/scrub/common.h
@@ -161,7 +161,7 @@ void xchk_start_reaping(struct xfs_scrub *sc);
  */
 static inline bool xchk_need_fshook_drain(struct xfs_scrub *sc)
 {
-	return sc->flags & XCHK_TRY_HARDER;
+	return sc->flags & XCHK_NEED_DRAIN;
 }
 
 void xchk_fshooks_enable(struct xfs_scrub *sc, unsigned int scrub_fshooks);
diff --git a/fs/xfs/scrub/dabtree.c b/fs/xfs/scrub/dabtree.c
index d17cee177085..957a0b1a2f0b 100644
--- a/fs/xfs/scrub/dabtree.c
+++ b/fs/xfs/scrub/dabtree.c
@@ -39,6 +39,7 @@ xchk_da_process_error(
 
 	switch (*error) {
 	case -EDEADLOCK:
+	case -ECHRNG:
 		/* Used to restart an op with deadlock avoidance. */
 		trace_xchk_deadlock_retry(sc->ip, sc->sm, *error);
 		break;
diff --git a/fs/xfs/scrub/repair.c b/fs/xfs/scrub/repair.c
index 34fc0dc5f200..be65357285e6 100644
--- a/fs/xfs/scrub/repair.c
+++ b/fs/xfs/scrub/repair.c
@@ -60,6 +60,9 @@ xrep_attempt(
 		sc->sm->sm_flags &= ~XFS_SCRUB_FLAGS_OUT;
 		sc->flags |= XREP_ALREADY_FIXED;
 		return -EAGAIN;
+	case -ECHRNG:
+		sc->flags |= XCHK_NEED_DRAIN;
+		return -EAGAIN;
 	case -EDEADLOCK:
 		/* Tell the caller to try again having grabbed all the locks. */
 		if (!(sc->flags & XCHK_TRY_HARDER)) {
diff --git a/fs/xfs/scrub/scrub.c b/fs/xfs/scrub/scrub.c
index 8f8a4eb758ea..7a3557a69fe0 100644
--- a/fs/xfs/scrub/scrub.c
+++ b/fs/xfs/scrub/scrub.c
@@ -510,6 +510,8 @@ xfs_scrub_metadata(
 	error = sc->ops->setup(sc);
 	if (error == -EDEADLOCK && !(sc->flags & XCHK_TRY_HARDER))
 		goto try_harder;
+	if (error == -ECHRNG && !(sc->flags & XCHK_NEED_DRAIN))
+		goto need_drain;
 	if (error)
 		goto out_teardown;
 
@@ -517,6 +519,8 @@ xfs_scrub_metadata(
 	error = sc->ops->scrub(sc);
 	if (error == -EDEADLOCK && !(sc->flags & XCHK_TRY_HARDER))
 		goto try_harder;
+	if (error == -ECHRNG && !(sc->flags & XCHK_NEED_DRAIN))
+		goto need_drain;
 	if (error || (sm->sm_flags & XFS_SCRUB_OFLAG_INCOMPLETE))
 		goto out_teardown;
 
@@ -575,6 +579,12 @@ xfs_scrub_metadata(
 		error = 0;
 	}
 	return error;
+need_drain:
+	error = xchk_teardown(sc, 0);
+	if (error)
+		goto out_sc;
+	sc->flags |= XCHK_NEED_DRAIN;
+	goto retry_op;
 try_harder:
 	/*
 	 * Scrubbers return -EDEADLOCK to mean 'try harder'.  Tear down
diff --git a/fs/xfs/scrub/scrub.h b/fs/xfs/scrub/scrub.h
index daf87c4331fd..92b383061e88 100644
--- a/fs/xfs/scrub/scrub.h
+++ b/fs/xfs/scrub/scrub.h
@@ -98,6 +98,7 @@ struct xfs_scrub {
 #define XCHK_TRY_HARDER		(1 << 0)  /* can't get resources, try again */
 #define XCHK_REAPING_DISABLED	(1 << 1)  /* background block reaping paused */
 #define XCHK_FSHOOKS_DRAIN	(1 << 2)  /* defer ops draining enabled */
+#define XCHK_NEED_DRAIN		(1 << 3)  /* scrub needs to use intent drain */
 #define XREP_ALREADY_FIXED	(1 << 31) /* checking our repair work */
 
 #define XCHK_FSHOOKS_ALL	(XCHK_FSHOOKS_DRAIN)
diff --git a/fs/xfs/scrub/trace.h b/fs/xfs/scrub/trace.h
index 89414eb743f1..8738e6881ffc 100644
--- a/fs/xfs/scrub/trace.h
+++ b/fs/xfs/scrub/trace.h
@@ -97,6 +97,7 @@ TRACE_DEFINE_ENUM(XFS_SCRUB_TYPE_FSCOUNTERS);
 	{ XCHK_TRY_HARDER,			"try_harder" }, \
 	{ XCHK_REAPING_DISABLED,		"reaping_disabled" }, \
 	{ XCHK_FSHOOKS_DRAIN,			"fshooks_drain" }, \
+	{ XCHK_NEED_DRAIN,			"need_drain" }, \
 	{ XREP_ALREADY_FIXED,			"already_fixed" }
 
 DECLARE_EVENT_CLASS(xchk_class,


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

end of thread, other threads:[~2022-10-02 18:30 UTC | newest]

Thread overview: 6+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2022-10-02 18:19 [PATCHSET v23.1 0/5] xfs: drain deferred work items when scrubbing Darrick J. Wong
2022-10-02 18:19 ` [PATCH 1/5] xfs: allow queued AG intents to drain before scrubbing Darrick J. Wong
2022-10-02 18:19 ` [PATCH 4/5] xfs: minimize overhead of drain wakeups by using jump labels Darrick J. Wong
2022-10-02 18:19 ` [PATCH 3/5] xfs: clean up scrub context if scrub setup returns -EDEADLOCK Darrick J. Wong
2022-10-02 18:19 ` [PATCH 5/5] xfs: scrub should use ECHRNG to signal that the drain is needed Darrick J. Wong
2022-10-02 18:19 ` [PATCH 2/5] xfs: use per-cpu counters to implement intent draining Darrick J. Wong

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