linux-xfs.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH 0/3 v7] xfs: make CIL pipelining work
@ 2021-08-10  5:22 Dave Chinner
  2021-08-10  5:22 ` [PATCH 1/3] xfs: AIL needs asynchronous CIL forcing Dave Chinner
                   ` (2 more replies)
  0 siblings, 3 replies; 4+ messages in thread
From: Dave Chinner @ 2021-08-10  5:22 UTC (permalink / raw)
  To: linux-xfs

This patchset improves the behaviour of the CIL by increasing
the processing capacity available for pushing changes into the
journal.

There are two aspects to this. The first is to reduce latency for
callers that require non-blocking log force behaviour such as the
AIL.

The AIL only needs to push on the CIL to get items unpinned, and it
doesn't need to wait for it to complete, either, before it continues
onwards trying to push out items to disk. The AIL will back off when
it reaches it's push target, so it doesn't need to wait on log
forces to back off when there are pinned items in the AIL.

Hence we add a mechanism to async pushes on the CIL
that do not block and convert the AIL to use it. This results in
the AIL backing off on it's own short timeouts and trying to make
progress repeatedly instead of stalling for seconds waiting for log
large CIL forces to complete.

This ability to run async CIL pushes then highlights a problem with
pipelining of the CIL pushes. The pipelining isn't working as
intended, it's actually serialising and only allowing a single CIL
push work to be in progress at once.

This can result in the CIL push work being CPU bound and limiting
the rate at which items can be pushed to the journal. It is also
creating excessive push latency where the CIL fills and hits the
hard throttle while waiting for the push work to finish the current
push and then start on the new push and swap in a new CIL context
that can be committed to.

Essentially, the problem is an implementation problem, not a design
flaw. The implementation has a single work attached to the CIL,
meaning we can only have a single outstanding push work in progress
at any time. The workqueue can handle more, but we only have a
single work. So the fix is to move the work to the CIL context so we
can queue and process multiple works at the same time, thereby
actually allowing the CIL push work to pipeline in the intended
manner.

With this change, it's also very clear that the CIL workqueue really
belongs to the CIL, not the xfs_mount. Having the CIL push have to
reference through the log and the xfs_mount to reach it's private
workqueue is quite the layering violation, so fix this up, too.

This has been run through thousands of cycles of generic/019 and
generic/0475 since the start record ordering issues were fixed by
"xfs: strictly order log start records" without any log recovery
failures or corruptions being recorded.

Version 7:
- rebase on 5.14-rc4 + for-next + "xfs: strictly order log start records"

Version 6:
- https://lore.kernel.org/linux-xfs/20210714050600.2632218-1-david@fromorbit.com/
- split out from aggregated patchset
- add dependency on "xfs: strictly order log start records" for
  correct log recovery and runtime AIL ordering behaviour.
- rebase on 5.14-rc1 + "xfs: strictly order log start records"
- add patch moving CIL push workqueue into the CIL itself rather
  than having to go back up to the xfs_mount to access it at
  runtime.

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


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

* [PATCH 1/3] xfs: AIL needs asynchronous CIL forcing
  2021-08-10  5:22 [PATCH 0/3 v7] xfs: make CIL pipelining work Dave Chinner
