Discussion:
[PATCH 2/3] xfs: convert xfsbufd to use a workqueue
(too old to reply)
Dave Chinner
2011-07-18 03:49:48 UTC
Permalink
From: Dave Chinner <***@redhat.com>

There is no reason we need a thread per filesystem to do the
flushing of the delayed write buffer queue. This can be easily
handled by a global concurrency managed workqueue.

Convert the delayed write buffer handling to use workqueues and
workqueue flushes to implement buffer writeback by embedding a
delayed work structure into the struct xfs_buftarg and using that to
control flushing. While there, group all the delayed write list and
buffer handling functions into the same section of code to make it
easier to find all the relevant code.

This greatly simplifes the process of flushing and also removes a
bunch of duplicated code between buftarg flushing and delwri buffer
writeback.

Signed-off-by: Dave Chinner <***@redhat.com>
---
fs/xfs/linux-2.6/xfs_buf.c | 522 ++++++++++++++++++++------------------------
fs/xfs/linux-2.6/xfs_buf.h | 5 +-
fs/xfs/quota/xfs_dquot.c | 1 -
fs/xfs/xfs_trans_ail.c | 2 +-
4 files changed, 244 insertions(+), 286 deletions(-)

diff --git a/fs/xfs/linux-2.6/xfs_buf.c b/fs/xfs/linux-2.6/xfs_buf.c
index b2b4119..1d2b5f9 100644
--- a/fs/xfs/linux-2.6/xfs_buf.c
+++ b/fs/xfs/linux-2.6/xfs_buf.c
@@ -33,6 +33,7 @@
#include <linux/migrate.h>
#include <linux/backing-dev.h>
#include <linux/freezer.h>
+#include <linux/workqueue.h>

#include "xfs_sb.h"
#include "xfs_inum.h"
@@ -42,9 +43,9 @@
#include "xfs_trace.h"

static kmem_zone_t *xfs_buf_zone;
-STATIC int xfsbufd(void *);
STATIC void xfs_buf_delwri_queue(xfs_buf_t *, int);

+static struct workqueue_struct *xfs_buf_wq;
static struct workqueue_struct *xfslogd_workqueue;
struct workqueue_struct *xfsdatad_workqueue;
struct workqueue_struct *xfsconvertd_workqueue;
@@ -1367,9 +1368,236 @@ xfs_buf_iomove(
}
}

+
+
+/*
+ * Delayed write buffer handling
+ */
+STATIC void
+xfs_buf_delwri_queue(
+ xfs_buf_t *bp,
+ int unlock)
+{
+ struct list_head *dwq = &bp->b_target->bt_delwrite_queue;
+ spinlock_t *dwlk = &bp->b_target->bt_delwrite_lock;
+
+ trace_xfs_buf_delwri_queue(bp, _RET_IP_);
+
+ ASSERT((bp->b_flags&(XBF_DELWRI|XBF_ASYNC)) == (XBF_DELWRI|XBF_ASYNC));
+
+ spin_lock(dwlk);
+ /* If already in the queue, dequeue and place at tail */
+ if (!list_empty(&bp->b_list)) {
+ ASSERT(bp->b_flags & _XBF_DELWRI_Q);
+ if (unlock)
+ atomic_dec(&bp->b_hold);
+ list_del(&bp->b_list);
+ }
+
+ if (list_empty(dwq)) {
+ /* queue a delayed flush as we are about to queue a buffer */
+ queue_delayed_work(xfs_buf_wq, &bp->b_target->bt_delwrite_work,
+ xfs_buf_timer_centisecs * msecs_to_jiffies(10));
+ }
+
+ bp->b_flags |= _XBF_DELWRI_Q;
+ list_add_tail(&bp->b_list, dwq);
+ bp->b_queuetime = jiffies;
+ spin_unlock(dwlk);
+
+ if (unlock)
+ xfs_buf_unlock(bp);
+}
+
+void
+xfs_buf_delwri_dequeue(
+ xfs_buf_t *bp)
+{
+ spinlock_t *dwlk = &bp->b_target->bt_delwrite_lock;
+ int dequeued = 0;
+
+ spin_lock(dwlk);
+ if ((bp->b_flags & XBF_DELWRI) && !list_empty(&bp->b_list)) {
+ ASSERT(bp->b_flags & _XBF_DELWRI_Q);
+ list_del_init(&bp->b_list);
+ dequeued = 1;
+ }
+ bp->b_flags &= ~(XBF_DELWRI|_XBF_DELWRI_Q);
+ spin_unlock(dwlk);
+
+ if (dequeued)
+ xfs_buf_rele(bp);
+
+ trace_xfs_buf_delwri_dequeue(bp, _RET_IP_);
+}
+
+/*
+ * If a delwri buffer needs to be pushed before it has aged out, then promote
+ * it to the head of the delwri queue so that it will be flushed on the next
+ * xfsbufd run. We do this by resetting the queuetime of the buffer to be older
+ * than the age currently needed to flush the buffer. Hence the next time the
+ * xfsbufd sees it is guaranteed to be considered old enough to flush.
+ */
+void
+xfs_buf_delwri_promote(
+ struct xfs_buf *bp)
+{
+ struct xfs_buftarg *btp = bp->b_target;
+ long age = xfs_buf_age_centisecs * msecs_to_jiffies(10) + 1;
+
+ ASSERT(bp->b_flags & XBF_DELWRI);
+ ASSERT(bp->b_flags & _XBF_DELWRI_Q);
+
+ /*
+ * Check the buffer age before locking the delayed write queue as we
+ * don't need to promote buffers that are already past the flush age.
+ */
+ if (bp->b_queuetime < jiffies - age)
+ return;
+ bp->b_queuetime = jiffies - age;
+ spin_lock(&btp->bt_delwrite_lock);
+ list_move(&bp->b_list, &btp->bt_delwrite_queue);
+ spin_unlock(&btp->bt_delwrite_lock);
+}
+
+/*
+ * Move buffers older than the age specified to the supplied list, avoiding
+ * locked buffers to prevent deadlocks.
+ */
+STATIC void
+xfs_buf_delwri_split(
+ xfs_buftarg_t *target,
+ struct list_head *list,
+ unsigned long age,
+ int force)
+{
+ xfs_buf_t *bp, *n;
+ struct list_head *dwq = &target->bt_delwrite_queue;
+ spinlock_t *dwlk = &target->bt_delwrite_lock;
+
+ INIT_LIST_HEAD(list);
+ spin_lock(dwlk);
+ list_for_each_entry_safe(bp, n, dwq, b_list) {
+ ASSERT(bp->b_flags & XBF_DELWRI);
+
+ if (!XFS_BUF_ISPINNED(bp) && xfs_buf_trylock(bp)) {
+ if (!force &&
+ time_before(jiffies, bp->b_queuetime + age)) {
+ xfs_buf_unlock(bp);
+ break;
+ }
+
+ bp->b_flags &= ~(XBF_DELWRI | _XBF_DELWRI_Q);
+ bp->b_flags |= XBF_WRITE;
+ list_move_tail(&bp->b_list, list);
+ trace_xfs_buf_delwri_split(bp, _RET_IP_);
+ }
+ }
+ spin_unlock(dwlk);
+}
+
/*
- * Handling of buffer targets (buftargs).
+ * Compare function is more complex than it needs to be because
+ * the return value is only 32 bits and we are doing comparisons
+ * on 64 bit values
*/
+static int
+xfs_buf_cmp(
+ void *priv,
+ struct list_head *a,
+ struct list_head *b)
+{
+ struct xfs_buf *ap = container_of(a, struct xfs_buf, b_list);
+ struct xfs_buf *bp = container_of(b, struct xfs_buf, b_list);
+ xfs_daddr_t diff;
+
+ diff = ap->b_bn - bp->b_bn;
+ if (diff < 0)
+ return -1;
+ if (diff > 0)
+ return 1;
+ return 0;
+}
+
+/*
+ * If we are doing a forced flush, then we need to wait for the IO that we
+ * issue to complete.
+ */
+static void
+xfs_buf_delwri_work(
+ struct work_struct *work)
+{
+ struct xfs_buftarg *btp = container_of(to_delayed_work(work),
+ struct xfs_buftarg, bt_delwrite_work);
+ struct xfs_buf *bp;
+ struct blk_plug plug;
+ LIST_HEAD(tmp_list);
+ LIST_HEAD(wait_list);
+ long age = xfs_buf_age_centisecs * msecs_to_jiffies(10);
+ int force = 0;
+
+ if (test_and_clear_bit(XBT_FORCE_FLUSH, &btp->bt_flags)) {
+ force = 1;
+ age = 0;
+ }
+
+ xfs_buf_delwri_split(btp, &tmp_list, age, force);
+ list_sort(NULL, &tmp_list, xfs_buf_cmp);
+
+ blk_start_plug(&plug);
+ while (!list_empty(&tmp_list)) {
+ bp = list_first_entry(&tmp_list, struct xfs_buf, b_list);
+ list_del_init(&bp->b_list);
+ if (force) {
+ bp->b_flags &= ~XBF_ASYNC;
+ list_add(&bp->b_list, &wait_list);
+ }
+ xfs_bdstrat_cb(bp);
+ }
+ blk_finish_plug(&plug);
+
+ if (force) {
+ /* Wait for IO to complete. */
+ while (!list_empty(&wait_list)) {
+ bp = list_first_entry(&wait_list, struct xfs_buf, b_list);
+
+ list_del_init(&bp->b_list);
+ xfs_buf_iowait(bp);
+ xfs_buf_relse(bp);
+ }
+ }
+
+ if (list_empty(&btp->bt_delwrite_queue))
+ return;
+
+ queue_delayed_work(xfs_buf_wq, &btp->bt_delwrite_work,
+ xfs_buf_timer_centisecs * msecs_to_jiffies(10));
+}
+
+/*
+ * Go through all incore buffers, and release buffers if they belong to
+ * the given device. This is used in filesystem error handling to
+ * preserve the consistency of its metadata.
+ *
+ * If we flush all the delayed write metadata, return 0. Otherwise, return 1 to
+ * indicate more work needs to be done.
+ */
+int
+xfs_flush_buftarg(
+ xfs_buftarg_t *target,
+ int wait)
+{
+ flush_workqueue(xfsconvertd_workqueue);
+ flush_workqueue(xfsdatad_workqueue);
+ flush_workqueue(xfslogd_workqueue);
+
+ set_bit(XBT_FORCE_FLUSH, &target->bt_flags);
+ flush_delayed_work_sync(&target->bt_delwrite_work);
+
+ if (!list_empty(&target->bt_delwrite_queue))
+ return 1;
+ return 0;
+}

