All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCHSET] backing-dev: replace private thread pool with workqueue
@ 2010-09-06 12:46 Tejun Heo
  2010-09-06 12:46 ` [PATCH 1/5] workqueue: implement workqueue_on_rescuer() Tejun Heo
                   ` (5 more replies)
  0 siblings, 6 replies; 15+ messages in thread
From: Tejun Heo @ 2010-09-06 12:46 UTC (permalink / raw)
  To: jaxboe, linux-kernel, hch

Hello,

This patchset converts backing-dev to use workqueue instead of the
private thread pool.  The conversion is pretty straight forward.  The
only notable thing is that the writeback work skips usual
bdi->work_list processing if it's being executed from the rescuer
thread.  Being executed from the rescuer thread means that the system
is under memory pressure and it just starts writeouts by calling
bdi_flush_io() and reschedules itself if necessary.  This effectly is
the same to what bdi_forker_thread() did when it failed to create a
thread for a pending BDI.

This patch contains the following five patches.

 0001-workqueue-implement-workqueue_on_rescuer.patch
 0002-backing-dev-kill-unused-bdi_writeback-nr.patch
 0003-backing-dev-replace-private-thread-pool-with-workque.patch
 0004-backing-dev-update-trace-points.patch
 0005-backing-dev-replace-sync_supers_tsk-timer-with-a-del.patch

It's currently based on v2.6.36-rc3 + wq#for-linus.  Once Linus pulls
from wq#for-linus, I'll rebase these, but the patches apply on
v2.6.36-rc3 too.  These patches are also available in the following
git tree,

 git://git.kernel.org/pub/scm/linux/kernel/git/tj/misc.git wq-wb

and contains the following changes.

 fs/fs-writeback.c                |   91 +----------
 include/linux/backing-dev.h      |   18 --
 include/linux/workqueue.h        |    1 
 include/trace/events/writeback.h |    9 -
 kernel/workqueue.c               |   15 +
 mm/backing-dev.c                 |  302 ++++++++++-----------------------------
 6 files changed, 111 insertions(+), 325 deletions(-)

Thanks.

--
tejun

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

* [PATCH 1/5] workqueue: implement workqueue_on_rescuer()
  2010-09-06 12:46 [PATCHSET] backing-dev: replace private thread pool with workqueue Tejun Heo
@ 2010-09-06 12:46 ` Tejun Heo
  2010-09-06 12:46 ` [PATCH 2/5] backing-dev: kill unused bdi_writeback->nr Tejun Heo
                   ` (4 subsequent siblings)
  5 siblings, 0 replies; 15+ messages in thread
From: Tejun Heo @ 2010-09-06 12:46 UTC (permalink / raw)
  To: jaxboe, linux-kernel, hch; +Cc: Tejun Heo

workqueue_on_rescuer() tests whether the caller is running on the
rescuer of the specified workqueue.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 include/linux/workqueue.h |    1 +
 kernel/workqueue.c        |   15 +++++++++++++++
 2 files changed, 16 insertions(+), 0 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index f11100f..c988063 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -339,6 +339,7 @@ extern int cancel_work_sync(struct work_struct *work);
 extern void workqueue_set_max_active(struct workqueue_struct *wq,
 				     int max_active);
 extern bool workqueue_congested(unsigned int cpu, struct workqueue_struct *wq);
+extern bool workqueue_on_rescuer(struct workqueue_struct *wq);
 extern unsigned int work_cpu(struct work_struct *work);
 extern unsigned int work_busy(struct work_struct *work);
 
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 727f24e..a91e1ca 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -2941,6 +2941,21 @@ bool workqueue_congested(unsigned int cpu, struct workqueue_struct *wq)
 EXPORT_SYMBOL_GPL(workqueue_congested);
 
 /**
+ * workqueue_on_rescuer - test whether executing on the rescuer
+ * @wq: target workqueue
+ *
+ * Test whether the caller is currently executing on @wq's rescuer.
+ *
+ * RETURNS:
+ * %true if executing on @wq's rescuer, %false otherwise.
+ */
+bool workqueue_on_rescuer(struct workqueue_struct *wq)
+{
+	return wq->rescuer && current == wq->rescuer->task;
+}
+EXPORT_SYMBOL_GPL(workqueue_on_rescuer);
+
+/**
  * work_cpu - return the last known associated cpu for @work
  * @work: the work of interest
  *
-- 
1.7.1


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

* [PATCH 2/5] backing-dev: kill unused bdi_writeback->nr
  2010-09-06 12:46 [PATCHSET] backing-dev: replace private thread pool with workqueue Tejun Heo
  2010-09-06 12:46 ` [PATCH 1/5] workqueue: implement workqueue_on_rescuer() Tejun Heo
@ 2010-09-06 12:46 ` Tejun Heo
  2010-09-08  8:57   ` Artem Bityutskiy
  2010-09-06 12:46 ` [PATCH 3/5] backing-dev: replace private thread pool with workqueue Tejun Heo
                   ` (3 subsequent siblings)
  5 siblings, 1 reply; 15+ messages in thread
From: Tejun Heo @ 2010-09-06 12:46 UTC (permalink / raw)
  To: jaxboe, linux-kernel, hch; +Cc: Tejun Heo

bdi_writeback->nr is no longer used.  Kill it.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 include/linux/backing-dev.h |    1 -
 1 files changed, 0 insertions(+), 1 deletions(-)