@ 2021-08-10  5:22 ` Dave Chinner
  2021-08-10  5:22 ` [PATCH 2/3] xfs: CIL work is serialised, not pipelined Dave Chinner
  2021-08-10  5:22 ` [PATCH 3/3] xfs: move the CIL workqueue to the CIL Dave Chinner
  2 siblings, 0 replies; 4+ messages in thread
From: Dave Chinner @ 2021-08-10  5:22 UTC (permalink / raw)
  To: linux-xfs

From: Dave Chinner <dchinner@redhat.com>

The AIL pushing is stalling on log forces when it comes across
pinned items. This is happening on removal workloads where the AIL
is dominated by stale items that are removed from AIL when the
checkpoint that marks the items stale is committed to the journal.
This results is relatively few items in the AIL, but those that are
are often pinned as directories items are being removed from are
still being logged.

As a result, many push cycles through the CIL will first issue a
blocking log force to unpin the items. This can take some time to
complete, with tracing regularly showing push delays of half a
second and sometimes up into the range of several seconds. Sequences
like this aren't uncommon:

....
 399.829437:  xfsaild: last lsn 0x11002dd000 count 101 stuck 101 flushing 0 tout 20
<wanted 20ms, got 270ms delay>
 400.099622:  xfsaild: target 0x11002f3600, prev 0x11002f3600, last lsn 0x0
 400.099623:  xfsaild: first lsn 0x11002f3600
 400.099679:  xfsaild: last lsn 0x1100305000 count 16 stuck 11 flushing 0 tout 50
<wanted 50ms, got 500ms delay>
 400.589348:  xfsaild: target 0x110032e600, prev 0x11002f3600, last lsn 0x0
 400.589349:  xfsaild: first lsn 0x1100305000
 400.589595:  xfsaild: last lsn 0x110032e600 count 156 stuck 101 flushing 30 tout 50
<wanted 50ms, got 460ms delay>
 400.950341:  xfsaild: target 0x1100353000, prev 0x110032e600, last lsn 0x0
 400.950343:  xfsaild: first lsn 0x1100317c00
 400.950436:  xfsaild: last lsn 0x110033d200 count 105 stuck 101 flushing 0 tout 20
<wanted 20ms, got 200ms delay>
 401.142333:  xfsaild: target 0x1100361600, prev 0x1100353000, last lsn 0x0
 401.142334:  xfsaild: first lsn 0x110032e600
 401.142535:  xfsaild: last lsn 0x1100353000 count 122 stuck 101 flushing 8 tout 10
<wanted 10ms, got 10ms delay>
 401.154323:  xfsaild: target 0x1100361600, prev 0x1100361600, last lsn 0x1100353000
 401.154328:  xfsaild: first lsn 0x1100353000
 401.154389:  xfsaild: last lsn 0x1100353000 count 101 stuck 101 flushing 0 tout 20
<wanted 20ms, got 300ms delay>
 401.451525:  xfsaild: target 0x1100361600, prev 0x1100361600, last lsn 0x0
 401.451526:  xfsaild: first lsn 0x1100353000
 401.451804:  xfsaild: last lsn 0x1100377200 count 170 stuck 22 flushing 122 tout 50
<wanted 50ms, got 500ms delay>
 401.933581:  xfsaild: target 0x1100361600, prev 0x1100361600, last lsn 0x0
....

In each of these cases, every AIL pass saw 101 log items stuck on
the AIL (pinned) with very few other items being found. Each pass, a
log force was issued, and delay between last/first is the sleep time
+ the sync log force time.

Some of these 101 items pinned the tail of the log. The tail of the
log does slowly creep forward (first lsn), but the problem is that
the log is actually out of reservation space because it's been
running so many transactions that stale items that never reach the
AIL but consume log space. Hence we have a largely empty AIL, with
long term pins on items that pin the tail of the log that don't get
pushed frequently enough to keep log space available.

The problem is the hundreds of milliseconds that we block in the log
force pushing the CIL out to disk. The AIL should not be stalled
like this - it needs to run and flush items that are at the tail of
the log with minimal latency. What we really need to do is trigger a
log flush, but then not wait for it at all - we've already done our
waiting for stuff to complete when we backed off prior to the log
force being issued.

Even if we remove the XFS_LOG_SYNC from the xfs_log_force() call, we
still do a blocking flush of the CIL and that is what is causing the
issue. Hence we need a new interface for the CIL to trigger an
immediate background push of the CIL to get it moving faster but not
to wait on that to occur. While the CIL is pushing, the AIL can also
be pushing.

We already have an internal interface to do this -
xlog_cil_push_now() - but we need a wrapper for it to be used
externally. xlog_cil_force_seq() can easily be extended to do what
we need as it already implements the synchronous CIL push via
xlog_cil_push_now(). Add the necessary flags and "push current
sequence" semantics to xlog_cil_force_seq() and convert the AIL
pushing to use it.

One of the complexities here is that the CIL push does not guarantee
that the commit record for the CIL checkpoint is written to disk.
The current log force ensures this by submitting the current ACTIVE
iclog that the commit record was written to. We need the CIL to
actually write this commit record to disk for an async push to
ensure that the checkpoint actually makes it to disk and unpins the
pinned items in the checkpoint on completion. Hence we need to pass
down to the CIL push that we are doing an async flush so that it can
switch out the commit_iclog if necessary to get written to disk when
the commit iclog is finally released.

Signed-off-by: Dave Chinner <dchinner@redhat.com>
Reviewed-by: Darrick J. Wong <djwong@kernel.org>
Reviewed-by: Allison Henderson <allison.henderson@oracle.com>
---
 fs/xfs/xfs_log.c       | 38 ++++++++++++++------------
 fs/xfs/xfs_log.h       |  1 +
 fs/xfs/xfs_log_cil.c   | 62 ++++++++++++++++++++++++++++++++++++------
 fs/xfs/xfs_log_priv.h  |  4 +++
 fs/xfs/xfs_sysfs.c     |  1 +
 fs/xfs/xfs_trace.c     |  1 +
 fs/xfs/xfs_trans.c     |  2 +-
 fs/xfs/xfs_trans_ail.c | 11 ++++++--
 8 files changed, 91 insertions(+), 29 deletions(-)

diff --git a/fs/xfs/xfs_log.c b/fs/xfs/xfs_log.c
index 098f5c8ceb29..c738e8c79efa 100644
--- a/fs/xfs/xfs_log.c
+++ b/fs/xfs/xfs_log.c
@@ -52,11 +52,6 @@ xlog_state_get_iclog_space(
 	int			*continued_write,
 	int			*logoffsetp);
 STATIC void
-xlog_state_switch_iclogs(
-	struct xlog		*log,
-	struct xlog_in_core	*iclog,
-	int			eventual_size);
-STATIC void
 xlog_grant_push_ail(
 	struct xlog		*log,
 	int			need_bytes);
@@ -3174,7 +3169,7 @@ xfs_log_ticket_ungrant(
  * This routine will mark the current iclog in the ring as WANT_SYNC and move
  * the current iclog pointer to the next iclog in the ring.
  */
-STATIC void
+void
 xlog_state_switch_iclogs(
 	struct xlog		*log,
 	struct xlog_in_core	*iclog,
@@ -3346,6 +3341,20 @@ xfs_log_force(
 	return -EIO;
 }
 
+/*
+ * Force the log to a specific LSN.
+ *
+ * If an iclog with that lsn can be found:
+ *	If it is in the DIRTY state, just return.
+ *	If it is in the ACTIVE state, move the in-core log into the WANT_SYNC
+ *		state and go to sleep or return.
+ *	If it is in any other state, go to sleep or return.
+ *
+ * Synchronous forces are implemented with a wait queue.  All callers trying
+ * to force a given lsn to disk must wait on the queue attached to the
+ * specific in-core log.  When given in-core log finally completes its write
+ * to disk, that thread will wake up all threads waiting on the queue.
+ */
 static int
 xlog_force_lsn(
 	struct xlog		*log,
@@ -3431,18 +3440,13 @@ xlog_force_lsn(
 }
 
 /*
- * Force the in-core log to disk for a specific LSN.
+ * Force the log to a specific checkpoint sequence.
  *
- * Find in-core log with lsn.
- *	If it is in the DIRTY state, just return.
- *	If it is in the ACTIVE state, move the in-core log into the WANT_SYNC
- *		state and go to sleep or return.
- *	If it is in any other state, go to sleep or return.
- *
- * Synchronous forces are implemented with a wait queue.  All callers trying
- * to force a given lsn to disk must wait on the queue attached to the
- * specific in-core log.  When given in-core log finally completes its write
- * to disk, that thread will wake up all threads waiting on the queue.
+ * First force the CIL so that all the required changes have been flushed to the
+ * iclogs. If the CIL force completed it will return a commit LSN that indicates
+ * the iclog that needs to be flushed to stable storage. If the caller needs
+ * a synchronous log force, we will wait on the iclog with the LSN returned by
+ * xlog_cil_force_seq() to be completed.
  */
 int
 xfs_log_force_seq(
diff --git a/fs/xfs/xfs_log.h b/fs/xfs/xfs_log.h
index 8431a2f4bd13..dc1b77b92fc1 100644
--- a/fs/xfs/xfs_log.h
+++ b/fs/xfs/xfs_log.h
@@ -104,6 +104,7 @@ struct xlog_ticket;
 struct xfs_log_item;
 struct xfs_item_ops;
 struct xfs_trans;
+struct xlog;
 
 int	  xfs_log_force(struct xfs_mount *mp, uint flags);
 int	  xfs_log_force_seq(struct xfs_mount *mp, xfs_csn_t seq, uint flags,
diff --git a/fs/xfs/xfs_log_cil.c b/fs/xfs/xfs_log_cil.c
index f6c4e4e8f112..59d3bd45543b 100644
--- a/fs/xfs/xfs_log_cil.c
+++ b/fs/xfs/xfs_log_cil.c
@@ -840,6 +840,7 @@ xlog_cil_push_work(
 	xfs_csn_t		push_seq;
 	struct bio		bio;
 	DECLARE_COMPLETION_ONSTACK(bdev_flush);
+	bool			push_commit_stable;
 
 	new_ctx = kmem_zalloc(sizeof(*new_ctx), KM_NOFS);
 	new_ctx->ticket = xlog_cil_ticket_alloc(log);
@@ -850,6 +851,8 @@ xlog_cil_push_work(
 	spin_lock(&cil->xc_push_lock);
 	push_seq = cil->xc_push_seq;
 	ASSERT(push_seq <= ctx->sequence);
+	push_commit_stable = cil->xc_push_commit_stable;
+	cil->xc_push_commit_stable = false;
 
 	/*
 	 * As we are about to switch to a new, empty CIL context, we no longer
@@ -1066,8 +1069,16 @@ xlog_cil_push_work(
 	 * The commit iclog must be written to stable storage to guarantee
 	 * journal IO vs metadata writeback IO is correctly ordered on stable
 	 * storage.
+	 *
+	 * If the push caller needs the commit to be immediately stable and the
+	 * commit_iclog is not yet marked as XLOG_STATE_WANT_SYNC to indicate it
+	 * will be written when released, switch it's state to WANT_SYNC right
+	 * now.
 	 */
 	ctx->commit_iclog->ic_flags |= XLOG_ICL_NEED_FUA;
+	if (push_commit_stable &&
+	    ctx->commit_iclog->ic_state == XLOG_STATE_ACTIVE)
+		xlog_state_switch_iclogs(log, ctx->commit_iclog, 0);
 	xlog_state_release_iclog(log, ctx->commit_iclog, preflush_tail_lsn);
 
 	/* Not safe to reference ctx now! */
@@ -1161,13 +1172,26 @@ xlog_cil_push_background(
 /*
  * xlog_cil_push_now() is used to trigger an immediate CIL push to the sequence
  * number that is passed. When it returns, the work will be queued for
- * @push_seq, but it won't be completed. The caller is expected to do any
- * waiting for push_seq to complete if it is required.
+ * @push_seq, but it won't be completed.
+ *
+ * If the caller is performing a synchronous force, we will flush the workqueue
+ * to get previously queued work moving to minimise the wait time they will
+ * undergo waiting for all outstanding pushes to complete. The caller is
+ * expected to do the required waiting for push_seq to complete.
+ *
+ * If the caller is performing an async push, we need to ensure that the
+ * checkpoint is fully flushed out of the iclogs when we finish the push. If we
+ * don't do this, then the commit record may remain sitting in memory in an
+ * ACTIVE iclog. This then requires another full log force to push to disk,
+ * which defeats the purpose of having an async, non-blocking CIL force
+ * mechanism. Hence in this case we need to pass a flag to the push work to
+ * indicate it needs to flush the commit record itself.
  */
 static void
 xlog_cil_push_now(
 	struct xlog	*log,
-	xfs_lsn_t	push_seq)
+	xfs_lsn_t	push_seq,
+	bool		async)
 {
 	struct xfs_cil	*cil = log->l_cilp;
 
@@ -1177,7 +1201,8 @@ xlog_cil_push_now(
 	ASSERT(push_seq && push_seq <= cil->xc_current_sequence);
 
 	/* start on any pending background push to minimise wait time on it */
-	flush_work(&cil->xc_push_work);
+	if (!async)
+		flush_work(&cil->xc_push_work);
 
 	/*
 	 * If the CIL is empty or we've already pushed the sequence then
@@ -1190,6 +1215,7 @@ xlog_cil_push_now(
 	}
 
 	cil->xc_push_seq = push_seq;
+	cil->xc_push_commit_stable = async;
 	queue_work(log->l_mp->m_cil_workqueue, &cil->xc_push_work);
 	spin_unlock(&cil->xc_push_lock);
 }
@@ -1274,12 +1300,27 @@ xlog_cil_commit(
 	xlog_cil_push_background(log);
 }
 
+/*
+ * Flush the CIL to stable storage but don't wait for it to complete. This
+ * requires the CIL push to ensure the commit record for the push hits the disk,
+ * but otherwise is no different to a push done from a log force.
+ */
+void
+xlog_cil_flush(
+	struct xlog	*log)
+{
+	xfs_csn_t	seq = log->l_cilp->xc_current_sequence;
+
+	trace_xfs_log_force(log->l_mp, seq, _RET_IP_);
+	xlog_cil_push_now(log, seq, true);
+}
+
 /*
  * Conditionally push the CIL based on the sequence passed in.
  *
- * We only need to push if we haven't already pushed the sequence
- * number given. Hence the only time we will trigger a push here is
- * if the push sequence is the same as the current context.
+ * We only need to push if we haven't already pushed the sequence number given.
+ * Hence the only time we will trigger a push here is if the push sequence is
+ * the same as the current context.
  *
  * We return the current commit lsn to allow the callers to determine if a
  * iclog flush is necessary following this call.
@@ -1295,13 +1336,17 @@ xlog_cil_force_seq(
 
 	ASSERT(sequence <= cil->xc_current_sequence);
 
+	if (!sequence)
+		sequence = cil->xc_current_sequence;
+	trace_xfs_log_force(log->l_mp, sequence, _RET_IP_);
+
 	/*
 	 * check to see if we need to force out the current context.
 	 * xlog_cil_push() handles racing pushes for the same sequence,
 	 * so no need to deal with it here.
 	 */
 restart:
-	xlog_cil_push_now(log, sequence);
+	xlog_cil_push_now(log, sequence, false);
 
 	/*
 	 * See if we can find a previous sequence still committing.
@@ -1325,6 +1370,7 @@ xlog_cil_force_seq(
 			 * It is still being pushed! Wait for the push to
 			 * complete, then start again from the beginning.
 			 */
+			XFS_STATS_INC(log->l_mp, xs_log_force_sleep);
 			xlog_wait(&cil->xc_commit_wait, &cil->xc_push_lock);
 			goto restart;
 		}
diff --git a/fs/xfs/xfs_log_priv.h b/fs/xfs/xfs_log_priv.h
index 1ed299803904..014e0dc0ba97 100644
--- a/fs/xfs/xfs_log_priv.h
+++ b/fs/xfs/xfs_log_priv.h
@@ -277,6 +277,7 @@ struct xfs_cil {
 
 	spinlock_t		xc_push_lock ____cacheline_aligned_in_smp;
 	xfs_csn_t		xc_push_seq;
+	bool			xc_push_commit_stable;
 	struct list_head	xc_committing;
 	wait_queue_head_t	xc_commit_wait;
 	wait_queue_head_t	xc_start_wait;
@@ -520,6 +521,8 @@ int	xlog_write(struct xlog *log, struct xfs_cil_ctx *ctx,
 void	xfs_log_ticket_ungrant(struct xlog *log, struct xlog_ticket *ticket);
 void	xfs_log_ticket_regrant(struct xlog *log, struct xlog_ticket *ticket);
 
+void xlog_state_switch_iclogs(struct xlog *log, struct xlog_in_core *iclog,
+		int eventual_size);
 int xlog_state_release_iclog(struct xlog *log, struct xlog_in_core *iclog,
 		xfs_lsn_t log_tail_lsn);
 
@@ -594,6 +597,7 @@ void	xlog_cil_set_ctx_write_state(struct xfs_cil_ctx *ctx,
 /*
  * CIL force routines
  */
+void xlog_cil_flush(struct xlog *log);
 xfs_lsn_t xlog_cil_force_seq(struct xlog *log, xfs_csn_t sequence);
 
 static inline void
diff --git a/fs/xfs/xfs_sysfs.c b/fs/xfs/xfs_sysfs.c
index f1bc88f4367c..18dc5eca6c04 100644
--- a/fs/xfs/xfs_sysfs.c
+++ b/fs/xfs/xfs_sysfs.c
@@ -10,6 +10,7 @@
 #include "xfs_log_format.h"
 #include "xfs_trans_resv.h"
 #include "xfs_sysfs.h"
+#include "xfs_log.h"
 #include "xfs_log_priv.h"
 #include "xfs_mount.h"
 
diff --git a/fs/xfs/xfs_trace.c b/fs/xfs/xfs_trace.c
index 7e01e00550ac..4c86afad1617 100644
--- a/fs/xfs/xfs_trace.c
+++ b/fs/xfs/xfs_trace.c
@@ -20,6 +20,7 @@
 #include "xfs_bmap.h"
 #include "xfs_attr.h"
 #include "xfs_trans.h"
+#include "xfs_log.h"
 #include "xfs_log_priv.h"
 #include "xfs_buf_item.h"
 #include "xfs_quota.h"
diff --git a/fs/xfs/xfs_trans.c b/fs/xfs/xfs_trans.c
index b52394b0e1f4..70868f8b5911 100644
--- a/fs/xfs/xfs_trans.c
+++ b/fs/xfs/xfs_trans.c
@@ -9,7 +9,6 @@
 #include "xfs_shared.h"
 #include "xfs_format.h"
 #include "xfs_log_format.h"
-#include "xfs_log_priv.h"
 #include "xfs_trans_resv.h"
 #include "xfs_mount.h"
 #include "xfs_extent_busy.h"
@@ -17,6 +16,7 @@
 #include "xfs_trans.h"
 #include "xfs_trans_priv.h"
 #include "xfs_log.h"
+#include "xfs_log_priv.h"
 #include "xfs_trace.h"
 #include "xfs_error.h"
 #include "xfs_defer.h"
diff --git a/fs/xfs/xfs_trans_ail.c b/fs/xfs/xfs_trans_ail.c
index dbb69b4bf3ed..69aac416e2ce 100644
--- a/fs/xfs/xfs_trans_ail.c
+++ b/fs/xfs/xfs_trans_ail.c
@@ -17,6 +17,7 @@
 #include "xfs_errortag.h"
 #include "xfs_error.h"
 #include "xfs_log.h"
+#include "xfs_log_priv.h"
 
 #ifdef DEBUG
 /*
@@ -429,8 +430,12 @@ xfsaild_push(
 
 	/*
 	 * If we encountered pinned items or did not finish writing out all
-	 * buffers the last time we ran, force the log first and wait for it
-	 * before pushing again.
+	 * buffers the last time we ran, force a background CIL push to get the
+	 * items unpinned in the near future. We do not wait on the CIL push as
+	 * that could stall us for seconds if there is enough background IO
+	 * load. Stalling for that long when the tail of the log is pinned and
+	 * needs flushing will hard stop the transaction subsystem when log
+	 * space runs out.
 	 */
 	if (ailp->ail_log_flush && ailp->ail_last_pushed_lsn == 0 &&
 	    (!list_empty_careful(&ailp->ail_buf_list) ||
@@ -438,7 +443,7 @@ xfsaild_push(
 		ailp->ail_log_flush = 0;
 
 		XFS_STATS_INC(mp, xs_push_ail_flush);
-		xfs_log_force(mp, XFS_LOG_SYNC);
+		xlog_cil_flush(mp->m_log);
 	}
 
 	spin_lock(&ailp->ail_lock);
-- 
2.31.1


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

* [PATCH 2/3] xfs: CIL work is serialised, not pipelined
  2021-08-10  5:22 [PATCH 0/3 v7] xfs: make CIL pipelining work Dave Chinner
  2021-08-10  5:22 ` [PATCH 1/3] xfs: AIL needs asynchronous CIL forcing Dave Chinner