/*
* Wait for any bufs with callbacks that have been submitted but have not yet
@@ -1463,7 +1691,6 @@ xfs_free_buftarg(
if (mp->m_flags & XFS_MOUNT_BARRIER)
xfs_blkdev_issue_flush(btp);

- kthread_stop(btp->bt_task);
kmem_free(btp);
}

@@ -1511,19 +1738,6 @@ xfs_setsize_buftarg(
return xfs_setsize_buftarg_flags(btp, blocksize, sectorsize, 1);
}

-STATIC int
-xfs_alloc_delwrite_queue(
- xfs_buftarg_t *btp,
- const char *fsname)
-{
- INIT_LIST_HEAD(&btp->bt_delwrite_queue);
- spin_lock_init(&btp->bt_delwrite_lock);
- btp->bt_flags = 0;
- btp->bt_task = kthread_run(xfsbufd, btp, "xfsbufd/%s", fsname);
- if (IS_ERR(btp->bt_task))
- return PTR_ERR(btp->bt_task);
- return 0;
-}

xfs_buftarg_t *
xfs_alloc_buftarg(
@@ -1545,10 +1759,13 @@ xfs_alloc_buftarg(

INIT_LIST_HEAD(&btp->bt_lru);
spin_lock_init(&btp->bt_lru_lock);
+
+ INIT_LIST_HEAD(&btp->bt_delwrite_queue);
+ spin_lock_init(&btp->bt_delwrite_lock);
+ INIT_DELAYED_WORK(&btp->bt_delwrite_work, xfs_buf_delwri_work);
+
if (xfs_setsize_buftarg_early(btp, bdev))
goto error;
- if (xfs_alloc_delwrite_queue(btp, fsname))
- goto error;
btp->bt_shrinker.shrink = xfs_buftarg_shrink;
btp->bt_shrinker.seeks = DEFAULT_SEEKS;
register_shrinker(&btp->bt_shrinker);
@@ -1559,269 +1776,6 @@ error:
return NULL;
}

-
-/*
- * Delayed write buffer handling
- */
-STATIC void
-xfs_buf_delwri_queue(
- xfs_buf_t *bp,
- int unlock)
-{
- struct list_head *dwq = &bp->b_target->bt_delwrite_queue;
- spinlock_t *dwlk = &bp->b_target->bt_delwrite_lock;
-
- trace_xfs_buf_delwri_queue(bp, _RET_IP_);
-
- ASSERT((bp->b_flags&(XBF_DELWRI|XBF_ASYNC)) == (XBF_DELWRI|XBF_ASYNC));
-
- spin_lock(dwlk);
- /* If already in the queue, dequeue and place at tail */
- if (!list_empty(&bp->b_list)) {
- ASSERT(bp->b_flags & _XBF_DELWRI_Q);
- if (unlock)
- atomic_dec(&bp->b_hold);
- list_del(&bp->b_list);
- }
-
- if (list_empty(dwq)) {
- /* start xfsbufd as it is about to have something to do */
- wake_up_process(bp->b_target->bt_task);
- }
-
- bp->b_flags |= _XBF_DELWRI_Q;
- list_add_tail(&bp->b_list, dwq);
- bp->b_queuetime = jiffies;
- spin_unlock(dwlk);
-
- if (unlock)
- xfs_buf_unlock(bp);
-}
-
-void
-xfs_buf_delwri_dequeue(
- xfs_buf_t *bp)
-{
- spinlock_t *dwlk = &bp->b_target->bt_delwrite_lock;
- int dequeued = 0;
-
- spin_lock(dwlk);
- if ((bp->b_flags & XBF_DELWRI) && !list_empty(&bp->b_list)) {
- ASSERT(bp->b_flags & _XBF_DELWRI_Q);
- list_del_init(&bp->b_list);
- dequeued = 1;
- }
- bp->b_flags &= ~(XBF_DELWRI|_XBF_DELWRI_Q);
- spin_unlock(dwlk);
-
- if (dequeued)
- xfs_buf_rele(bp);
-
- trace_xfs_buf_delwri_dequeue(bp, _RET_IP_);
-}
-
-/*
- * If a delwri buffer needs to be pushed before it has aged out, then promote
- * it to the head of the delwri queue so that it will be flushed on the next
- * xfsbufd run. We do this by resetting the queuetime of the buffer to be older
- * than the age currently needed to flush the buffer. Hence the next time the
- * xfsbufd sees it is guaranteed to be considered old enough to flush.
- */
-void
-xfs_buf_delwri_promote(
- struct xfs_buf *bp)
-{
- struct xfs_buftarg *btp = bp->b_target;
- long age = xfs_buf_age_centisecs * msecs_to_jiffies(10) + 1;
-
- ASSERT(bp->b_flags & XBF_DELWRI);
- ASSERT(bp->b_flags & _XBF_DELWRI_Q);
-
- /*
- * Check the buffer age before locking the delayed write queue as we
- * don't need to promote buffers that are already past the flush age.
- */
- if (bp->b_queuetime < jiffies - age)
- return;
- bp->b_queuetime = jiffies - age;
- spin_lock(&btp->bt_delwrite_lock);
- list_move(&bp->b_list, &btp->bt_delwrite_queue);
- spin_unlock(&btp->bt_delwrite_lock);
-}
-
-STATIC void
-xfs_buf_runall_queues(
- struct workqueue_struct *queue)
-{
- flush_workqueue(queue);
-}
-
-/*
- * Move as many buffers as specified to the supplied list
- * idicating if we skipped any buffers to prevent deadlocks.
- */
-STATIC int
-xfs_buf_delwri_split(
- xfs_buftarg_t *target,
- struct list_head *list,
- unsigned long age)
-{
- xfs_buf_t *bp, *n;
- struct list_head *dwq = &target->bt_delwrite_queue;
- spinlock_t *dwlk = &target->bt_delwrite_lock;
- int skipped = 0;
- int force;
-
- force = test_and_clear_bit(XBT_FORCE_FLUSH, &target->bt_flags);
- INIT_LIST_HEAD(list);
- spin_lock(dwlk);
- list_for_each_entry_safe(bp, n, dwq, b_list) {
- ASSERT(bp->b_flags & XBF_DELWRI);
-
- if (!XFS_BUF_ISPINNED(bp) && xfs_buf_trylock(bp)) {
- if (!force &&
- time_before(jiffies, bp->b_queuetime + age)) {
- xfs_buf_unlock(bp);
- break;
- }
-
- bp->b_flags &= ~(XBF_DELWRI | _XBF_DELWRI_Q);
- bp->b_flags |= XBF_WRITE;
- list_move_tail(&bp->b_list, list);
- trace_xfs_buf_delwri_split(bp, _RET_IP_);
- } else
- skipped++;
- }
- spin_unlock(dwlk);
-
- return skipped;
-
-}
-
-/*
- * Compare function is more complex than it needs to be because
- * the return value is only 32 bits and we are doing comparisons
- * on 64 bit values
- */
-static int
-xfs_buf_cmp(
- void *priv,
- struct list_head *a,
- struct list_head *b)
-{
- struct xfs_buf *ap = container_of(a, struct xfs_buf, b_list);
- struct xfs_buf *bp = container_of(b, struct xfs_buf, b_list);
- xfs_daddr_t diff;
-
- diff = ap->b_bn - bp->b_bn;
- if (diff < 0)
- return -1;
- if (diff > 0)
- return 1;
- return 0;
-}
-
-STATIC int
-xfsbufd(
- void *data)
-{
- xfs_buftarg_t *target = (xfs_buftarg_t *)data;
-
- current->flags |= PF_MEMALLOC;
-
- set_freezable();
-
- do {
- long age = xfs_buf_age_centisecs * msecs_to_jiffies(10);
- long tout = xfs_buf_timer_centisecs * msecs_to_jiffies(10);
- struct list_head tmp;
- struct blk_plug plug;
-
- if (unlikely(freezing(current))) {
- set_bit(XBT_FORCE_SLEEP, &target->bt_flags);
- refrigerator();
- } else {
- clear_bit(XBT_FORCE_SLEEP, &target->bt_flags);
- }
-
- /* sleep for a long time if there is nothing to do. */
- if (list_empty(&target->bt_delwrite_queue))
- tout = MAX_SCHEDULE_TIMEOUT;
- schedule_timeout_interruptible(tout);
-
- xfs_buf_delwri_split(target, &tmp, age);
- list_sort(NULL, &tmp, xfs_buf_cmp);
-
- blk_start_plug(&plug);
- while (!list_empty(&tmp)) {
- struct xfs_buf *bp;
- bp = list_first_entry(&tmp, struct xfs_buf, b_list);
- list_del_init(&bp->b_list);
- xfs_bdstrat_cb(bp);
- }
- blk_finish_plug(&plug);
- } while (!kthread_should_stop());
-
- return 0;
-}
-
-/*
- * Go through all incore buffers, and release buffers if they belong to
- * the given device. This is used in filesystem error handling to
- * preserve the consistency of its metadata.
- */
-int
-xfs_flush_buftarg(
- xfs_buftarg_t *target,
- int wait)
-{
- xfs_buf_t *bp;
- int pincount = 0;
- LIST_HEAD(tmp_list);
- LIST_HEAD(wait_list);
- struct blk_plug plug;
-
- xfs_buf_runall_queues(xfsconvertd_workqueue);
- xfs_buf_runall_queues(xfsdatad_workqueue);
- xfs_buf_runall_queues(xfslogd_workqueue);
-
- set_bit(XBT_FORCE_FLUSH, &target->bt_flags);
- pincount = xfs_buf_delwri_split(target, &tmp_list, 0);
-
- /*
- * Dropped the delayed write list lock, now walk the temporary list.
- * All I/O is issued async and then if we need to wait for completion
- * we do that after issuing all the IO.
- */
- list_sort(NULL, &tmp_list, xfs_buf_cmp);
-
- blk_start_plug(&plug);
- while (!list_empty(&tmp_list)) {
- bp = list_first_entry(&tmp_list, struct xfs_buf, b_list);
- ASSERT(target == bp->b_target);
- list_del_init(&bp->b_list);
- if (wait) {
- bp->b_flags &= ~XBF_ASYNC;
- list_add(&bp->b_list, &wait_list);
- }
- xfs_bdstrat_cb(bp);
- }
- blk_finish_plug(&plug);
-
- if (wait) {
- /* Wait for IO to complete. */
- while (!list_empty(&wait_list)) {
- bp = list_first_entry(&wait_list, struct xfs_buf, b_list);
-
- list_del_init(&bp->b_list);
- xfs_buf_iowait(bp);
- xfs_buf_relse(bp);
- }
- }
-
- return pincount;
-}
-
int __init
xfs_buf_init(void)
{
@@ -1844,8 +1798,13 @@ xfs_buf_init(void)
if (!xfsconvertd_workqueue)
goto out_destroy_xfsdatad_workqueue;

+ xfs_buf_wq = alloc_workqueue("xfsbufd", WQ_MEM_RECLAIM, 8);
+ if (!xfs_buf_wq)
+ goto out_destroy_xfsconvertd_wq;
return 0;

+ out_destroy_xfsconvertd_wq:
+ destroy_workqueue(xfsconvertd_workqueue);
out_destroy_xfsdatad_workqueue:
destroy_workqueue(xfsdatad_workqueue);
out_destroy_xfslogd_workqueue:
@@ -1859,6 +1818,7 @@ xfs_buf_init(void)
void
xfs_buf_terminate(void)
{
+ destroy_workqueue(xfs_buf_wq);
destroy_workqueue(xfsconvertd_workqueue);
destroy_workqueue(xfsdatad_workqueue);
destroy_workqueue(xfslogd_workqueue);
diff --git a/fs/xfs/linux-2.6/xfs_buf.h b/fs/xfs/linux-2.6/xfs_buf.h
index 6a83b46..2196cd1 100644
--- a/fs/xfs/linux-2.6/xfs_buf.h
+++ b/fs/xfs/linux-2.6/xfs_buf.h
@@ -90,8 +90,7 @@ typedef unsigned int xfs_buf_flags_t;
{ _XBF_DELWRI_Q, "DELWRI_Q" }

typedef enum {
- XBT_FORCE_SLEEP = 0,
- XBT_FORCE_FLUSH = 1,
+ XBT_FORCE_FLUSH = 0,
} xfs_buftarg_flags_t;

typedef struct xfs_buftarg {
@@ -104,7 +103,7 @@ typedef struct xfs_buftarg {
size_t bt_smask;

/* per device delwri queue */
- struct task_struct *bt_task;
+ struct delayed_work bt_delwrite_work;
struct list_head bt_delwrite_queue;
spinlock_t bt_delwrite_lock;
unsigned long bt_flags;
diff --git a/fs/xfs/quota/xfs_dquot.c b/fs/xfs/quota/xfs_dquot.c
index 837f311..0136928 100644
--- a/fs/xfs/quota/xfs_dquot.c
+++ b/fs/xfs/quota/xfs_dquot.c
@@ -1450,7 +1450,6 @@ xfs_qm_dqflock_pushbuf_wait(
if (XFS_BUF_ISPINNED(bp))
xfs_log_force(mp, 0);
xfs_buf_delwri_promote(bp);
- wake_up_process(bp->b_target->bt_task);
}
xfs_buf_relse(bp);
out_lock:
diff --git a/fs/xfs/xfs_trans_ail.c b/fs/xfs/xfs_trans_ail.c
index 8cd329b..3cfd6e6 100644
--- a/fs/xfs/xfs_trans_ail.c
+++ b/fs/xfs/xfs_trans_ail.c
@@ -505,7 +505,7 @@ xfs_ail_worker(

if (push_xfsbufd) {
/* we've got delayed write buffers to flush */
- wake_up_process(mp->m_ddev_targp->bt_task);
+ flush_delayed_work(&mp->m_ddev_targp->bt_delwrite_work);
}

/* assume we have more work to do in a short while */
--
1.7.5.1
Dave Chinner
2011-07-18 03:49:47 UTC
Permalink
From: Dave Chinner <***@redhat.com>

We currently have significant issues with the amount of stack that
allocation in XFS uses, especially in the writeback path. We can
easily consume 4k of stack between mapping the page, manipulating
the bmap btree and allocating blocks from the free list. Not to
mention btree block readahead and other functionality that issues IO
in the allocation path.

As a result, we can no longer fit allocation in the writeback path
in the stack space provided on x86_64. To alleviate this problem,
introduce an allocation workqueue and move all allocations to a
seperate context. This can be easily added as an interposing layer
into xfs_alloc_vextent(), which takes a single argument structure
and does not return until the allocation is complete or has failed.

To do this, add a work structure and a completion to the allocation
args structure. This allows xfs_alloc_vextent to queue the args onto
the workqueue and wait for it to be completed by the worker. This
can be done completely transparently to the caller.

The worker function needs to ensure that it sets and clears the
PF_TRANS flag appropriately as it is being run in an active
transaction context. Work can also be queued in a memory reclaim
context, so a rescuer is needed for the workqueue.

Signed-off-by: Dave Chinner <***@redhat.com>
---
fs/xfs/linux-2.6/xfs_super.c | 13 +++++++++++++
fs/xfs/xfs_alloc.c | 34 +++++++++++++++++++++++++++++++++-
fs/xfs/xfs_alloc.h | 5 +++++
3 files changed, 51 insertions(+), 1 deletions(-)

diff --git a/fs/xfs/linux-2.6/xfs_super.c b/fs/xfs/linux-2.6/xfs_super.c
index 9a72dda..6a6d4d9 100644
--- a/fs/xfs/linux-2.6/xfs_super.c
+++ b/fs/xfs/linux-2.6/xfs_super.c
@@ -1672,8 +1672,21 @@ xfs_init_workqueues(void)
if (!xfs_ail_wq)
goto out_destroy_syncd;

+ /*
+ * The allocation workqueue can be used in memory reclaim situations
+ * (writepage path), and parallelism is only limited by the number of
+ * AGs in all the filesystems mounted. Hence maxactive is set very
+ * high.
+ */
+ xfs_alloc_wq = alloc_workqueue("xfsalloc",
+ WQ_CPU_INTENSIVE | WQ_MEM_RECLAIM, 128);
+ if (!xfs_alloc_wq)
+ goto out_destroy_ail;
+
return 0;

+out_destroy_ail:
+ destroy_workqueue(xfs_ail_wq);
out_destroy_syncd:
destroy_workqueue(xfs_syncd_wq);
out:
diff --git a/fs/xfs/xfs_alloc.c b/fs/xfs/xfs_alloc.c
index 1e00b3e..5883972 100644
--- a/fs/xfs/xfs_alloc.c
+++ b/fs/xfs/xfs_alloc.c
@@ -35,6 +35,7 @@
#include "xfs_error.h"
#include "xfs_trace.h"

+struct workqueue_struct *xfs_alloc_wq;

#define XFS_ABSDIFF(a,b) (((a) <= (b)) ? ((b) - (a)) : ((a) - (b)))

@@ -2208,7 +2209,7 @@ xfs_alloc_read_agf(
* group or loop over the allocation groups to find the result.
*/
int /* error */
-xfs_alloc_vextent(
+__xfs_alloc_vextent(
xfs_alloc_arg_t *args) /* allocation argument structure */
{
xfs_agblock_t agsize; /* allocation group size */
@@ -2418,6 +2419,37 @@ error0:
return error;
}

+static void
+xfs_alloc_vextent_worker(
+ struct work_struct *work)
+{
+ struct xfs_alloc_arg *args = container_of(work,
+ struct xfs_alloc_arg, work);
+ unsigned long pflags;
+
+ /* we are in a transaction context here */
+ current_set_flags_nested(&pflags, PF_FSTRANS);
+
+ args->result = __xfs_alloc_vextent(args);
+ complete(args->done);
+
+ current_restore_flags_nested(&pflags, PF_FSTRANS);
+}
+
+
+int /* error */
+xfs_alloc_vextent(
+ xfs_alloc_arg_t *args) /* allocation argument structure */
+{
+ DECLARE_COMPLETION_ONSTACK(done);
+
+ args->done = &done;
+ INIT_WORK(&args->work, xfs_alloc_vextent_worker);
+ queue_work(xfs_alloc_wq, &args->work);
+ wait_for_completion(&done);
+ return args->result;
+}
+
/*
* Free an extent.
* Just break up the extent address and hand off to xfs_free_ag_extent
diff --git a/fs/xfs/xfs_alloc.h b/fs/xfs/xfs_alloc.h
index 2f52b92..ab5d0fd 100644
--- a/fs/xfs/xfs_alloc.h
+++ b/fs/xfs/xfs_alloc.h
@@ -25,6 +25,8 @@ struct xfs_perag;
struct xfs_trans;
struct xfs_busy_extent;

+extern struct workqueue_struct *xfs_alloc_wq;
+
/*
* Freespace allocation types. Argument to xfs_alloc_[v]extent.
*/
@@ -119,6 +121,9 @@ typedef struct xfs_alloc_arg {
char isfl; /* set if is freelist blocks - !acctg */
char userdata; /* set if this is user data */
xfs_fsblock_t firstblock; /* io first block allocated */
+ struct completion *done;
+ struct work_struct work;
+ int result;
} xfs_alloc_arg_t;

/*
--
1.7.5.1
Christoph Hellwig
2011-07-18 16:00:46 UTC
Permalink
Post by Dave Chinner
We currently have significant issues with the amount of stack that
allocation in XFS uses, especially in the writeback path. We can
easily consume 4k of stack between mapping the page, manipulating
the bmap btree and allocating blocks from the free list. Not to
mention btree block readahead and other functionality that issues IO
in the allocation path.
As a result, we can no longer fit allocation in the writeback path
in the stack space provided on x86_64. To alleviate this problem,
introduce an allocation workqueue and move all allocations to a
seperate context. This can be easily added as an interposing layer
into xfs_alloc_vextent(), which takes a single argument structure
and does not return until the allocation is complete or has failed.
I've mentioned before that I really don't like it, but I suspect there's
not much of an way around it giving the small stacks, and significant
amount of stacks that's already used above and below XFS.

Can we at least have a sysctl nob or mount option to switch back to
direct allocator calls so that we can still debug any performance
or other issues with this one?
Dave Chinner
2011-07-19 01:24:50 UTC
Permalink
Post by Christoph Hellwig
Post by Dave Chinner
We currently have significant issues with the amount of stack that
allocation in XFS uses, especially in the writeback path. We can
easily consume 4k of stack between mapping the page, manipulating
the bmap btree and allocating blocks from the free list. Not to
mention btree block readahead and other functionality that issues IO
in the allocation path.
As a result, we can no longer fit allocation in the writeback path
in the stack space provided on x86_64. To alleviate this problem,
introduce an allocation workqueue and move all allocations to a
seperate context. This can be easily added as an interposing layer
into xfs_alloc_vextent(), which takes a single argument structure
and does not return until the allocation is complete or has failed.
I've mentioned before that I really don't like it, but I suspect there's
not much of an way around it giving the small stacks, and significant
amount of stacks that's already used above and below XFS.
Can we at least have a sysctl nob or mount option to switch back to
direct allocator calls so that we can still debug any performance
or other issues with this one?
Honestly, I'd prefer not to do that because it's a slippery slope.
I've got plenty more "do stuff in the background via workqueues"
patches lined up, so if we start adding knobs/mount options to turn
each of them off "just in case there's an issue".

So far I haven't found any issues at all and I've been running this
split allocation stack like this in -all- my performance testing for
the past 2-3 months. I know that is not conclusive, but if the
bechmarks I've been using to improve XFS performance over the past
18 months don't show regressions, that's fairly indicative of the
fact that most workloads won't even notice the change....

Cheers,

Dave.
--
Dave Chinner
***@fromorbit.com
Christoph Hellwig
2011-07-19 02:02:14 UTC
Permalink
Post by Dave Chinner
Honestly, I'd prefer not to do that because it's a slippery slope.
I've got plenty more "do stuff in the background via workqueues"
patches lined up, so if we start adding knobs/mount options to turn
each of them off "just in case there's an issue".
So far I haven't found any issues at all and I've been running this
split allocation stack like this in -all- my performance testing for
the past 2-3 months. I know that is not conclusive, but if the
bechmarks I've been using to improve XFS performance over the past
18 months don't show regressions, that's fairly indicative of the
fact that most workloads won't even notice the change....
Maybe. One thing I'd like to see is stuff like high-iop direct or
O_SYNC I/O that actually calls the allocator.
Dave Chinner
2011-07-19 03:14:29 UTC
Permalink
Post by Christoph Hellwig
Post by Dave Chinner
Honestly, I'd prefer not to do that because it's a slippery slope.
I've got plenty more "do stuff in the background via workqueues"
patches lined up, so if we start adding knobs/mount options to turn
each of them off "just in case there's an issue".
So far I haven't found any issues at all and I've been running this
split allocation stack like this in -all- my performance testing for
the past 2-3 months. I know that is not conclusive, but if the
bechmarks I've been using to improve XFS performance over the past
18 months don't show regressions, that's fairly indicative of the
fact that most workloads won't even notice the change....
Maybe. One thing I'd like to see is stuff like high-iop direct or
O_SYNC I/O that actually calls the allocator.
What do you want me to run? I don't have any particularly high IOP
hardware here, but I might be able to do something that just hits
the BBWC....

Cheers,

Dave.
--
Dave Chinner
***@fromorbit.com
Dave Chinner
2011-07-18 03:49:49 UTC
Permalink
From: Dave Chinner <***@redhat.com>

Doing background CIL flushes adds significant latency to whatever
async transaction that triggers it. To avoid blocking async
transactions on things like waiting for log buffer IO to complete,
move the CIL push off into a workqueue. By moving the push work
into a workqueue, we remove all the latency that the commit adds
from the foreground transaction commit path. This also means that
single threaded workloads won't do the CIL push procssing, leaving
them more CPU to do more async transactions.

To do this, we need to keep track of the sequence nnumber we have
pushed work for. This avoids having many transaction commits
attempting to schedule work for the same sequence, and ensures that
we only ever have one push (background or forced) in progress at a
time. It also means that we don't need to take the CIL lock in write
mode to check for potential background push races, which reduces
lock contention.

Signed-off-by: Dave Chinner <***@redhat.com>
---
fs/xfs/linux-2.6/xfs_super.c | 7 +
fs/xfs/xfs_log_cil.c | 260 +++++++++++++++++++++++++-----------------
fs/xfs/xfs_log_priv.h | 4 +
3 files changed, 166 insertions(+), 105 deletions(-)

diff --git a/fs/xfs/linux-2.6/xfs_super.c b/fs/xfs/linux-2.6/xfs_super.c
index 6a6d4d9..b3ace86 100644
--- a/fs/xfs/linux-2.6/xfs_super.c
+++ b/fs/xfs/linux-2.6/xfs_super.c
@@ -1683,8 +1683,14 @@ xfs_init_workqueues(void)
if (!xfs_alloc_wq)
goto out_destroy_ail;

+ xfs_cil_wq = alloc_workqueue("xfscil", WQ_MEM_RECLAIM, 8);
+ if (!xfs_ail_wq)
+ goto out_destroy_alloc;
+
return 0;

+out_destroy_alloc:
+ destroy_workqueue(xfs_alloc_wq);
out_destroy_ail:
destroy_workqueue(xfs_ail_wq);
out_destroy_syncd:
@@ -1696,6 +1702,7 @@ out:
STATIC void
xfs_destroy_workqueues(void)
{
+ destroy_workqueue(xfs_cil_wq);
destroy_workqueue(xfs_ail_wq);
destroy_workqueue(xfs_syncd_wq);
}
diff --git a/fs/xfs/xfs_log_cil.c b/fs/xfs/xfs_log_cil.c
index c7755d5..9e652d2 100644
--- a/fs/xfs/xfs_log_cil.c
+++ b/fs/xfs/xfs_log_cil.c
@@ -31,67 +31,7 @@
#include "xfs_alloc.h"
#include "xfs_discard.h"

-/*
- * Perform initial CIL structure initialisation. If the CIL is not
- * enabled in this filesystem, ensure the log->l_cilp is null so
- * we can check this conditional to determine if we are doing delayed
- * logging or not.
- */
-int
-xlog_cil_init(
- struct log *log)
-{
- struct xfs_cil *cil;
- struct xfs_cil_ctx *ctx;
-
- log->l_cilp = NULL;
- if (!(log->l_mp->m_flags & XFS_MOUNT_DELAYLOG))
- return 0;
-
- cil = kmem_zalloc(sizeof(*cil), KM_SLEEP|KM_MAYFAIL);
- if (!cil)
- return ENOMEM;
-
- ctx = kmem_zalloc(sizeof(*ctx), KM_SLEEP|KM_MAYFAIL);
- if (!ctx) {
- kmem_free(cil);
- return ENOMEM;
- }
-
- INIT_LIST_HEAD(&cil->xc_cil);
- INIT_LIST_HEAD(&cil->xc_committing);
- spin_lock_init(&cil->xc_cil_lock);
- init_rwsem(&cil->xc_ctx_lock);
- 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;
- return 0;
-}
-
-void
-xlog_cil_destroy(
- struct log *log)
-{
- if (!log->l_cilp)
- return;
-
- if (log->l_cilp->xc_ctx) {
- if (log->l_cilp->xc_ctx->ticket)
- xfs_log_ticket_put(log->l_cilp->xc_ctx->ticket);
- kmem_free(log->l_cilp->xc_ctx);
- }
-
- ASSERT(list_empty(&log->l_cilp->xc_cil));
- kmem_free(log->l_cilp);
-}
+struct workqueue_struct *xfs_cil_wq;

/*
* Allocate a new ticket. Failing to get a new ticket makes it really hard to
@@ -401,12 +341,58 @@ xlog_cil_committed(
* get a race between multiple pushes for the same sequence they will block on
* the first one and then abort, hence avoiding needless pushes.
*/
-STATIC int
+static void
xlog_cil_push(
- struct log *log,
- xfs_lsn_t push_seq)
+ struct log *log,
+ xfs_lsn_t push_seq)
{
- struct xfs_cil *cil = log->l_cilp;
+ struct xfs_cil *cil = log->l_cilp;
+
+ if (!cil)
+ return;
+
+ ASSERT(!push_seq || push_seq <= cil->xc_current_sequence);
+
+ /*
+ * don't do a background push if we haven't used up all the
+ * space available yet.
+ */
+ if (!push_seq && cil->xc_ctx->space_used < XLOG_CIL_SPACE_LIMIT(log))
+ return;
+
+ /*
+ * if we are being asked to push to a specific sequence, and we have
+ * already queued a larger push, then nothing to do.
+ */
+ if (push_seq && push_seq <= cil->xc_push_seq)
+ return;
+
+ spin_lock(&cil->xc_cil_lock);
+ if (!push_seq)
+ push_seq = cil->xc_current_sequence;
+
+ /*
+ * if the CIL is empty, or we've already pushed the sequence, then
+ * there's no work we need to do.
+ */
+ if (list_empty(&cil->xc_cil) || push_seq <= cil->xc_push_seq) {
+ spin_unlock(&cil->xc_cil_lock);
+ return;
+ }
+
+ cil->xc_push_seq = push_seq;
+ queue_work(xfs_cil_wq, &cil->xc_push_work);
+ spin_unlock(&cil->xc_cil_lock);
+
+}
+
+static void
+xlog_cil_push_work(
+ struct work_struct *work)
+{
+ struct xfs_cil *cil = container_of(work, struct xfs_cil,
+ xc_push_work);
+ struct log *log = cil->xc_log;
struct xfs_log_vec *lv;
struct xfs_cil_ctx *ctx;
struct xfs_cil_ctx *new_ctx;
@@ -419,40 +405,34 @@ xlog_cil_push(
struct xfs_trans_header thdr;
struct xfs_log_iovec lhdr;
struct xfs_log_vec lvhdr = { NULL };
+ xfs_lsn_t push_seq;
xfs_lsn_t commit_lsn;

- if (!cil)
- return 0;
-
- ASSERT(!push_seq || push_seq <= cil->xc_ctx->sequence);
-
new_ctx = kmem_zalloc(sizeof(*new_ctx), KM_SLEEP|KM_NOFS);
new_ctx->ticket = xlog_cil_ticket_alloc(log);

- /*
- * Lock out transaction commit, but don't block for background pushes
- * unless we are well over the CIL space limit. See the definition of
- * XLOG_CIL_HARD_SPACE_LIMIT() for the full explanation of the logic
- * used here.
- */
- if (!down_write_trylock(&cil->xc_ctx_lock)) {
- if (!push_seq &&
- cil->xc_ctx->space_used < XLOG_CIL_HARD_SPACE_LIMIT(log))
- goto out_free_ticket;
- down_write(&cil->xc_ctx_lock);
- }
+ /* Lock out transaction commiti until we've switch contexts */
+ down_write(&cil->xc_ctx_lock);
ctx = cil->xc_ctx;

- /* check if we've anything to push */
- if (list_empty(&cil->xc_cil))
- goto out_skip;
+ spin_lock(&cil->xc_cil_lock);
+ push_seq = cil->xc_push_seq;
+ ASSERT(push_seq > 0 && push_seq <= ctx->sequence);

- /* check for spurious background flush */
- if (!push_seq && cil->xc_ctx->space_used < XLOG_CIL_SPACE_LIMIT(log))
+ /*
+ * Check if we've anything to push. If there is nothing, then we don't
+ * move on to a new sequence number and so we have to be able to push
+ * this sequence again later.
+ */
+ if (list_empty(&cil->xc_cil)) {
+ cil->xc_push_seq = 0;
+ spin_unlock(&cil->xc_cil_lock);
goto out_skip;
+ }
+ spin_unlock(&cil->xc_cil_lock);

/* check for a previously pushed seqeunce */
- if (push_seq && push_seq < cil->xc_ctx->sequence)
+ if (push_seq < ctx->sequence)
goto out_skip;

/*
@@ -602,20 +582,19 @@ restart:
spin_unlock(&cil->xc_cil_lock);

/* release the hounds! */
- return xfs_log_release_iclog(log->l_mp, commit_iclog);
+ xfs_log_release_iclog(log->l_mp, commit_iclog);
+ return;

out_skip:
up_write(&cil->xc_ctx_lock);
-out_free_ticket:
xfs_log_ticket_put(new_ctx->ticket);
kmem_free(new_ctx);
- return 0;
+ return;

out_abort_free_ticket:
xfs_log_ticket_put(tic);
out_abort:
xlog_cil_committed(ctx, XFS_LI_ABORTED);
- return XFS_ERROR(EIO);
}

/*
@@ -645,7 +624,6 @@ xfs_log_commit_cil(
{
struct log *log = mp->m_log;
int log_flags = 0;
- int push = 0;

if (flags & XFS_TRANS_RELEASE_LOG_RES)
log_flags = XFS_LOG_REL_PERM_RESERV;
@@ -694,12 +672,6 @@ xfs_log_commit_cil(
*/
xfs_trans_free_items(tp, *commit_lsn, 0);

- /* check for background commit before unlock */
- if (log->l_cilp->xc_ctx->space_used > XLOG_CIL_SPACE_LIMIT(log))
- push = 1;
-
- up_read(&log->l_cilp->xc_ctx_lock);
-
/*
* We need to push CIL every so often so we don't cache more than we
* can fit in the log. The limit really is that a checkpoint can't be
@@ -707,8 +679,8 @@ xfs_log_commit_cil(
* overwrite the previous checkpoint), but commit latency and memory
* usage limit this to a smaller size in most cases.
*/
- if (push)
- xlog_cil_push(log, 0);
+ xlog_cil_push(log, 0);
+ up_read(&log->l_cilp->xc_ctx_lock);
}

/*
@@ -720,9 +692,6 @@ xfs_log_commit_cil(
*
* We return the current commit lsn to allow the callers to determine if a
* iclog flush is necessary following this call.
- *
- * XXX: Initially, just push the CIL unconditionally and return whatever
- * commit lsn is there. It'll be empty, so this is broken for now.
*/
xfs_lsn_t
xlog_cil_force_lsn(
@@ -733,6 +702,8 @@ xlog_cil_force_lsn(
struct xfs_cil_ctx *ctx;
xfs_lsn_t commit_lsn = NULLCOMMITLSN;

+ /* lock out background commit */
+ down_read(&log->l_cilp->xc_ctx_lock);
ASSERT(sequence <= cil->xc_current_sequence);

/*
@@ -740,8 +711,23 @@ xlog_cil_force_lsn(
* xlog_cil_push() handles racing pushes for the same sequence,
* so no need to deal with it here.
*/
- if (sequence == cil->xc_current_sequence)
+ if (sequence == cil->xc_current_sequence) {
xlog_cil_push(log, sequence);
+ up_read(&log->l_cilp->xc_ctx_lock);
+
+ /*
+ * We have to block waiting for the push to execute even if we
+ * didn't push the sequence out as we need to wait for the push
+ * to get queued into the committing list. Once it is in the
+ * committing list, we can harvest the commit_lsn of the
+ * checkpoint issued by the push.
+ *
+ * We don't hold the ctx lock while doing this as the push work
+ * needs to hold it.
+ */
+ flush_work_sync(&cil->xc_push_work);
+ } else
+ up_read(&log->l_cilp->xc_ctx_lock);

/*
* See if we can find a previous sequence still committing.
@@ -802,3 +788,67 @@ xfs_log_item_in_current_chkpt(
return false;
return true;
}
+
+/*
+ * Perform initial CIL structure initialisation. If the CIL is not
+ * enabled in this filesystem, ensure the log->l_cilp is null so
+ * we can check this conditional to determine if we are doing delayed
+ * logging or not.
+ */
+int
+xlog_cil_init(
+ struct log *log)
+{
+ struct xfs_cil *cil;
+ struct xfs_cil_ctx *ctx;
+
+ log->l_cilp = NULL;
+ if (!(log->l_mp->m_flags & XFS_MOUNT_DELAYLOG))
+ return 0;
+
+ cil = kmem_zalloc(sizeof(*cil), KM_SLEEP|KM_MAYFAIL);
+ if (!cil)
+ return ENOMEM;
+
+ ctx = kmem_zalloc(sizeof(*ctx), KM_SLEEP|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);
+ init_rwsem(&cil->xc_ctx_lock);
+ 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;
+ return 0;
+}
+
+void
+xlog_cil_destroy(
+ struct log *log)
+{
+ if (!log->l_cilp)
+ return;
+
+ if (log->l_cilp->xc_ctx) {
+ if (log->l_cilp->xc_ctx->ticket)
+ xfs_log_ticket_put(log->l_cilp->xc_ctx->ticket);
+ kmem_free(log->l_cilp->xc_ctx);
+ }
+
+ ASSERT(list_empty(&log->l_cilp->xc_cil));
+ kmem_free(log->l_cilp);
+}
+
diff --git a/fs/xfs/xfs_log_priv.h b/fs/xfs/xfs_log_priv.h
index 2d3b6a4..61d55f9 100644
--- a/fs/xfs/xfs_log_priv.h
+++ b/fs/xfs/xfs_log_priv.h
@@ -417,8 +417,12 @@ struct xfs_cil {
struct list_head xc_committing;
wait_queue_head_t xc_commit_wait;
xfs_lsn_t xc_current_sequence;
+ struct work_struct xc_push_work;
+ xfs_lsn_t xc_push_seq;
};

+extern struct workqueue_struct *xfs_cil_wq;
+
/*
* The amount of log space we allow the CIL to aggregate is difficult to size.
* Whatever we choose, we have to make sure we can get a reservation for the
--
1.7.5.1
Christoph Hellwig
2011-07-19 02:03:17 UTC
Permalink
Generally looks okay, but doing a context switch in every log force
might bite us. Less the general context switch overhead, but more
the nasty interactions with cfq, which are causing huge problems
for ext3/4, with no good way to fix them for workqueues.
Dave Chinner
2011-07-19 03:05:51 UTC
Permalink
Post by Christoph Hellwig
Generally looks okay, but doing a context switch in every log force
might bite us. Less the general context switch overhead, but more
the nasty interactions with cfq, which are causing huge problems
for ext3/4,
Quite frankly, I don't recommend CFQ unless you need block level
throttling or use IO prioritisation seriously. CFQ is way too smart
for it's own good trying to do everything for everyone, and as such
suffers from different regressions every release. It has weird
workload specific heuristics in it to try to address issues that
don't solve the general class of problem, and so is always being
patched to fix the next occurrence of the same problem. e.g. the IO
stalls caused by dependent IOs being issued by different threads
that ext3/4 fsync hits all the time.
Post by Christoph Hellwig
with no good way to fix them for workqueues.
Right, which I pointed out to them last time round of ext4 specific
hacks that tried to tell the journal thread that it's IO had
external depenencies.

And let's face it - every time we move IO into a workqueue, we
introduce new cases of IO dependencies between threads. e.g.
anything waiting on a log force in progress is already dependent on
dispatch from a different thread, so the xfssyncd xfsaild and busy
extent log forces all will suffer to some extent from CFQ's existing
deficiencies in this regard. Moving the log IO into a workqueue
doesn't change this at all....

I'm of the opinion that anyone with a RAID controller with a BBWC
doesn't need the smarts in CFQ because the BBWC provides a much
larger and smarter IO re-order window than the Linux IO schedulers
and hence do a better job of IO scheduling than Linux can ever do.
We shouldn't penalise the target market for XFS for having fast
storage by catering to difficiencies of IO schedulers that are
mostly redundant for the hardware XFS typically runs on....

Cheers,

Dave.
--
Dave Chinner
***@fromorbit.com
Christoph Hellwig
2011-07-19 03:11:47 UTC
Permalink
Post by Dave Chinner
Post by Christoph Hellwig
Generally looks okay, but doing a context switch in every log force
might bite us. Less the general context switch overhead, but more
the nasty interactions with cfq, which are causing huge problems
for ext3/4,
Quite frankly, I don't recommend CFQ unless you need block level
throttling or use IO prioritisation seriously. CFQ is way too smart
for it's own good trying to do everything for everyone, and as such
suffers from different regressions every release. It has weird
workload specific heuristics in it to try to address issues that
don't solve the general class of problem, and so is always being
patched to fix the next occurrence of the same problem. e.g. the IO
stalls caused by dependent IOs being issued by different threads
that ext3/4 fsync hits all the time.
I don't like CFQ very much either. But it's the default for both
mainline Linux and all major distros, so screwing it means a major
support burden as well as losing all kinds of benchmarks.
Post by Dave Chinner
I'm of the opinion that anyone with a RAID controller with a BBWC
doesn't need the smarts in CFQ because the BBWC provides a much
larger and smarter IO re-order window than the Linux IO schedulers
and hence do a better job of IO scheduling than Linux can ever do.
We shouldn't penalise the target market for XFS for having fast
storage by catering to difficiencies of IO schedulers that are
mostly redundant for the hardware XFS typically runs on....
What penatlity do we get for doing the cil force in line from log
force and only doing it in the background when it needs to be
written because of filling up the buffers?
Dave Chinner
2011-07-19 04:28:45 UTC
Permalink
Post by Christoph Hellwig
On Mon, Jul 18, 2011 at 10:03:17PM -0400, Christoph Hellwig
Post by Christoph Hellwig
Generally looks okay, but doing a context switch in every log
force might bite us. Less the general context switch
overhead, but more the nasty interactions with cfq, which are
causing huge problems for ext3/4,
Quite frankly, I don't recommend CFQ unless you need block level
throttling or use IO prioritisation seriously. CFQ is way too
smart for it's own good trying to do everything for everyone,
and as such suffers from different regressions every release.
It has weird workload specific heuristics in it to try to
address issues that don't solve the general class of problem,
and so is always being patched to fix the next occurrence of the
same problem. e.g. the IO stalls caused by dependent IOs being
issued by different threads that ext3/4 fsync hits all the time.
I don't like CFQ very much either. But it's the default for both
mainline Linux and all major distros, so screwing it means a major
support burden
We never tuned for AS or really cared how it performed when it was
the kernel and major distro default, either. The answer was always
"don't use AS if you care about performance". That's the same advice
major distro's give to their users of XFS w.r.t CFQ, anyway...
Post by Christoph Hellwig
as well as losing all kinds of benchmarks.
Do we really care about benchmarketing? I don't really...
Post by Christoph Hellwig
I'm of the opinion that anyone with a RAID controller with a BBWC
doesn't need the smarts in CFQ because the BBWC provides a much
larger and smarter IO re-order window than the Linux IO schedulers
and hence do a better job of IO scheduling than Linux can ever do.
We shouldn't penalise the target market for XFS for having fast
storage by catering to difficiencies of IO schedulers that are
mostly redundant for the hardware XFS typically runs on....
What penatlity do we get for doing the cil force in line from log
force and only doing it in the background when it needs to be
written because of filling up the buffers?
I can make the log force code do the push in line, it just
complicates things a little with the need for wrapper functions to
handle the different calling conventions. The log force has to wait
on the workqueue anyway (and will still have to do so even if it
pushes directly itself), so doing the push work directly won't
change the performance there at all.

It's really the background push that I want out of line, so I'll
rework it such that only the background push uses the workqueue.
That should alleviate most of the concerns with fsync+CFQ.

Cheers,

Dave.
--
Dave Chinner
***@fromorbit.com
Christoph Hellwig
2011-07-18 16:01:29 UTC
Permalink
Post by Dave Chinner
control flushing. While there, group all the delayed write list and
buffer handling functions into the same section of code to make it
easier to find all the relevant code.
Any chance you could make the re-grouping a separate patch? It's
really hard to read it right now as the actual changes are hidden too
much.
Dave Chinner
2011-07-19 01:25:07 UTC
Permalink
Post by Christoph Hellwig
Post by Dave Chinner
control flushing. While there, group all the delayed write list and
buffer handling functions into the same section of code to make it
easier to find all the relevant code.
Any chance you could make the re-grouping a separate patch? It's
really hard to read it right now as the actual changes are hidden too
much.
OK.

Cheers,

Dave.
--
Dave Chinner
***@fromorbit.com
Loading...