diff --git a/include/linux/backing-dev.h b/include/linux/backing-dev.h
index 35b0074..97842ab 100644
--- a/include/linux/backing-dev.h
+++ b/include/linux/backing-dev.h
@@ -47,7 +47,6 @@ enum bdi_stat_item {
 
 struct bdi_writeback {
 	struct backing_dev_info *bdi;	/* our parent bdi */
-	unsigned int nr;
 
 	unsigned long last_old_flush;	/* last old data flush */
 	unsigned long last_active;	/* last time bdi thread was active */
-- 
1.7.1


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

* [PATCH 3/5] backing-dev: replace private thread pool with workqueue
  2010-09-06 12:46 [PATCHSET] backing-dev: replace private thread pool with workqueue Tejun Heo
  2010-09-06 12:46 ` [PATCH 1/5] workqueue: implement workqueue_on_rescuer() Tejun Heo
  2010-09-06 12:46 ` [PATCH 2/5] backing-dev: kill unused bdi_writeback->nr Tejun Heo
@ 2010-09-06 12:46 ` Tejun Heo
  2010-09-06 12:46 ` [PATCH 4/5] backing-dev: update trace points Tejun Heo
                   ` (2 subsequent siblings)
  5 siblings, 0 replies; 15+ messages in thread
From: Tejun Heo @ 2010-09-06 12:46 UTC (permalink / raw)
  To: jaxboe, linux-kernel, hch; +Cc: Tejun Heo

bdi writeback has been using a private thread pool.  Now that
workqueue can provide flexible concurrency, drop the dedicated thread
pool and use workqueue instead.

bdi_writeback->task and ->wakeup_timer are replaced with ->work and
->timer.  A separate timer is used instead of delayed_work because
mixing immediate and delayed queueing doesn't work too well with
delayed_work.

bdi_wq is added to execute bdi_writeback->work.  It's an unbound
freezeable workqueue w/ a rescuer.  As the thread pool is now managed
by the workqueue code, the special forker thread is no longer
necessary and removed along with bdi_forker_thread().

The work function for bdi_writeback->work is bdi_work_fn() and mostly
equivalent to a single iteration of bdi_writeback_thread() - IOW, it
calls wb_do_writeback() until bdi->work_list is exhausted and
reschedules itself if there is dirty data to write out.

One special provision is that the writeback work doesn't run
bdi->worklist if it's being executed from the rescuer.  Instead, it
just calls bdi_flush_io() and reschedules itself if necessary.  This
basically is the same behavior as bdi_forker_thread()'s when it failed
to create a thread for a pending bdi.  Writebacks are issued but not
waited upon so that other bdi's have a chance to clean and free
memory.

Trace points which make sense without change are preserved.  The next
patch will fix up the rest.

bdi_wakeup_thread_delayed() is renamed to bdi_delayed_writeback().

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 fs/fs-writeback.c           |   91 ++--------------
 include/linux/backing-dev.h |   17 +--
 mm/backing-dev.c            |  249 ++++++++++---------------------------------
 3 files changed, 72 insertions(+), 285 deletions(-)

diff --git a/fs/fs-writeback.c b/fs/fs-writeback.c
index 7d9d06b..1e9b807 100644
--- a/fs/fs-writeback.c
+++ b/fs/fs-writeback.c
@@ -78,16 +78,7 @@ static void bdi_queue_work(struct backing_dev_info *bdi,
 
 	spin_lock_bh(&bdi->wb_lock);
 	list_add_tail(&work->list, &bdi->work_list);
-	if (bdi->wb.task) {
-		wake_up_process(bdi->wb.task);
-	} else {
-		/*
-		 * The bdi thread isn't there, wake up the forker thread which
-		 * will create and run it.
-		 */
-		trace_writeback_nothread(bdi, work);
-		wake_up_process(default_backing_dev_info.wb.task);
-	}
+	queue_work(bdi_wq, &bdi->wb.work);
 	spin_unlock_bh(&bdi->wb_lock);
 }
 
@@ -99,14 +90,12 @@ __bdi_start_writeback(struct backing_dev_info *bdi, long nr_pages,
 
 	/*
 	 * This is WB_SYNC_NONE writeback, so if allocation fails just
-	 * wakeup the thread for old dirty data writeback
+	 * queue the work for old dirty data writeback.
 	 */
 	work = kzalloc(sizeof(*work), GFP_ATOMIC);
 	if (!work) {
-		if (bdi->wb.task) {
-			trace_writeback_nowork(bdi);
-			wake_up_process(bdi->wb.task);
-		}
+		trace_writeback_nowork(bdi);
+		queue_work(bdi_wq, &bdi->wb.work);
 		return;
 	}
 
@@ -773,70 +762,6 @@ long wb_do_writeback(struct bdi_writeback *wb, int force_wait)
 }
 
 /*
- * Handle writeback of dirty data for the device backed by this bdi. Also
- * wakes up periodically and does kupdated style flushing.
- */
-int bdi_writeback_thread(void *data)
-{
-	struct bdi_writeback *wb = data;
-	struct backing_dev_info *bdi = wb->bdi;
-	long pages_written;
-
-	current->flags |= PF_FLUSHER | PF_SWAPWRITE;
-	set_freezable();
-	wb->last_active = jiffies;
-
-	/*
-	 * Our parent may run at a different priority, just set us to normal
-	 */
-	set_user_nice(current, 0);
-
-	trace_writeback_thread_start(bdi);
-
-	while (!kthread_should_stop()) {
-		/*
-		 * Remove own delayed wake-up timer, since we are already awake
-		 * and we'll take care of the preriodic write-back.
-		 */
-		del_timer(&wb->wakeup_timer);
-
-		pages_written = wb_do_writeback(wb, 0);
-
-		trace_writeback_pages_written(pages_written);
-
-		if (pages_written)
-			wb->last_active = jiffies;
-
-		set_current_state(TASK_INTERRUPTIBLE);
-		if (!list_empty(&bdi->work_list)) {
-			__set_current_state(TASK_RUNNING);
-			continue;
-		}
-
-		if (wb_has_dirty_io(wb) && dirty_writeback_interval)
-			schedule_timeout(msecs_to_jiffies(dirty_writeback_interval * 10));
-		else {
-			/*
-			 * We have nothing to do, so can go sleep without any
-			 * timeout and save power. When a work is queued or
-			 * something is made dirty - we will be woken up.
-			 */
-			schedule();
-		}
-
-		try_to_freeze();
-	}
-
-	/* Flush any work that raced with us exiting */
-	if (!list_empty(&bdi->work_list))
-		wb_do_writeback(wb, 1);
-
-	trace_writeback_thread_stop(bdi);
-	return 0;
-}
-
-
-/*
  * Start writeback of `nr_pages' pages.  If `nr_pages' is zero, write back
  * the whole world.
  */
@@ -911,7 +836,7 @@ void __mark_inode_dirty(struct inode *inode, int flags)
 {
 	struct super_block *sb = inode->i_sb;
 	struct backing_dev_info *bdi = NULL;
-	bool wakeup_bdi = false;
+	bool delayed_wb = false;
 
 	/*
 	 * Don't do this for I_DIRTY_PAGES - that doesn't actually
@@ -978,7 +903,7 @@ void __mark_inode_dirty(struct inode *inode, int flags)
 				 * write-back happens later.
 				 */
 				if (!wb_has_dirty_io(&bdi->wb))
-					wakeup_bdi = true;
+					delayed_wb = true;
 			}
 
 			inode->dirtied_when = jiffies;
@@ -988,8 +913,8 @@ void __mark_inode_dirty(struct inode *inode, int flags)
 out:
 	spin_unlock(&inode_lock);
 
-	if (wakeup_bdi)
-		bdi_wakeup_thread_delayed(bdi);
+	if (delayed_wb)
+		bdi_delayed_writeback(bdi);
 }
 EXPORT_SYMBOL(__mark_inode_dirty);
 
diff --git a/include/linux/backing-dev.h b/include/linux/backing-dev.h
index 97842ab..ebeed5b 100644
--- a/include/linux/backing-dev.h
+++ b/include/linux/backing-dev.h
@@ -13,8 +13,8 @@
 #include <linux/proportions.h>
 #include <linux/kernel.h>
 #include <linux/fs.h>
-#include <linux/sched.h>
 #include <linux/timer.h>
+#include <linux/workqueue.h>
 #include <linux/writeback.h>
 #include <asm/atomic.h>
 
@@ -26,7 +26,6 @@ struct dentry;
  * Bits in backing_dev_info.state
  */
 enum bdi_state {
-	BDI_pending,		/* On its way to being activated */
 	BDI_wb_alloc,		/* Default embedded wb allocated */
 	BDI_async_congested,	/* The async (write) queue is getting full */
 	BDI_sync_congested,	/* The sync queue is getting full */
@@ -49,10 +48,9 @@ struct bdi_writeback {
 	struct backing_dev_info *bdi;	/* our parent bdi */
 
 	unsigned long last_old_flush;	/* last old data flush */
-	unsigned long last_active;	/* last time bdi thread was active */
 
-	struct task_struct *task;	/* writeback thread */
-	struct timer_list wakeup_timer; /* used for delayed bdi thread wakeup */
+	struct work_struct work;	/* writeback work */
+	struct timer_list timer;	/* used for delayed bdi execution */
 	struct list_head b_dirty;	/* dirty inodes */
 	struct list_head b_io;		/* parked for writeback */
 	struct list_head b_more_io;	/* parked for more writeback */
@@ -103,13 +101,13 @@ void bdi_unregister(struct backing_dev_info *bdi);
 int bdi_setup_and_register(struct backing_dev_info *, char *, unsigned int);
 void bdi_start_writeback(struct backing_dev_info *bdi, long nr_pages);
 void bdi_start_background_writeback(struct backing_dev_info *bdi);
-int bdi_writeback_thread(void *data);
 int bdi_has_dirty_io(struct backing_dev_info *bdi);
 void bdi_arm_supers_timer(void);
-void bdi_wakeup_thread_delayed(struct backing_dev_info *bdi);
+void bdi_delayed_writeback(struct backing_dev_info *bdi);
 
 extern spinlock_t bdi_lock;
 extern struct list_head bdi_list;
+extern struct workqueue_struct *bdi_wq;
 
 static inline int wb_has_dirty_io(struct bdi_writeback *wb)
 {
@@ -308,11 +306,6 @@ static inline bool bdi_cap_swap_backed(struct backing_dev_info *bdi)
 	return bdi->capabilities & BDI_CAP_SWAP_BACKED;
 }
 
-static inline bool bdi_cap_flush_forker(struct backing_dev_info *bdi)
-{
-	return bdi == &default_backing_dev_info;
-}
-
 static inline bool mapping_cap_writeback_dirty(struct address_space *mapping)
 {
 	return bdi_cap_writeback_dirty(mapping->backing_dev_info);
diff --git a/mm/backing-dev.c b/mm/backing-dev.c
index eaa4a5b..e874916 100644
--- a/mm/backing-dev.c
+++ b/mm/backing-dev.c
@@ -43,6 +43,7 @@ static struct class *bdi_class;
 DEFINE_SPINLOCK(bdi_lock);
 LIST_HEAD(bdi_list);
 LIST_HEAD(bdi_pending_list);
+struct workqueue_struct *bdi_wq;
 
 static struct task_struct *sync_supers_tsk;
 static struct timer_list sync_supers_timer;
@@ -234,6 +235,10 @@ static int __init default_bdi_init(void)
 {
 	int err;
 
+	bdi_wq = alloc_workqueue("bdi",
+				 WQ_UNBOUND | WQ_FREEZEABLE | WQ_RESCUER, 0);
+	BUG_ON(!bdi_wq);
+
 	sync_supers_tsk = kthread_run(bdi_sync_supers, NULL, "sync_supers");
 	BUG_ON(IS_ERR(sync_supers_tsk));
 
@@ -305,24 +310,11 @@ static void sync_supers_timer_fn(unsigned long unused)
 	bdi_arm_supers_timer();
 }
 
-static void wakeup_timer_fn(unsigned long data)
+static void bdi_timer_fn(unsigned long data)
 {
 	struct backing_dev_info *bdi = (struct backing_dev_info *)data;
 
-	spin_lock_bh(&bdi->wb_lock);
-	if (bdi->wb.task) {
-		trace_writeback_wake_thread(bdi);
-		wake_up_process(bdi->wb.task);
-	} else {
-		/*
-		 * When bdi tasks are inactive for long time, they are killed.
-		 * In this case we have to wake-up the forker thread which
-		 * should create and run the bdi thread.
-		 */
-		trace_writeback_wake_forker_thread(bdi);
-		wake_up_process(default_backing_dev_info.wb.task);
-	}
-	spin_unlock_bh(&bdi->wb_lock);
+	queue_work(bdi_wq, &bdi->wb.work);
 }
 
 /*
@@ -336,165 +328,56 @@ static void wakeup_timer_fn(unsigned long data)
  * fast-path (used by '__mark_inode_dirty()'), so we save few context switches
  * by delaying the wake-up.
  */
-void bdi_wakeup_thread_delayed(struct backing_dev_info *bdi)
+void bdi_delayed_writeback(struct backing_dev_info *bdi)
 {
 	unsigned long timeout;
 
 	timeout = msecs_to_jiffies(dirty_writeback_interval * 10);
-	mod_timer(&bdi->wb.wakeup_timer, jiffies + timeout);
+	mod_timer(&bdi->wb.timer, jiffies + timeout);
 }
 
 /*
- * Calculate the longest interval (jiffies) bdi threads are allowed to be
- * inactive.
+ * Handles writeback of dirty data for the device backed by this bdi.
+ * Also schedules itself periodically for kupdated style flushing.
  */
-static unsigned long bdi_longest_inactive(void)
+static void bdi_work_fn(struct work_struct *work)
 {
-	unsigned long interval;
+	struct bdi_writeback *wb =
+		container_of(work, struct bdi_writeback, work);
+	struct backing_dev_info *bdi = wb->bdi;
+	long pages_written;
 
-	interval = msecs_to_jiffies(dirty_writeback_interval * 10);
-	return max(5UL * 60 * HZ, interval);
-}
+	WARN(!test_bit(BDI_registered, &bdi->state),
+	     "bdi %p/%s is not registered!\n", bdi, bdi->name);
 
-static int bdi_forker_thread(void *ptr)
-{
-	struct bdi_writeback *me = ptr;
+	/*
+	 * Remove own delayed timer, since we are already running and
+	 * we'll take care of the periodic write-back.
+	 */
+	del_timer(&wb->timer);
 
 	current->flags |= PF_FLUSHER | PF_SWAPWRITE;
-	set_freezable();
 
 	/*
-	 * Our parent may run at a different priority, just set us to normal
+	 * Writeback works might block and we don't want to hog the
+	 * rescuer.  If we are running off the rescuer, skip works,
+	 * fire off writebacks and yield to other bdi's.
 	 */
-	set_user_nice(current, 0);
-
-	for (;;) {
-		struct task_struct *task = NULL;
-		struct backing_dev_info *bdi;
-		enum {
-			NO_ACTION,   /* Nothing to do */
-			FORK_THREAD, /* Fork bdi thread */
-			KILL_THREAD, /* Kill inactive bdi thread */
-		} action = NO_ACTION;
-
-		/*
-		 * Temporary measure, we want to make sure we don't see
-		 * dirty data on the default backing_dev_info
-		 */
-		if (wb_has_dirty_io(me) || !list_empty(&me->bdi->work_list)) {
-			del_timer(&me->wakeup_timer);
-			wb_do_writeback(me, 0);
-		}
-
-		spin_lock_bh(&bdi_lock);
-		set_current_state(TASK_INTERRUPTIBLE);
-
-		list_for_each_entry(bdi, &bdi_list, bdi_list) {
-			bool have_dirty_io;
-
-			if (!bdi_cap_writeback_dirty(bdi) ||
-			     bdi_cap_flush_forker(bdi))
-				continue;
-
-			WARN(!test_bit(BDI_registered, &bdi->state),
-			     "bdi %p/%s is not registered!\n", bdi, bdi->name);
-
-			have_dirty_io = !list_empty(&bdi->work_list) ||
-					wb_has_dirty_io(&bdi->wb);
-
-			/*
-			 * If the bdi has work to do, but the thread does not
-			 * exist - create it.
-			 */
-			if (!bdi->wb.task && have_dirty_io) {
-				/*
-				 * Set the pending bit - if someone will try to
-				 * unregister this bdi - it'll wait on this bit.
-				 */
-				set_bit(BDI_pending, &bdi->state);
-				action = FORK_THREAD;
-				break;
-			}
-
-			spin_lock(&bdi->wb_lock);
-
-			/*
-			 * If there is no work to do and the bdi thread was
-			 * inactive long enough - kill it. The wb_lock is taken
-			 * to make sure no-one adds more work to this bdi and
-			 * wakes the bdi thread up.
-			 */
-			if (bdi->wb.task && !have_dirty_io &&
-			    time_after(jiffies, bdi->wb.last_active +
-						bdi_longest_inactive())) {
-				task = bdi->wb.task;
-				bdi->wb.task = NULL;
-				spin_unlock(&bdi->wb_lock);
-				set_bit(BDI_pending, &bdi->state);
-				action = KILL_THREAD;
-				break;
-			}
-			spin_unlock(&bdi->wb_lock);
-		}
-		spin_unlock_bh(&bdi_lock);
-
-		/* Keep working if default bdi still has things to do */
-		if (!list_empty(&me->bdi->work_list))
-			__set_current_state(TASK_RUNNING);
-
-		switch (action) {
-		case FORK_THREAD:
-			__set_current_state(TASK_RUNNING);
-			task = kthread_run(bdi_writeback_thread, &bdi->wb, "flush-%s",
-					   dev_name(bdi->dev));
-			if (IS_ERR(task)) {
-				/*
-				 * If thread creation fails, force writeout of
-				 * the bdi from the thread.
-				 */
-				bdi_flush_io(bdi);
-			} else {
-				/*
-				 * The spinlock makes sure we do not lose
-				 * wake-ups when racing with 'bdi_queue_work()'.
-				 */
-				spin_lock_bh(&bdi->wb_lock);
-				bdi->wb.task = task;
-				spin_unlock_bh(&bdi->wb_lock);
-			}
-			break;
-
-		case KILL_THREAD:
-			__set_current_state(TASK_RUNNING);
-			kthread_stop(task);
-			break;
-
-		case NO_ACTION:
-			if (!wb_has_dirty_io(me) || !dirty_writeback_interval)
-				/*
-				 * There are no dirty data. The only thing we
-				 * should now care about is checking for
-				 * inactive bdi threads and killing them. Thus,
-				 * let's sleep for longer time, save energy and
-				 * be friendly for battery-driven devices.
-				 */
-				schedule_timeout(bdi_longest_inactive());
-			else
-				schedule_timeout(msecs_to_jiffies(dirty_writeback_interval * 10));
-			try_to_freeze();
-			/* Back to the main loop */
-			continue;
-		}
-
-		/*
-		 * Clear pending bit and wakeup anybody waiting to tear us down.
-		 */
-		clear_bit(BDI_pending, &bdi->state);
-		smp_mb__after_clear_bit();
-		wake_up_bit(&bdi->state, BDI_pending);
+	if (likely(!workqueue_on_rescuer(bdi_wq))) {
+		do {
+			pages_written = wb_do_writeback(wb, 0);
+			trace_writeback_pages_written(pages_written);
+		} while (!list_empty(&bdi->work_list));
+	} else {
+		bdi_flush_io(bdi);
+		if (!list_empty(&bdi->work_list))
+			queue_work(bdi_wq, work);
 	}
 
-	return 0;
+	if (wb_has_dirty_io(wb) && dirty_writeback_interval)
+		bdi_delayed_writeback(bdi);
+
+	current->flags &= ~(PF_FLUSHER | PF_SWAPWRITE);
 }
 
 /*
@@ -526,20 +409,6 @@ int bdi_register(struct backing_dev_info *bdi, struct device *parent,
 
 	bdi->dev = dev;
 
-	/*
-	 * Just start the forker thread for our default backing_dev_info,
-	 * and add other bdi's to the list. They will get a thread created
-	 * on-demand when they need it.
-	 */
-	if (bdi_cap_flush_forker(bdi)) {
-		struct bdi_writeback *wb = &bdi->wb;
-
-		wb->task = kthread_run(bdi_forker_thread, wb, "bdi-%s",
-						dev_name(dev));
-		if (IS_ERR(wb->task))
-			return PTR_ERR(wb->task);
-	}
-
 	bdi_debug_register(bdi, dev_name(dev));
 	set_bit(BDI_registered, &bdi->state);
 
@@ -563,30 +432,25 @@ EXPORT_SYMBOL(bdi_register_dev);
  */
 static void bdi_wb_shutdown(struct backing_dev_info *bdi)
 {
+	unsigned int saved_flags = current->flags & (PF_FLUSHER | PF_SWAPWRITE);
+
 	if (!bdi_cap_writeback_dirty(bdi))
 		return;
 
 	/*
-	 * Make sure nobody finds us on the bdi_list anymore
+	 * Make sure nobody finds us on the bdi_list anymore and
+	 * writeback work isn't running.
 	 */
 	bdi_remove_from_list(bdi);
+	cancel_work_sync(&bdi->wb.work);
 
-	/*
-	 * If setup is pending, wait for that to complete first
-	 */
-	wait_on_bit(&bdi->state, BDI_pending, bdi_sched_wait,
-			TASK_UNINTERRUPTIBLE);
+	/* Flush any work that raced with shutdown */
+	current->flags |= PF_FLUSHER | PF_SWAPWRITE;
 
-	/*
-	 * Finally, kill the kernel thread. We don't need to be RCU
-	 * safe anymore, since the bdi is gone from visibility. Force
-	 * unfreeze of the thread before calling kthread_stop(), otherwise
-	 * it would never exet if it is currently stuck in the refrigerator.
-	 */
-	if (bdi->wb.task) {
-		thaw_process(bdi->wb.task);
-		kthread_stop(bdi->wb.task);
-	}
+	wb_do_writeback(&bdi->wb, 1);
+
+	current->flags &= ~(PF_FLUSHER | PF_SWAPWRITE);
+	current->flags |= saved_flags;
 }
 
 /*
@@ -609,10 +473,9 @@ void bdi_unregister(struct backing_dev_info *bdi)
 	if (bdi->dev) {
 		trace_writeback_bdi_unregister(bdi);
 		bdi_prune_sb(bdi);
-		del_timer_sync(&bdi->wb.wakeup_timer);
+		del_timer_sync(&bdi->wb.timer);
 
-		if (!bdi_cap_flush_forker(bdi))
-			bdi_wb_shutdown(bdi);
+		bdi_wb_shutdown(bdi);
 		bdi_debug_unregister(bdi);
 		device_unregister(bdi->dev);
 		bdi->dev = NULL;
@@ -629,7 +492,13 @@ static void bdi_wb_init(struct bdi_writeback *wb, struct backing_dev_info *bdi)
 	INIT_LIST_HEAD(&wb->b_dirty);
 	INIT_LIST_HEAD(&wb->b_io);
 	INIT_LIST_HEAD(&wb->b_more_io);
-	setup_timer(&wb->wakeup_timer, wakeup_timer_fn, (unsigned long)bdi);
+	/*
+	 * work and separate timer are used instead of delayed_work
+	 * because mixing immediate and delayed queueing doesn't work
+	 * too well with delayed_work.
+	 */
+	INIT_WORK(&wb->work, bdi_work_fn);
+	setup_timer(&wb->timer, bdi_timer_fn, (unsigned long)bdi);
 }
 
 int bdi_init(struct backing_dev_info *bdi)
-- 
1.7.1


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

* [PATCH 4/5] backing-dev: update trace points
  2010-09-06 12:46 [PATCHSET] backing-dev: replace private thread pool with workqueue Tejun Heo
                   ` (2 preceding siblings ...)
  2010-09-06 12:46 ` [PATCH 3/5] backing-dev: replace private thread pool with workqueue Tejun Heo
@ 2010-09-06 12:46 ` Tejun Heo
  2010-09-06 12:46 ` [PATCH 5/5] backing-dev: replace sync_supers_tsk/timer with a delayed_work Tejun Heo
  2010-09-07 12:16 ` [PATCHSET] backing-dev: replace private thread pool with workqueue Christoph Hellwig
  5 siblings, 0 replies; 15+ messages in thread
From: Tejun Heo @ 2010-09-06 12:46 UTC (permalink / raw)
  To: jaxboe, linux-kernel, hch; +Cc: Tejun Heo

* writeback_nothread is replaced with writeback_rescuer, which is
  triggered when a writeback work is called from the bdi_wq rescuer.

* writeback_wake[_forker]_thread are replaced with writeback_timer.

* writeback_thread_start/stop are replaced with writeback_start/stop
  which are triggered when a writeback work starts and finishes
  execution.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 include/trace/events/writeback.h |    9 ++++-----
 mm/backing-dev.c                 |    6 ++++++
 2 files changed, 10 insertions(+), 5 deletions(-)

diff --git a/include/trace/events/writeback.h b/include/trace/events/writeback.h
index f345f66..58e1f01 100644
--- a/include/trace/events/writeback.h
+++ b/include/trace/events/writeback.h
@@ -46,7 +46,7 @@ DECLARE_EVENT_CLASS(writeback_work_class,
 DEFINE_EVENT(writeback_work_class, name, \
 	TP_PROTO(struct backing_dev_info *bdi, struct wb_writeback_work *work), \
 	TP_ARGS(bdi, work))
-DEFINE_WRITEBACK_WORK_EVENT(writeback_nothread);
+DEFINE_WRITEBACK_WORK_EVENT(writeback_rescuer);
 DEFINE_WRITEBACK_WORK_EVENT(writeback_queue);
 DEFINE_WRITEBACK_WORK_EVENT(writeback_exec);
 
@@ -81,12 +81,11 @@ DEFINE_EVENT(writeback_class, name, \
 	TP_ARGS(bdi))
 
 DEFINE_WRITEBACK_EVENT(writeback_nowork);
-DEFINE_WRITEBACK_EVENT(writeback_wake_thread);
-DEFINE_WRITEBACK_EVENT(writeback_wake_forker_thread);
+DEFINE_WRITEBACK_EVENT(writeback_timer);
 DEFINE_WRITEBACK_EVENT(writeback_bdi_register);
 DEFINE_WRITEBACK_EVENT(writeback_bdi_unregister);
-DEFINE_WRITEBACK_EVENT(writeback_thread_start);
-DEFINE_WRITEBACK_EVENT(writeback_thread_stop);
+DEFINE_WRITEBACK_EVENT(writeback_start);
+DEFINE_WRITEBACK_EVENT(writeback_stop);
 
 DECLARE_EVENT_CLASS(wbc_class,
 	TP_PROTO(struct writeback_control *wbc, struct backing_dev_info *bdi),
diff --git a/mm/backing-dev.c b/mm/backing-dev.c
index e874916..3b2a657 100644
--- a/mm/backing-dev.c
+++ b/mm/backing-dev.c
@@ -314,6 +314,7 @@ static void bdi_timer_fn(unsigned long data)
 {
 	struct backing_dev_info *bdi = (struct backing_dev_info *)data;
 
+	trace_writeback_timer(bdi);
 	queue_work(bdi_wq, &bdi->wb.work);
 }
 
@@ -347,6 +348,8 @@ static void bdi_work_fn(struct work_struct *work)
 	struct backing_dev_info *bdi = wb->bdi;
 	long pages_written;
 
+	trace_writeback_start(bdi);
+
 	WARN(!test_bit(BDI_registered, &bdi->state),
 	     "bdi %p/%s is not registered!\n", bdi, bdi->name);
 
@@ -369,6 +372,7 @@ static void bdi_work_fn(struct work_struct *work)
 			trace_writeback_pages_written(pages_written);
 		} while (!list_empty(&bdi->work_list));
 	} else {
+		trace_writeback_rescuer(bdi, work);
 		bdi_flush_io(bdi);
 		if (!list_empty(&bdi->work_list))
 			queue_work(bdi_wq, work);
@@ -378,6 +382,8 @@ static void bdi_work_fn(struct work_struct *work)
 		bdi_delayed_writeback(bdi);
 
 	current->flags &= ~(PF_FLUSHER | PF_SWAPWRITE);
+
+	trace_writeback_stop(bdi);
 }
 
 /*
-- 
1.7.1


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

* [PATCH 5/5] backing-dev: replace sync_supers_tsk/timer with a delayed_work
  2010-09-06 12:46 [PATCHSET] backing-dev: replace private thread pool with workqueue Tejun Heo
                   ` (3 preceding siblings ...)
  2010-09-06 12:46 ` [PATCH 4/5] backing-dev: update trace points Tejun Heo
@ 2010-09-06 12:46 ` Tejun Heo
  2010-09-08  9:00   ` Artem Bityutskiy
  2010-09-07 12:16 ` [PATCHSET] backing-dev: replace private thread pool with workqueue Christoph Hellwig
  5 siblings, 1 reply; 15+ messages in thread
From: Tejun Heo @ 2010-09-06 12:46 UTC (permalink / raw)
  To: jaxboe, linux-kernel, hch; +Cc: Tejun Heo

sync_supers_task/timer can trivially replaced with a delayed_work.  As
it's not on the allocation path, run it on the system_wq.

NOTE: Both before and after this patch, sync_supers is not freezeable.
      This might need to be changed.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 mm/backing-dev.c |   49 ++++++++++++++-----------------------------------
 1 files changed, 14 insertions(+), 35 deletions(-)

diff --git a/mm/backing-dev.c b/mm/backing-dev.c
index 3b2a657..4fb8095 100644
--- a/mm/backing-dev.c
+++ b/mm/backing-dev.c
@@ -45,11 +45,8 @@ LIST_HEAD(bdi_list);
 LIST_HEAD(bdi_pending_list);
 struct workqueue_struct *bdi_wq;
 
-static struct task_struct *sync_supers_tsk;
-static struct timer_list sync_supers_timer;
-
-static int bdi_sync_supers(void *);
-static void sync_supers_timer_fn(unsigned long);
+static void bdi_sync_supers_work_fn(struct work_struct *work);
+static DECLARE_DELAYED_WORK(bdi_sync_supers_work, bdi_sync_supers_work_fn);
 
 #ifdef CONFIG_DEBUG_FS
 #include <linux/debugfs.h>
@@ -239,10 +236,6 @@ static int __init default_bdi_init(void)
 				 WQ_UNBOUND | WQ_FREEZEABLE | WQ_RESCUER, 0);
 	BUG_ON(!bdi_wq);
 
-	sync_supers_tsk = kthread_run(bdi_sync_supers, NULL, "sync_supers");
-	BUG_ON(IS_ERR(sync_supers_tsk));
-
-	setup_timer(&sync_supers_timer, sync_supers_timer_fn, 0);
 	bdi_arm_supers_timer();
 
 	err = bdi_init(&default_backing_dev_info);
@@ -271,43 +264,29 @@ static void bdi_flush_io(struct backing_dev_info *bdi)
 }
 
 /*
- * kupdated() used to do this. We cannot do it from the bdi_forker_thread()
- * or we risk deadlocking on ->s_umount. The longer term solution would be
- * to implement sync_supers_bdi() or similar and simply do it from the
- * bdi writeback thread individually.
+ * kupdated() used to do this.  Don't do it from bdi_wq as we might
+ * deadlock on ->s_umount.
  */
-static int bdi_sync_supers(void *unused)
+static void bdi_sync_supers_work_fn(struct work_struct *work)
 {
-	set_user_nice(current, 0);
-
-	while (!kthread_should_stop()) {
-		set_current_state(TASK_INTERRUPTIBLE);
-		schedule();
-
-		/*
-		 * Do this periodically, like kupdated() did before.
-		 */
-		sync_supers();
-	}
+	/* Do this periodically, like kupdated() did before */
+	sync_supers();
 
-	return 0;
+	if (dirty_writeback_interval)
+		schedule_delayed_work(&bdi_sync_supers_work,
+			msecs_to_jiffies(dirty_writeback_interval * 10));
 }
 
 void bdi_arm_supers_timer(void)
 {
-	unsigned long next;
+	/* Interval might have decreased, cancel delayed work first */
+	cancel_delayed_work(&bdi_sync_supers_work);
 
 	if (!dirty_writeback_interval)
 		return;
 
-	next = msecs_to_jiffies(dirty_writeback_interval * 10) + jiffies;
-	mod_timer(&sync_supers_timer, round_jiffies_up(next));
-}
-
-static void sync_supers_timer_fn(unsigned long unused)
-{
-	wake_up_process(sync_supers_tsk);
-	bdi_arm_supers_timer();
+	schedule_delayed_work(&bdi_sync_supers_work,
+			      msecs_to_jiffies(dirty_writeback_interval * 10));
 }
 
 static void bdi_timer_fn(unsigned long data)
-- 
1.7.1


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

* Re: [PATCHSET] backing-dev: replace private thread pool with workqueue
  2010-09-06 12:46 [PATCHSET] backing-dev: replace private thread pool with workqueue Tejun Heo
                   ` (4 preceding siblings ...)
  2010-09-06 12:46 ` [PATCH 5/5] backing-dev: replace sync_supers_tsk/timer with a delayed_work Tejun Heo
@ 2010-09-07 12:16 ` Christoph Hellwig
  2010-09-07 12:19   ` Tejun Heo
  5 siblings, 1 reply; 15+ messages in thread
From: Christoph Hellwig @ 2010-09-07 12:16 UTC (permalink / raw)
  To: Tejun Heo; +Cc: jaxboe, linux-kernel, hch

What does this conversion buy us?

Having explicit per-bdi threads makes the model very simple and safe,
while I don't quite trust the now extremly complex work queues yet.


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

* Re: [PATCHSET] backing-dev: replace private thread pool with workqueue
  2010-09-07 12:16 ` [PATCHSET] backing-dev: replace private thread pool with workqueue Christoph Hellwig
@ 2010-09-07 12:19   ` Tejun Heo
  2010-09-07 12:29     ` Jens Axboe
  0 siblings, 1 reply; 15+ messages in thread
From: Tejun Heo @ 2010-09-07 12:19 UTC (permalink / raw)
  To: Christoph Hellwig; +Cc: jaxboe, linux-kernel

Hello,

On 09/07/2010 02:16 PM, Christoph Hellwig wrote:
> What does this conversion buy us?
> 
> Having explicit per-bdi threads makes the model very simple and safe,
> while I don't quite trust the now extremly complex work queues yet.

Well, the whole idea behind the new workqueue is unifying all those
dedicated pools of threads.  If the conversion is too early given the
new workqueue implementation, we can postpone bdi conversion by one
more release cycle, but in the long run there is no point in keeping
these private thread pools.

Thanks.

-- 
tejun

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

* Re: [PATCHSET] backing-dev: replace private thread pool with  workqueue
  2010-09-07 12:19   ` Tejun Heo
@ 2010-09-07 12:29     ` Jens Axboe
  2010-09-07 12:36       ` Tejun Heo
  0 siblings, 1 reply; 15+ messages in thread
From: Jens Axboe @ 2010-09-07 12:29 UTC (permalink / raw)
  To: Tejun Heo; +Cc: Christoph Hellwig, linux-kernel

On 2010-09-07 14:19, Tejun Heo wrote:
> Hello,
> 
> On 09/07/2010 02:16 PM, Christoph Hellwig wrote:
>> What does this conversion buy us?
>>
>> Having explicit per-bdi threads makes the model very simple and safe,
>> while I don't quite trust the now extremly complex work queues yet.
> 
> Well, the whole idea behind the new workqueue is unifying all those
> dedicated pools of threads.  If the conversion is too early given the
> new workqueue implementation, we can postpone bdi conversion by one
> more release cycle, but in the long run there is no point in keeping
> these private thread pools.

I agree (with both of you). It's definitely too early to convert it
over, but if we can in the longer run, it never hurts to get rid of
code. The writeback threads aren't a typical threadpool, in that the
threads stick around and only go away when idle for too long. If they
stick around, you get the same process hammering IO at your device. So
converting that over to the generic cwq may or may not be at a
performance cost, it'll definitely have to be tested.

-- 
Jens Axboe


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

* Re: [PATCHSET] backing-dev: replace private thread pool with  workqueue
  2010-09-07 12:29     ` Jens Axboe
@ 2010-09-07 12:36       ` Tejun Heo
  2010-11-27 16:02         ` Tejun Heo
  0 siblings, 1 reply; 15+ messages in thread
From: Tejun Heo @ 2010-09-07 12:36 UTC (permalink / raw)
  To: Jens Axboe; +Cc: Christoph Hellwig, linux-kernel

Hello,

On 09/07/2010 02:29 PM, Jens Axboe wrote:
> I agree (with both of you). It's definitely too early to convert it
> over, but if we can in the longer run, it never hurts to get rid of
> code. The writeback threads aren't a typical threadpool, in that the
> threads stick around and only go away when idle for too long. If they
> stick around, you get the same process hammering IO at your device. So
> converting that over to the generic cwq may or may not be at a
> performance cost, it'll definitely have to be tested.

One thing to try is removing WQ_UNBOUND and see how it affects the
performance.  I put WQ_UNBOUND there mainly to keep the behavior about
the same as the current code but given what it does I think it would
probably fare better with workers bound to CPUs.

Thanks.

-- 
tejun

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

* Re: [PATCH 2/5] backing-dev: kill unused bdi_writeback->nr
  2010-09-06 12:46 ` [PATCH 2/5] backing-dev: kill unused bdi_writeback->nr Tejun Heo
@ 2010-09-08  8:57   ` Artem Bityutskiy
  0 siblings, 0 replies; 15+ messages in thread
From: Artem Bityutskiy @ 2010-09-08  8:57 UTC (permalink / raw)
  To: Tejun Heo; +Cc: jaxboe, linux-kernel, hch

On Mon, 2010-09-06 at 14:46 +0200, Tejun Heo wrote:
> bdi_writeback->nr is no longer used.  Kill it.
> 
> Signed-off-by: Tejun Heo <tj@kernel.org>

This is obviously an independent clean-up which can be picked straight
away?

-- 
Best Regards,
Artem Bityutskiy (Артём Битюцкий)


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

* Re: [PATCH 5/5] backing-dev: replace sync_supers_tsk/timer with a delayed_work
  2010-09-06 12:46 ` [PATCH 5/5] backing-dev: replace sync_supers_tsk/timer with a delayed_work Tejun Heo
@ 2010-09-08  9:00   ` Artem Bityutskiy
  0 siblings, 0 replies; 15+ messages in thread
From: Artem Bityutskiy @ 2010-09-08  9:00 UTC (permalink / raw)
  To: Tejun Heo; +Cc: jaxboe, linux-kernel, hch

On Mon, 2010-09-06 at 14:46 +0200, Tejun Heo wrote:
> sync_supers_task/timer can trivially replaced with a delayed_work.  As
> it's not on the allocation path, run it on the system_wq.
> 
> NOTE: Both before and after this patch, sync_supers is not freezeable.
>       This might need to be changed.
> 
> Signed-off-by: Tejun Heo <tj@kernel.org>
> ---
>  mm/backing-dev.c |   49 ++++++++++++++-----------------------------------
>  1 files changed, 14 insertions(+), 35 deletions(-)

Actually what Al Viro wants is to make sync_supers() go away and make
every FS to take care of its own superblock. So I guess this change is
useful in a sense that it'll be then easier to push this down to FSes.

-- 
Best Regards,
Artem Bityutskiy (Артём Битюцкий)


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

* Re: [PATCHSET] backing-dev: replace private thread pool with  workqueue
  2010-09-07 12:36       ` Tejun Heo
@ 2010-11-27 16:02         ` Tejun Heo
  2010-11-27 18:50           ` Jens Axboe
  0 siblings, 1 reply; 15+ messages in thread
From: Tejun Heo @ 2010-11-27 16:02 UTC (permalink / raw)
  To: Jens Axboe; +Cc: Christoph Hellwig, linux-kernel

On 09/07/2010 02:36 PM, Tejun Heo wrote:
> Hello,
> 
> On 09/07/2010 02:29 PM, Jens Axboe wrote:
>> I agree (with both of you). It's definitely too early to convert it
>> over, but if we can in the longer run, it never hurts to get rid of
>> code. The writeback threads aren't a typical threadpool, in that the
>> threads stick around and only go away when idle for too long. If they
>> stick around, you get the same process hammering IO at your device. So
>> converting that over to the generic cwq may or may not be at a
>> performance cost, it'll definitely have to be tested.
> 
> One thing to try is removing WQ_UNBOUND and see how it affects the
> performance.  I put WQ_UNBOUND there mainly to keep the behavior about
> the same as the current code but given what it does I think it would
> probably fare better with workers bound to CPUs.

cmwq now seems pretty solid.  There hasn't been any noticeable failure
yet.  I think we can move on with this conversion now.  Shall I
refresh the patchset against the current block tree?

Thanks.

-- 
tejun

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

* Re: [PATCHSET] backing-dev: replace private thread pool with   workqueue
  2010-11-27 16:02         ` Tejun Heo
@ 2010-11-27 18:50           ` Jens Axboe
  2010-11-27 20:30             ` Tejun Heo
  0 siblings, 1 reply; 15+ messages in thread
From: Jens Axboe @ 2010-11-27 18:50 UTC (permalink / raw)
  To: Tejun Heo; +Cc: Christoph Hellwig, linux-kernel

On 2010-11-27 17:02, Tejun Heo wrote:
> On 09/07/2010 02:36 PM, Tejun Heo wrote:
>> Hello,
>>
>> On 09/07/2010 02:29 PM, Jens Axboe wrote:
>>> I agree (with both of you). It's definitely too early to convert it
>>> over, but if we can in the longer run, it never hurts to get rid of
>>> code. The writeback threads aren't a typical threadpool, in that the
>>> threads stick around and only go away when idle for too long. If they
>>> stick around, you get the same process hammering IO at your device. So
>>> converting that over to the generic cwq may or may not be at a
>>> performance cost, it'll definitely have to be tested.
>>
>> One thing to try is removing WQ_UNBOUND and see how it affects the
>> performance.  I put WQ_UNBOUND there mainly to keep the behavior about
>> the same as the current code but given what it does I think it would
>> probably fare better with workers bound to CPUs.
> 
> cmwq now seems pretty solid.  There hasn't been any noticeable failure
> yet.  I think we can move on with this conversion now.  Shall I
> refresh the patchset against the current block tree?

I'd still prefer to wait a while. The writeback code is still very much
a moving target, so I don't think mixing in a different work queue
scheme is likely going to do anyone any good at this point in time.

-- 
Jens Axboe


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

* Re: [PATCHSET] backing-dev: replace private thread pool with   workqueue
  2010-11-27 18:50           ` Jens Axboe
@ 2010-11-27 20:30             ` Tejun Heo
  0 siblings, 0 replies; 15+ messages in thread
From: Tejun Heo @ 2010-11-27 20:30 UTC (permalink / raw)
  To: Jens Axboe; +Cc: Christoph Hellwig, linux-kernel

Hello,

On 11/27/2010 07:50 PM, Jens Axboe wrote:
>> cmwq now seems pretty solid.  There hasn't been any noticeable failure
>> yet.  I think we can move on with this conversion now.  Shall I
>> refresh the patchset against the current block tree?
> 
> I'd still prefer to wait a while. The writeback code is still very much
> a moving target, so I don't think mixing in a different work queue
> scheme is likely going to do anyone any good at this point in time.

Alright, will ping again after the next merge window.

Thanks.

-- 
tejun

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

end of thread, other threads:[~2010-11-27 20:30 UTC | newest]

Thread overview: 15+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2010-09-06 12:46 [PATCHSET] backing-dev: replace private thread pool with workqueue Tejun Heo
2010-09-06 12:46 ` [PATCH 1/5] workqueue: implement workqueue_on_rescuer() Tejun Heo
2010-09-06 12:46 ` [PATCH 2/5] backing-dev: kill unused bdi_writeback->nr Tejun Heo
2010-09-08  8:57   ` Artem Bityutskiy
2010-09-06 12:46 ` [PATCH 3/5] backing-dev: replace private thread pool with workqueue Tejun Heo
2010-09-06 12:46 ` [PATCH 4/5] backing-dev: update trace points Tejun Heo
2010-09-06 12:46 ` [PATCH 5/5] backing-dev: replace sync_supers_tsk/timer with a delayed_work Tejun Heo
2010-09-08  9:00   ` Artem Bityutskiy
2010-09-07 12:16 ` [PATCHSET] backing-dev: replace private thread pool with workqueue Christoph Hellwig
2010-09-07 12:19   ` Tejun Heo
2010-09-07 12:29     ` Jens Axboe
2010-09-07 12:36       ` Tejun Heo
2010-11-27 16:02         ` Tejun Heo
2010-11-27 18:50           ` Jens Axboe
2010-11-27 20:30             ` Tejun Heo

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