@ 2021-08-10  5:22 ` Dave Chinner
  2021-08-10  5:22 ` [PATCH 3/3] xfs: move the CIL workqueue to the CIL Dave Chinner
  2 siblings, 0 replies; 4+ messages in thread
From: Dave Chinner @ 2021-08-10  5:22 UTC (permalink / raw)
  To: linux-xfs

From: Dave Chinner <dchinner@redhat.com>

Because we use a single work structure attached to the CIL rather
than the CIL context, we can only queue a single work item at a
time. This results in the CIL being single threaded and limits
performance when it becomes CPU bound.

The design of the CIL is that it is pipelined and multiple commits
can be running concurrently, but the way the work is currently
implemented means that it is not pipelining as it was intended. The
critical work to switch the CIL context can take a few milliseconds
to run, but the rest of the CIL context flush can take hundreds of
milliseconds to complete. The context switching is the serialisation
point of the CIL, once the context has been switched the rest of the
context push can run asynchrnously with all other context pushes.

Hence we can move the work to the CIL context so that we can run
multiple CIL pushes at the same time and spread the majority of
the work out over multiple CPUs. We can keep the per-cpu CIL commit
state on the CIL rather than the context, because the context is
pinned to the CIL until the switch is done and we aggregate and
drain the per-cpu state held on the CIL during the context switch.

However, because we no longer serialise the CIL work, we can have
effectively unlimited CIL pushes in progress. We don't want to do
this - not only does it create contention on the iclogs and the
state machine locks, we can run the log right out of space with
outstanding pushes. Instead, limit the work concurrency to 4
concurrent works being processed at a time. This is enough
concurrency to remove the CIL from being a CPU bound bottleneck but
not enough to create new contention points or unbound concurrency
issues.

Signed-off-by: Dave Chinner <dchinner@redhat.com>
Reviewed-by: Darrick J. Wong <djwong@kernel.org>
---
 fs/xfs/xfs_log_cil.c  | 80 +++++++++++++++++++++++--------------------
 fs/xfs/xfs_log_priv.h |  2 +-
 fs/xfs/xfs_super.c    |  6 +++-
 3 files changed, 48 insertions(+), 40 deletions(-)

diff --git a/fs/xfs/xfs_log_cil.c b/fs/xfs/xfs_log_cil.c
index 59d3bd45543b..17785f4d50f7 100644
--- a/fs/xfs/xfs_log_cil.c
+++ b/fs/xfs/xfs_log_cil.c
@@ -47,6 +47,34 @@ xlog_cil_ticket_alloc(
 	return tic;
 }
 
+/*
+ * Unavoidable forward declaration - xlog_cil_push_work() calls
+ * xlog_cil_ctx_alloc() itself.
+ */
+static void xlog_cil_push_work(struct work_struct *work);
+
+static struct xfs_cil_ctx *
+xlog_cil_ctx_alloc(void)
+{
+	struct xfs_cil_ctx	*ctx;
+
+	ctx = kmem_zalloc(sizeof(*ctx), KM_NOFS);
+	INIT_LIST_HEAD(&ctx->committing);
+	INIT_LIST_HEAD(&ctx->busy_extents);
+	INIT_WORK(&ctx->push_work, xlog_cil_push_work);
+	return ctx;
+}
+
+static void
+xlog_cil_ctx_switch(
+	struct xfs_cil		*cil,
+	struct xfs_cil_ctx	*ctx)
+{
+	ctx->sequence = ++cil->xc_current_sequence;
+	ctx->cil = cil;
+	cil->xc_ctx = ctx;
+}
+
 /*
  * After the first stage of log recovery is done, we know where the head and
  * tail of the log are. We need this log initialisation done before we can
@@ -824,11 +852,11 @@ static void
 xlog_cil_push_work(
 	struct work_struct	*work)
 {
-	struct xfs_cil		*cil =
-		container_of(work, struct xfs_cil, xc_push_work);
+	struct xfs_cil_ctx	*ctx =
+		container_of(work, struct xfs_cil_ctx, push_work);
+	struct xfs_cil		*cil = ctx->cil;
 	struct xlog		*log = cil->xc_log;
 	struct xfs_log_vec	*lv;
-	struct xfs_cil_ctx	*ctx;
 	struct xfs_cil_ctx	*new_ctx;
 	struct xlog_ticket	*tic;
 	int			num_iovecs;
@@ -842,11 +870,10 @@ xlog_cil_push_work(
 	DECLARE_COMPLETION_ONSTACK(bdev_flush);
 	bool			push_commit_stable;
 
-	new_ctx = kmem_zalloc(sizeof(*new_ctx), KM_NOFS);
+	new_ctx = xlog_cil_ctx_alloc();
 	new_ctx->ticket = xlog_cil_ticket_alloc(log);
 
 	down_write(&cil->xc_ctx_lock);
-	ctx = cil->xc_ctx;
 
 	spin_lock(&cil->xc_push_lock);
 	push_seq = cil->xc_push_seq;
@@ -878,7 +905,7 @@ xlog_cil_push_work(
 
 
 	/* check for a previously pushed sequence */
-	if (push_seq < cil->xc_ctx->sequence) {
+	if (push_seq < ctx->sequence) {
 		spin_unlock(&cil->xc_push_lock);
 		goto out_skip;
 	}
@@ -951,19 +978,7 @@ xlog_cil_push_work(
 	}
 
 	/*
-	 * initialise the new context and attach it to the CIL. Then attach
-	 * the current context to the CIL committing list so it can be found
-	 * during log forces to extract the commit lsn of the sequence that
-	 * needs to be forced.
-	 */
-	INIT_LIST_HEAD(&new_ctx->committing);
-	INIT_LIST_HEAD(&new_ctx->busy_extents);
-	new_ctx->sequence = ctx->sequence + 1;
-	new_ctx->cil = cil;
-	cil->xc_ctx = new_ctx;
-
-	/*
-	 * The switch is now done, so we can drop the context lock and move out
+	 * Switch the contexts so we can drop the context lock and move out
 	 * of a shared context. We can't just go straight to the commit record,
 	 * though - we need to synchronise with previous and future commits so
 	 * that the commit records are correctly ordered in the log to ensure
@@ -988,7 +1003,7 @@ xlog_cil_push_work(
 	 * deferencing a freed context pointer.
 	 */
 	spin_lock(&cil->xc_push_lock);
-	cil->xc_current_sequence = new_ctx->sequence;
+	xlog_cil_ctx_switch(cil, new_ctx);
 	spin_unlock(&cil->xc_push_lock);
 	up_write(&cil->xc_ctx_lock);
 
@@ -1136,7 +1151,7 @@ xlog_cil_push_background(
 	spin_lock(&cil->xc_push_lock);
 	if (cil->xc_push_seq < cil->xc_current_sequence) {
 		cil->xc_push_seq = cil->xc_current_sequence;
-		queue_work(log->l_mp->m_cil_workqueue, &cil->xc_push_work);
+		queue_work(log->l_mp->m_cil_workqueue, &cil->xc_ctx->push_work);
 	}
 
 	/*
@@ -1202,7 +1217,7 @@ xlog_cil_push_now(
 
 	/* start on any pending background push to minimise wait time on it */
 	if (!async)
-		flush_work(&cil->xc_push_work);
+		flush_workqueue(log->l_mp->m_cil_workqueue);
 
 	/*
 	 * If the CIL is empty or we've already pushed the sequence then
@@ -1216,7 +1231,7 @@ xlog_cil_push_now(
 
 	cil->xc_push_seq = push_seq;
 	cil->xc_push_commit_stable = async;
-	queue_work(log->l_mp->m_cil_workqueue, &cil->xc_push_work);
+	queue_work(log->l_mp->m_cil_workqueue, &cil->xc_ctx->push_work);
 	spin_unlock(&cil->xc_push_lock);
 }
 
@@ -1456,13 +1471,6 @@ xlog_cil_init(
 	if (!cil)
 		return -ENOMEM;
 
-	ctx = kmem_zalloc(sizeof(*ctx), KM_MAYFAIL);
-	if (!ctx) {
-		kmem_free(cil);
-		return -ENOMEM;
-	}
-
-	INIT_WORK(&cil->xc_push_work, xlog_cil_push_work);
 	INIT_LIST_HEAD(&cil->xc_cil);
 	INIT_LIST_HEAD(&cil->xc_committing);
 	spin_lock_init(&cil->xc_cil_lock);
@@ -1471,16 +1479,12 @@ xlog_cil_init(
 	init_rwsem(&cil->xc_ctx_lock);
 	init_waitqueue_head(&cil->xc_start_wait);
 	init_waitqueue_head(&cil->xc_commit_wait);
-
-	INIT_LIST_HEAD(&ctx->committing);
-	INIT_LIST_HEAD(&ctx->busy_extents);
-	ctx->sequence = 1;
-	ctx->cil = cil;
-	cil->xc_ctx = ctx;
-	cil->xc_current_sequence = ctx->sequence;
-
 	cil->xc_log = log;
 	log->l_cilp = cil;
+
+	ctx = xlog_cil_ctx_alloc();
+	xlog_cil_ctx_switch(cil, ctx);
+
 	return 0;
 }
 
diff --git a/fs/xfs/xfs_log_priv.h b/fs/xfs/xfs_log_priv.h
index 014e0dc0ba97..5aaaf5f0b35c 100644
--- a/fs/xfs/xfs_log_priv.h
+++ b/fs/xfs/xfs_log_priv.h
@@ -249,6 +249,7 @@ struct xfs_cil_ctx {
 	struct list_head	iclog_entry;
 	struct list_head	committing;	/* ctx committing list */
 	struct work_struct	discard_endio_work;
+	struct work_struct	push_work;
 };
 
 /*
@@ -282,7 +283,6 @@ struct xfs_cil {
 	wait_queue_head_t	xc_commit_wait;
 	wait_queue_head_t	xc_start_wait;
 	xfs_csn_t		xc_current_sequence;
-	struct work_struct	xc_push_work;
 	wait_queue_head_t	xc_push_wait;	/* background push throttle */
 } ____cacheline_aligned_in_smp;
 
diff --git a/fs/xfs/xfs_super.c b/fs/xfs/xfs_super.c
index 53ce25008948..6d42883b8fae 100644
--- a/fs/xfs/xfs_super.c
+++ b/fs/xfs/xfs_super.c
@@ -518,9 +518,13 @@ xfs_init_mount_workqueues(
 	if (!mp->m_unwritten_workqueue)
 		goto out_destroy_buf;
 
+	/*
+	 * Limit the CIL pipeline depth to 4 concurrent works to bound the
+	 * concurrency the log spinlocks will be exposed to.
+	 */
 	mp->m_cil_workqueue = alloc_workqueue("xfs-cil/%s",
 			XFS_WQFLAGS(WQ_FREEZABLE | WQ_MEM_RECLAIM | WQ_UNBOUND),
-			0, mp->m_super->s_id);
+			4, mp->m_super->s_id);
 	if (!mp->m_cil_workqueue)
 		goto out_destroy_unwritten;
 
-- 
2.31.1


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

* [PATCH 3/3] xfs: move the CIL workqueue to the CIL
  2021-08-10  5:22 [PATCH 0/3 v7] xfs: make CIL pipelining work Dave Chinner
  2021-08-10  5:22 ` [PATCH 1/3] xfs: AIL needs asynchronous CIL forcing Dave Chinner
  2021-08-10  5:22 ` [PATCH 2/3] xfs: CIL work is serialised, not pipelined Dave Chinner
@ 2021-08-10  5:22 ` Dave Chinner
  2 siblings, 0 replies; 4+ messages in thread
From: Dave Chinner @ 2021-08-10  5:22 UTC (permalink / raw)
  To: linux-xfs

From: Dave Chinner <dchinner@redhat.com>

We only use the CIL workqueue in the CIL, so it makes no sense to
hang it off the xfs_mount and have to walk multiple pointers back up
to the mount when we have the CIL structures right there.

Signed-off-by: Dave Chinner <dchinner@redhat.com>
Reviewed-by: Darrick J. Wong <djwong@kernel.org>
---
 fs/xfs/xfs_log_cil.c  | 20 +++++++++++++++++---
 fs/xfs/xfs_log_priv.h |  1 +
 fs/xfs/xfs_mount.h    |  1 -
 fs/xfs/xfs_super.c    | 15 +--------------
 4 files changed, 19 insertions(+), 18 deletions(-)

diff --git a/fs/xfs/xfs_log_cil.c b/fs/xfs/xfs_log_cil.c
index 17785f4d50f7..ccd621ea9412 100644
--- a/fs/xfs/xfs_log_cil.c
+++ b/fs/xfs/xfs_log_cil.c
@@ -1151,7 +1151,7 @@ xlog_cil_push_background(
 	spin_lock(&cil->xc_push_lock);
 	if (cil->xc_push_seq < cil->xc_current_sequence) {
 		cil->xc_push_seq = cil->xc_current_sequence;
-		queue_work(log->l_mp->m_cil_workqueue, &cil->xc_ctx->push_work);
+		queue_work(cil->xc_push_wq, &cil->xc_ctx->push_work);
 	}
 
 	/*
@@ -1217,7 +1217,7 @@ xlog_cil_push_now(
 
 	/* start on any pending background push to minimise wait time on it */
 	if (!async)
-		flush_workqueue(log->l_mp->m_cil_workqueue);
+		flush_workqueue(cil->xc_push_wq);
 
 	/*
 	 * If the CIL is empty or we've already pushed the sequence then
@@ -1231,7 +1231,7 @@ xlog_cil_push_now(
 
 	cil->xc_push_seq = push_seq;
 	cil->xc_push_commit_stable = async;
-	queue_work(log->l_mp->m_cil_workqueue, &cil->xc_ctx->push_work);
+	queue_work(cil->xc_push_wq, &cil->xc_ctx->push_work);
 	spin_unlock(&cil->xc_push_lock);
 }
 
@@ -1470,6 +1470,15 @@ xlog_cil_init(
 	cil = kmem_zalloc(sizeof(*cil), KM_MAYFAIL);
 	if (!cil)
 		return -ENOMEM;
+	/*
+	 * Limit the CIL pipeline depth to 4 concurrent works to bound the
+	 * concurrency the log spinlocks will be exposed to.
+	 */
+	cil->xc_push_wq = alloc_workqueue("xfs-cil/%s",
+			XFS_WQFLAGS(WQ_FREEZABLE | WQ_MEM_RECLAIM | WQ_UNBOUND),
+			4, log->l_mp->m_super->s_id);
+	if (!cil->xc_push_wq)
+		goto out_destroy_cil;
 
 	INIT_LIST_HEAD(&cil->xc_cil);
 	INIT_LIST_HEAD(&cil->xc_committing);
@@ -1486,6 +1495,10 @@ xlog_cil_init(
 	xlog_cil_ctx_switch(cil, ctx);
 
 	return 0;
+
+out_destroy_cil:
+	kmem_free(cil);
+	return -ENOMEM;
 }
 
 void
@@ -1499,6 +1512,7 @@ xlog_cil_destroy(
 	}
 
 	ASSERT(list_empty(&log->l_cilp->xc_cil));
+	destroy_workqueue(log->l_cilp->xc_push_wq);
 	kmem_free(log->l_cilp);
 }
 
diff --git a/fs/xfs/xfs_log_priv.h b/fs/xfs/xfs_log_priv.h
index 5aaaf5f0b35c..844fbeec3545 100644
--- a/fs/xfs/xfs_log_priv.h
+++ b/fs/xfs/xfs_log_priv.h
@@ -272,6 +272,7 @@ struct xfs_cil {
 	struct xlog		*xc_log;
 	struct list_head	xc_cil;
 	spinlock_t		xc_cil_lock;
+	struct workqueue_struct	*xc_push_wq;
 
 	struct rw_semaphore	xc_ctx_lock ____cacheline_aligned_in_smp;
 	struct xfs_cil_ctx	*xc_ctx;
diff --git a/fs/xfs/xfs_mount.h b/fs/xfs/xfs_mount.h
index 32143102cc91..2266c6a668cf 100644
--- a/fs/xfs/xfs_mount.h
+++ b/fs/xfs/xfs_mount.h
@@ -107,7 +107,6 @@ typedef struct xfs_mount {
 	struct xfs_mru_cache	*m_filestream;  /* per-mount filestream data */
 	struct workqueue_struct *m_buf_workqueue;
 	struct workqueue_struct	*m_unwritten_workqueue;
-	struct workqueue_struct	*m_cil_workqueue;
 	struct workqueue_struct	*m_reclaim_workqueue;
 	struct workqueue_struct	*m_sync_workqueue;
 	struct workqueue_struct *m_blockgc_wq;
diff --git a/fs/xfs/xfs_super.c b/fs/xfs/xfs_super.c
index 6d42883b8fae..7b55464f6de0 100644
--- a/fs/xfs/xfs_super.c
+++ b/fs/xfs/xfs_super.c
@@ -518,21 +518,11 @@ xfs_init_mount_workqueues(
 	if (!mp->m_unwritten_workqueue)
 		goto out_destroy_buf;
 
-	/*
-	 * Limit the CIL pipeline depth to 4 concurrent works to bound the
-	 * concurrency the log spinlocks will be exposed to.
-	 */
-	mp->m_cil_workqueue = alloc_workqueue("xfs-cil/%s",
-			XFS_WQFLAGS(WQ_FREEZABLE | WQ_MEM_RECLAIM | WQ_UNBOUND),
-			4, mp->m_super->s_id);
-	if (!mp->m_cil_workqueue)
-		goto out_destroy_unwritten;
-
 	mp->m_reclaim_workqueue = alloc_workqueue("xfs-reclaim/%s",
 			XFS_WQFLAGS(WQ_FREEZABLE | WQ_MEM_RECLAIM),
 			0, mp->m_super->s_id);
 	if (!mp->m_reclaim_workqueue)
-		goto out_destroy_cil;
+		goto out_destroy_unwritten;
 
 	mp->m_blockgc_wq = alloc_workqueue("xfs-blockgc/%s",
 			XFS_WQFLAGS(WQ_UNBOUND | WQ_FREEZABLE | WQ_MEM_RECLAIM),
@@ -559,8 +549,6 @@ xfs_init_mount_workqueues(
 	destroy_workqueue(mp->m_blockgc_wq);
 out_destroy_reclaim:
 	destroy_workqueue(mp->m_reclaim_workqueue);
-out_destroy_cil:
-	destroy_workqueue(mp->m_cil_workqueue);
 out_destroy_unwritten:
 	destroy_workqueue(mp->m_unwritten_workqueue);
 out_destroy_buf:
@@ -577,7 +565,6 @@ xfs_destroy_mount_workqueues(
 	destroy_workqueue(mp->m_blockgc_wq);
 	destroy_workqueue(mp->m_inodegc_wq);
 	destroy_workqueue(mp->m_reclaim_workqueue);
-	destroy_workqueue(mp->m_cil_workqueue);
 	destroy_workqueue(mp->m_unwritten_workqueue);
 	destroy_workqueue(mp->m_buf_workqueue);
 }
-- 
2.31.1


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

end of thread, other threads:[~2021-08-10  5:23 UTC | newest]

Thread overview: 4+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2021-08-10  5:22 [PATCH 0/3 v7] xfs: make CIL pipelining work Dave Chinner
2021-08-10  5:22 ` [PATCH 1/3] xfs: AIL needs asynchronous CIL forcing Dave Chinner
2021-08-10  5:22 ` [PATCH 2/3] xfs: CIL work is serialised, not pipelined Dave Chinner
2021-08-10  5:22 ` [PATCH 3/3] xfs: move the CIL workqueue to the CIL Dave Chinner

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