All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCH 1/2] watchdog: introduce touch_softlockup_watchdog_sched()
@ 2015-12-03  0:28 Tejun Heo
  2015-12-03  0:28 ` [PATCH 2/2] workqueue: implement lockup detector Tejun Heo
  2015-12-03  9:33 ` [PATCH 1/2] watchdog: introduce touch_softlockup_watchdog_sched() Peter Zijlstra
  0 siblings, 2 replies; 51+ messages in thread
From: Tejun Heo @ 2015-12-03  0:28 UTC (permalink / raw)
  To: Ulrich Obergfell, Ingo Molnar, Peter Zijlstra, Andrew Morton
  Cc: linux-kernel, kernel-team

Hello,

There haven't been too many workqueue stall bugs; however, good part
of them have been pretty painful to track down because there's no
lockup detection mechanism for workqueue and it isn't easy to tell
what's going on with workqueues; furthermore, some requirements are
tricky to get right - e.g. it's not too difficult to miss
WQ_MEM_RECLAIM for a workqueue which runs a work item which is flushed
by something which sits in the reclaim path.

To alleviate the situation, this two patch series implements workqueue
lockup detector.  Each worker_pool tracks the last time it made
forward progress and if no forward progress is made for longer than
threshold it triggers warnings and dumps workqueue state.  It's
controlled together with scheduler softlockup mechanism and uses the
same threshold value as it shares a lot of the characteristics.

Thanks.

------ 8< ------
touch_softlockup_watchdog() is used to tell watchdog that scheduler
stall is expected.  One group of usage is from paths where the task
may not be able to yield for a long time such as performing slow PIO
to finicky device and coming out of suspend.  The other is to account
for scheduler and timer going idle.

For scheduler softlockup detection, there's no reason to distinguish
the two cases; however, workqueue lockup detector is planned and it
can use the same signals from the former group while the latter would
spuriously prevent detection.  This patch introduces a new function
touch_softlockup_watchdog_sched() and convert the latter group to call
it instead.  For now, it just calls touch_softlockup_watchdog() and
there's no functional difference.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Ulrich Obergfell <uobergfe@redhat.com>
Cc: Ingo Molnar <mingo@redhat.com>
Cc: Peter Zijlstra <peterz@infradead.org>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: Andrew Morton <akpm@linux-foundation.org>
---
 include/linux/sched.h    |    4 ++++
 kernel/sched/clock.c     |    2 +-
 kernel/time/tick-sched.c |    6 +++---
 kernel/watchdog.c        |   15 ++++++++++++++-
 4 files changed, 22 insertions(+), 5 deletions(-)

--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -377,6 +377,7 @@ extern void scheduler_tick(void);
 extern void sched_show_task(struct task_struct *p);
 
 #ifdef CONFIG_LOCKUP_DETECTOR
+extern void touch_softlockup_watchdog_sched(void);
 extern void touch_softlockup_watchdog(void);
 extern void touch_softlockup_watchdog_sync(void);
 extern void touch_all_softlockup_watchdogs(void);
@@ -387,6 +388,9 @@ extern unsigned int  softlockup_panic;
 extern unsigned int  hardlockup_panic;
 void lockup_detector_init(void);
 #else
+static inline void touch_softlockup_watchdog_sched(void)
+{
+}
 static inline void touch_softlockup_watchdog(void)
 {
 }
--- a/kernel/sched/clock.c
+++ b/kernel/sched/clock.c
@@ -354,7 +354,7 @@ void sched_clock_idle_wakeup_event(u64 d
 		return;
 
 	sched_clock_tick();
-	touch_softlockup_watchdog();
+	touch_softlockup_watchdog_sched();
 }
 EXPORT_SYMBOL_GPL(sched_clock_idle_wakeup_event);
 
--- a/kernel/time/tick-sched.c
+++ b/kernel/time/tick-sched.c
@@ -143,7 +143,7 @@ static void tick_sched_handle(struct tic
 	 * when we go busy again does not account too much ticks.
 	 */
 	if (ts->tick_stopped) {
-		touch_softlockup_watchdog();
+		touch_softlockup_watchdog_sched();
 		if (is_idle_task(current))
 			ts->idle_jiffies++;
 	}
@@ -430,7 +430,7 @@ static void tick_nohz_update_jiffies(kti
 	tick_do_update_jiffies64(now);
 	local_irq_restore(flags);
 
-	touch_softlockup_watchdog();
+	touch_softlockup_watchdog_sched();
 }
 
 /*
@@ -701,7 +701,7 @@ static void tick_nohz_restart_sched_tick
 	update_cpu_load_nohz();
 
 	calc_load_exit_idle();
-	touch_softlockup_watchdog();
+	touch_softlockup_watchdog_sched();
 	/*
 	 * Cancel the scheduled timer and restore the tick
 	 */
--- a/kernel/watchdog.c
+++ b/kernel/watchdog.c
@@ -225,7 +225,15 @@ static void __touch_watchdog(void)
 	__this_cpu_write(watchdog_touch_ts, get_timestamp());
 }
 
-void touch_softlockup_watchdog(void)
+/**
+ * touch_softlockup_watchdog_sched - touch watchdog on scheduler stalls
+ *
+ * Call when the scheduler may have stalled for legitimate reasons
+ * preventing the watchdog task from executing - e.g. the scheduler
+ * entering idle state.  This should only be used for scheduler events.
+ * Use touch_softlockup_watchdog() for everything else.
+ */
+void touch_softlockup_watchdog_sched(void)
 {
 	/*
 	 * Preemption can be enabled.  It doesn't matter which CPU's timestamp
@@ -233,6 +241,11 @@ void touch_softlockup_watchdog(void)
 	 */
 	raw_cpu_write(watchdog_touch_ts, 0);
 }
+
+void touch_softlockup_watchdog(void)
+{
+	touch_softlockup_watchdog_sched();
+}
 EXPORT_SYMBOL(touch_softlockup_watchdog);
 
 void touch_all_softlockup_watchdogs(void)

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

* [PATCH 2/2] workqueue: implement lockup detector
  2015-12-03  0:28 [PATCH 1/2] watchdog: introduce touch_softlockup_watchdog_sched() Tejun Heo
@ 2015-12-03  0:28 ` Tejun Heo
  2015-12-03 14:49   ` Tejun Heo
                     ` (2 more replies)
  2015-12-03  9:33 ` [PATCH 1/2] watchdog: introduce touch_softlockup_watchdog_sched() Peter Zijlstra
  1 sibling, 3 replies; 51+ messages in thread
From: Tejun Heo @ 2015-12-03  0:28 UTC (permalink / raw)
  To: Ulrich Obergfell, Ingo Molnar, Peter Zijlstra, Andrew Morton
  Cc: linux-kernel, kernel-team

Workqueue stalls can happen from a variety of usage bugs such as
missing WQ_MEM_RECLAIM flag or concurrency managed work item
indefinitely staying RUNNING.  These stalls can be extremely difficult
to hunt down because the usual warning mechanisms can't detect
workqueue stalls and the internal state is pretty opaque.

To alleviate the situation, this patch implements workqueue lockup
detector.  It periodically monitors all worker_pools periodically and,
if any pool failed to make forward progress longer than the threshold
duration, triggers warning and dumps workqueue state as follows.

 BUG: workqueue lockup - pool cpus=0 node=0 flags=0x0 nice=0 stuck for 31s!
 Showing busy workqueues and worker pools:
 workqueue events: flags=0x0
   pwq 0: cpus=0 node=0 flags=0x0 nice=0 active=17/256
     pending: monkey_wrench_fn, e1000_watchdog, cache_reap, vmstat_shepherd, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, cgroup_release_agent
 workqueue events_power_efficient: flags=0x80
   pwq 0: cpus=0 node=0 flags=0x0 nice=0 active=2/256
     pending: check_lifetime, neigh_periodic_work
 workqueue cgroup_pidlist_destroy: flags=0x0
   pwq 0: cpus=0 node=0 flags=0x0 nice=0 active=1/1
     pending: cgroup_pidlist_destroy_work_fn
 ...

The detection mechanism is enabled/disabled together with scheduler
softlockup watchdog and uses the same threshold value; however, it
currently doesn't trigger panic.  We can do that later once the
detection mechanism proves to be reliable.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Ulrich Obergfell <uobergfe@redhat.com>
Cc: Michal Hocko <mhocko@suse.com>
Cc: Chris Mason <clm@fb.com>
Cc: Andrew Morton <akpm@linux-foundation.org>
---
 include/linux/sched.h     |    1 
 include/linux/workqueue.h |    6 +
 kernel/watchdog.c         |   15 +++-
 kernel/workqueue.c        |  146 +++++++++++++++++++++++++++++++++++++++++++++-
 4 files changed, 162 insertions(+), 6 deletions(-)

--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -381,6 +381,7 @@ extern void touch_softlockup_watchdog_sc
 extern void touch_softlockup_watchdog(void);
 extern void touch_softlockup_watchdog_sync(void);
 extern void touch_all_softlockup_watchdogs(void);
+extern int get_softlockup_thresh(void);
 extern int proc_dowatchdog_thresh(struct ctl_table *table, int write,
 				  void __user *buffer,
 				  size_t *lenp, loff_t *ppos);
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -618,4 +618,10 @@ static inline int workqueue_sysfs_regist
 { return 0; }
 #endif	/* CONFIG_SYSFS */
 
+#ifdef CONFIG_LOCKUP_DETECTOR
+void enable_workqueue_watchdog(void);
+void disable_workqueue_watchdog(void);
+void touch_workqueue_watchdog(int cpu);
+#endif	/* CONFIG_LOCKUP_DETECTOR */
+
 #endif
--- a/kernel/watchdog.c
+++ b/kernel/watchdog.c
@@ -20,6 +20,7 @@
 #include <linux/smpboot.h>
 #include <linux/sched/rt.h>
 #include <linux/tick.h>
+#include <linux/workqueue.h>
 
 #include <asm/irq_regs.h>
 #include <linux/kvm_para.h>
@@ -192,7 +193,7 @@ __setup("hardlockup_all_cpu_backtrace=",
  * the thresholds with a factor: we make the soft threshold twice the amount of
  * time the hard threshold is.
  */
-static int get_softlockup_thresh(void)
+int get_softlockup_thresh(void)
 {
 	return watchdog_thresh * 2;
 }
@@ -245,6 +246,7 @@ void touch_softlockup_watchdog_sched(voi
 void touch_softlockup_watchdog(void)
 {
 	touch_softlockup_watchdog_sched();
+	touch_workqueue_watchdog(raw_smp_processor_id());
 }
 EXPORT_SYMBOL(touch_softlockup_watchdog);
 
@@ -259,6 +261,7 @@ void touch_all_softlockup_watchdogs(void
 	 */
 	for_each_watchdog_cpu(cpu)
 		per_cpu(watchdog_touch_ts, cpu) = 0;
+	touch_workqueue_watchdog(-1);
 }
 
 #ifdef CONFIG_HARDLOCKUP_DETECTOR
@@ -795,13 +798,18 @@ static int watchdog_enable_all_cpus(void
 {
 	int err = 0;
 
+	disable_workqueue_watchdog();
+
 	if (!watchdog_running) {
 		err = smpboot_register_percpu_thread_cpumask(&watchdog_threads,
 							     &watchdog_cpumask);
-		if (err)
+		if (err) {
 			pr_err("Failed to create watchdog threads, disabled\n");
-		else
+		} else {
+			if (watchdog_enabled & SOFT_WATCHDOG_ENABLED)
+				enable_workqueue_watchdog();
 			watchdog_running = 1;
+		}
 	} else {
 		/*
 		 * Enable/disable the lockup detectors or
@@ -826,6 +834,7 @@ static void watchdog_disable_all_cpus(vo
 	if (watchdog_running) {
 		watchdog_running = 0;
 		smpboot_unregister_percpu_thread(&watchdog_threads);
+		disable_workqueue_watchdog();
 	}
 }
 
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -148,6 +148,8 @@ struct worker_pool {
 	int			id;		/* I: pool ID */
 	unsigned int		flags;		/* X: flags */
 
+	unsigned long		watchdog_ts;	/* L: watchdog timestamp */
+
 	struct list_head	worklist;	/* L: list of pending works */
 	int			nr_workers;	/* L: total number of workers */
 
@@ -1083,6 +1085,8 @@ static void pwq_activate_delayed_work(st
 	struct pool_workqueue *pwq = get_work_pwq(work);
 
 	trace_workqueue_activate_work(work);
+	if (list_empty(&pwq->pool->worklist))
+		pwq->pool->watchdog_ts = jiffies;
 	move_linked_works(work, &pwq->pool->worklist, NULL);
 	__clear_bit(WORK_STRUCT_DELAYED_BIT, work_data_bits(work));
 	pwq->nr_active++;
@@ -1385,6 +1389,8 @@ retry:
 		trace_workqueue_activate_work(work);
 		pwq->nr_active++;
 		worklist = &pwq->pool->worklist;
+		if (list_empty(worklist))
+			pwq->pool->watchdog_ts = jiffies;
 	} else {
 		work_flags |= WORK_STRUCT_DELAYED;
 		worklist = &pwq->delayed_works;
@@ -2157,6 +2163,8 @@ recheck:
 			list_first_entry(&pool->worklist,
 					 struct work_struct, entry);
 
+		pool->watchdog_ts = jiffies;
+
 		if (likely(!(*work_data_bits(work) & WORK_STRUCT_LINKED))) {
 			/* optimization path, not strictly necessary */
 			process_one_work(worker, work);
@@ -2240,6 +2248,7 @@ repeat:
 					struct pool_workqueue, mayday_node);
 		struct worker_pool *pool = pwq->pool;
 		struct work_struct *work, *n;
+		bool first = true;
 
 		__set_current_state(TASK_RUNNING);
 		list_del_init(&pwq->mayday_node);
@@ -2256,9 +2265,14 @@ repeat:
 		 * process'em.
 		 */
 		WARN_ON_ONCE(!list_empty(scheduled));
-		list_for_each_entry_safe(work, n, &pool->worklist, entry)
-			if (get_work_pwq(work) == pwq)
+		list_for_each_entry_safe(work, n, &pool->worklist, entry) {
+			if (get_work_pwq(work) == pwq) {
+				if (first)
+					pool->watchdog_ts = jiffies;
 				move_linked_works(work, scheduled, &n);
+			}
+			first = false;
+		}
 
 		if (!list_empty(scheduled)) {
 			process_scheduled_works(rescuer);
@@ -3069,6 +3083,7 @@ static int init_worker_pool(struct worke
 	pool->cpu = -1;
 	pool->node = NUMA_NO_NODE;
 	pool->flags |= POOL_DISASSOCIATED;
+	pool->watchdog_ts = jiffies;
 	INIT_LIST_HEAD(&pool->worklist);
 	INIT_LIST_HEAD(&pool->idle_list);
 	hash_init(pool->busy_hash);
@@ -4308,7 +4323,9 @@ void show_workqueue_state(void)
 
 		pr_info("pool %d:", pool->id);
 		pr_cont_pool_info(pool);
-		pr_cont(" workers=%d", pool->nr_workers);
+		pr_cont(" hung=%us workers=%d",
+			jiffies_to_msecs(jiffies - pool->watchdog_ts) / 1000,
+			pool->nr_workers);
 		if (pool->manager)
 			pr_cont(" manager: %d",
 				task_pid_nr(pool->manager->task));
@@ -5167,6 +5184,129 @@ static void workqueue_sysfs_unregister(s
 static void workqueue_sysfs_unregister(struct workqueue_struct *wq)	{ }
 #endif	/* CONFIG_SYSFS */
 
+/*
+ * Workqueue watchdog.
+ *
+ * Stall may be caused by various bugs - missing WQ_MEM_RECLAIM, illegal
+ * flush dependency, a concurrency managed work item which stays RUNNING
+ * indefinitely.  Workqueue stalls can be very difficult to debug as the
+ * usual warning mechanisms don't trigger and internal workqueue state is
+ * largely opaque.
+ *
+ * Workqueue watchdog monitors all worker pools periodically and dumps
+ * state if some pools failed to make forward progress for a while where
+ * forward progress is defined as the first item on ->worklist changing.
+ *
+ * The mechanism is enabled and disabled together with softlockup watchdog
+ * and uses the same threshold duration; however, it currently doesn't
+ * cause panic even if softlockup_panic is set.  Also, workqueue watchdog
+ * assumes that the usual jiffies and timer mechanisms are working as there
+ * isn't much point in warning about workqueue stalls when timer is broken.
+ */
+#ifdef CONFIG_LOCKUP_DETECTOR
+
+static void wq_watchdog_timer_fn(unsigned long data);
+
+static unsigned long wq_watchdog_thresh;
+static struct timer_list wq_watchdog_timer =
+	TIMER_DEFERRED_INITIALIZER(wq_watchdog_timer_fn, 0, 0);
+
+static unsigned long wq_watchdog_touched = INITIAL_JIFFIES;
+static DEFINE_PER_CPU(unsigned long, wq_watchdog_touched_cpu) = INITIAL_JIFFIES;
+
+static void wq_watchdog_reset_touched(void)
+{
+	int cpu;
+
+	wq_watchdog_touched = jiffies;
+	for_each_possible_cpu(cpu)
+		per_cpu(wq_watchdog_touched_cpu, cpu) = jiffies;
+}
+
+static void wq_watchdog_timer_fn(unsigned long data)
+{
+	unsigned long thresh = wq_watchdog_thresh;
+	bool lockup_detected = false;
+	struct worker_pool *pool;
+	int pi;
+
+	if (!thresh)
+		return;
+
+	rcu_read_lock();
+
+	for_each_pool(pool, pi) {
+		unsigned long pool_ts, touched, ts;
+
+		if (list_empty(&pool->worklist))
+			continue;
+
+		/* get the latest of pool and touched timestamps */
+		pool_ts = READ_ONCE(pool->watchdog_ts);
+		touched = READ_ONCE(wq_watchdog_touched);
+
+		if (time_after(pool_ts, touched))
+			ts = pool_ts;
+		else
+			ts = touched;
+
+		if (pool->cpu >= 0) {
+			unsigned long cpu_touched =
+				READ_ONCE(per_cpu(wq_watchdog_touched_cpu,
+						  pool->cpu));
+			if (time_after(cpu_touched, ts))
+				ts = cpu_touched;
+		}
+
+		/* did we stall? */
+		if (time_after(jiffies, ts + thresh)) {
+			lockup_detected = true;
+			pr_emerg("BUG: workqueue lockup - pool");
+			pr_cont_pool_info(pool);
+			pr_cont(" stuck for %us!\n",
+				jiffies_to_msecs(jiffies - pool_ts) / 1000);
+		}
+	}
+
+	rcu_read_unlock();
+
+	if (lockup_detected)
+		show_workqueue_state();
+
+	wq_watchdog_reset_touched();
+	mod_timer(&wq_watchdog_timer, jiffies + thresh);
+}
+
+void enable_workqueue_watchdog(void)
+{
+	wq_watchdog_thresh = get_softlockup_thresh() * HZ;
+
+	wq_watchdog_reset_touched();
+	mod_timer(&wq_watchdog_timer, jiffies + wq_watchdog_thresh);
+}
+
+void disable_workqueue_watchdog(void)
+{
+	wq_watchdog_thresh = 0;
+	del_timer_sync(&wq_watchdog_timer);
+}
+
+void touch_workqueue_watchdog(int cpu)
+{
+	/*
+	 * If not explicitly touched, these stamps are never updated, which
+	 * means that they may affect stall detection if jiffies wraps;
+	 * however, it's highly unlikely and the worst that can happen is
+	 * delaying stall detection by upto one threshold duration.
+	 */
+	if (cpu >= 0)
+		per_cpu(wq_watchdog_touched_cpu, cpu) = jiffies;
+	else
+		wq_watchdog_touched = jiffies;
+}
+
+#endif	/* CONFIG_LOCKUP_DETECTOR */
+
 static void __init wq_numa_init(void)
 {
 	cpumask_var_t *tbl;

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

* Re: [PATCH 1/2] watchdog: introduce touch_softlockup_watchdog_sched()
  2015-12-03  0:28 [PATCH 1/2] watchdog: introduce touch_softlockup_watchdog_sched() Tejun Heo
  2015-12-03  0:28 ` [PATCH 2/2] workqueue: implement lockup detector Tejun Heo
@ 2015-12-03  9:33 ` Peter Zijlstra
  2015-12-03 10:00   ` Peter Zijlstra
  1 sibling, 1 reply; 51+ messages in thread
From: Peter Zijlstra @ 2015-12-03  9:33 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Ulrich Obergfell, Ingo Molnar, Andrew Morton, linux-kernel, kernel-team

On Wed, Dec 02, 2015 at 07:28:10PM -0500, Tejun Heo wrote:
> Hello,
> 
> There haven't been too many workqueue stall bugs; however, good part
> of them have been pretty painful to track down because there's no
> lockup detection mechanism for workqueue and it isn't easy to tell
> what's going on with workqueues; furthermore, some requirements are
> tricky to get right - e.g. it's not too difficult to miss
> WQ_MEM_RECLAIM for a workqueue which runs a work item which is flushed
> by something which sits in the reclaim path.

have you considered something as simple as:

	WARN_ON(current->reclaim_state && !WQ_MEM_RECLAIM);

?

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

* Re: [PATCH 1/2] watchdog: introduce touch_softlockup_watchdog_sched()
  2015-12-03  9:33 ` [PATCH 1/2] watchdog: introduce touch_softlockup_watchdog_sched() Peter Zijlstra
@ 2015-12-03 10:00   ` Peter Zijlstra
  2015-12-03 14:48     ` Tejun Heo
  0 siblings, 1 reply; 51+ messages in thread
From: Peter Zijlstra @ 2015-12-03 10:00 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Ulrich Obergfell, Ingo Molnar, Andrew Morton, linux-kernel, kernel-team

On Thu, Dec 03, 2015 at 10:33:50AM +0100, Peter Zijlstra wrote:
> On Wed, Dec 02, 2015 at 07:28:10PM -0500, Tejun Heo wrote:
> > Hello,
> > 
> > There haven't been too many workqueue stall bugs; however, good part
> > of them have been pretty painful to track down because there's no
> > lockup detection mechanism for workqueue and it isn't easy to tell
> > what's going on with workqueues; furthermore, some requirements are
> > tricky to get right - e.g. it's not too difficult to miss
> > WQ_MEM_RECLAIM for a workqueue which runs a work item which is flushed
> > by something which sits in the reclaim path.
> 
> have you considered something as simple as:
> 
> 	WARN_ON(current->reclaim_state && !WQ_MEM_RECLAIM);
> 
> ?

Alternatively, you can 'abuse' the lockdep reclaim bits by marking
!MEM_RECLAIM workqueue 'locks' with lockdep_trace_alloc(GFP_KERNEL),
that way lockdep will yell if they get used in a reclaim context.

This might be a tad tricky in that you need 2 sets of (lockdep) keys for
things.

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

* Re: [PATCH 1/2] watchdog: introduce touch_softlockup_watchdog_sched()
  2015-12-03 10:00   ` Peter Zijlstra
@ 2015-12-03 14:48     ` Tejun Heo
  2015-12-03 15:04       ` Peter Zijlstra
  0 siblings, 1 reply; 51+ messages in thread
From: Tejun Heo @ 2015-12-03 14:48 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Ulrich Obergfell, Ingo Molnar, Andrew Morton, linux-kernel, kernel-team

Hey, Peter.

On Thu, Dec 03, 2015 at 11:00:18AM +0100, Peter Zijlstra wrote:
> > have you considered something as simple as:
> > 
> > 	WARN_ON(current->reclaim_state && !WQ_MEM_RECLAIM);
> > 
> > ?
>
> Alternatively, you can 'abuse' the lockdep reclaim bits by marking
> !MEM_RECLAIM workqueue 'locks' with lockdep_trace_alloc(GFP_KERNEL),
> that way lockdep will yell if they get used in a reclaim context.
> 
> This might be a tad tricky in that you need 2 sets of (lockdep) keys for
> things.

One of the latest bugs was an xfs work item in reclaim path which had
WQ_MEM_RECLAIM waiting on a waitqueue for an event which is to be
triggered by another work item which incorrectly was missing the flag.
Under memory pressure, it leads to silent deadlocks.  The other one
was vmstat update work busy looping waiting for a work item which is
queued behind it.  None of the dependency tracking mechanisms could
have detected either and both were pretty tricky to track down.

We can add MEM_RECLAIM -> !MEM_RECLAIM warning mechanism in addition
but I think adding stall detection is justified.

Thanks.

-- 
tejun

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

* Re: [PATCH 2/2] workqueue: implement lockup detector
  2015-12-03  0:28 ` [PATCH 2/2] workqueue: implement lockup detector Tejun Heo
@ 2015-12-03 14:49   ` Tejun Heo
  2015-12-03 17:50   ` Don Zickus
  2015-12-07 19:06   ` [PATCH v2 " Tejun Heo
  2 siblings, 0 replies; 51+ messages in thread
From: Tejun Heo @ 2015-12-03 14:49 UTC (permalink / raw)
  To: Ulrich Obergfell, Ingo Molnar, Peter Zijlstra, Andrew Morton
  Cc: linux-kernel, kernel-team

On Wed, Dec 02, 2015 at 07:28:39PM -0500, Tejun Heo wrote:
...
> +void touch_workqueue_watchdog(int cpu)
> +{
> +	/*
> +	 * If not explicitly touched, these stamps are never updated, which
> +	 * means that they may affect stall detection if jiffies wraps;
> +	 * however, it's highly unlikely and the worst that can happen is
> +	 * delaying stall detection by upto one threshold duration.
> +	 */

Oops, this comment is stale.  Updated the code but forgot to drop it.
Will remove in the next iteration.

Thanks.

-- 
tejun

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

* Re: [PATCH 1/2] watchdog: introduce touch_softlockup_watchdog_sched()
  2015-12-03 14:48     ` Tejun Heo
@ 2015-12-03 15:04       ` Peter Zijlstra
  2015-12-03 15:06         ` Tejun Heo
  0 siblings, 1 reply; 51+ messages in thread
From: Peter Zijlstra @ 2015-12-03 15:04 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Ulrich Obergfell, Ingo Molnar, Andrew Morton, linux-kernel, kernel-team

On Thu, Dec 03, 2015 at 09:48:11AM -0500, Tejun Heo wrote:
> We can add MEM_RECLAIM -> !MEM_RECLAIM warning mechanism in addition
> but I think adding stall detection is justified.

Sure, a stall mech is always nice, but I was thinking we should be able
to better catch some of these with explicit stuff.



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

* Re: [PATCH 1/2] watchdog: introduce touch_softlockup_watchdog_sched()
  2015-12-03 15:04       ` Peter Zijlstra
@ 2015-12-03 15:06         ` Tejun Heo
  2015-12-03 19:26           ` [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue Tejun Heo
  0 siblings, 1 reply; 51+ messages in thread
From: Tejun Heo @ 2015-12-03 15:06 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Ulrich Obergfell, Ingo Molnar, Andrew Morton, linux-kernel, kernel-team

On Thu, Dec 03, 2015 at 04:04:42PM +0100, Peter Zijlstra wrote:
> On Thu, Dec 03, 2015 at 09:48:11AM -0500, Tejun Heo wrote:
> > We can add MEM_RECLAIM -> !MEM_RECLAIM warning mechanism in addition
> > but I think adding stall detection is justified.
> 
> Sure, a stall mech is always nice, but I was thinking we should be able
> to better catch some of these with explicit stuff.

Yeah, sure, I don't think we even need to meddle with lockdep.  I'm
gonna add WARN_ON_ONCE()s which trigger whenever something on reclaim
path tries to flush !WQ_MEM_RECLAIM workqueues or work items.

Thanks.

-- 
tejun

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

* Re: [PATCH 2/2] workqueue: implement lockup detector
  2015-12-03  0:28 ` [PATCH 2/2] workqueue: implement lockup detector Tejun Heo
  2015-12-03 14:49   ` Tejun Heo
@ 2015-12-03 17:50   ` Don Zickus
  2015-12-03 19:43     ` Tejun Heo
  2015-12-07 19:06   ` [PATCH v2 " Tejun Heo
  2 siblings, 1 reply; 51+ messages in thread
From: Don Zickus @ 2015-12-03 17:50 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Ulrich Obergfell, Ingo Molnar, Peter Zijlstra, Andrew Morton,
	linux-kernel, kernel-team

On Wed, Dec 02, 2015 at 07:28:39PM -0500, Tejun Heo wrote:
> Workqueue stalls can happen from a variety of usage bugs such as
> missing WQ_MEM_RECLAIM flag or concurrency managed work item
> indefinitely staying RUNNING.  These stalls can be extremely difficult
> to hunt down because the usual warning mechanisms can't detect
> workqueue stalls and the internal state is pretty opaque.
> 
> To alleviate the situation, this patch implements workqueue lockup
> detector.  It periodically monitors all worker_pools periodically and,
> if any pool failed to make forward progress longer than the threshold
> duration, triggers warning and dumps workqueue state as follows.

This sort of looks like the hung task detector..

I am a little concerned because we just made a big effort to properly
separate the hardlockup and softlockup paths and yet retain the flexibility
to enable/disable them separately.  Now it seems the workqueue detector is
permanently entwined with the softlockup detector.  I am not entirely sure
that is correct thing to do.

It also seems awkward for the lockup code to have to jump to the workqueue
code to function properly. :-/  Though we have made exceptions for the virt
stuff and the workqueue code is simple..

Actually, I am curious, it seems if you just added a
/proc/sys/kernel/wq_watchdog entry, you could elminiate the entire need for
modifying the watchdog code to begin with.  As you really aren't using any
of it other than piggybacking on the touch_softlockup_watchdog stuff, which
could probably be easily added without all the extra enable/disable changes
in watchdog.c.

Again, this looks like what the hung task detector is doing, which I
struggled with years ago to integrate with the lockup code because in the
end I had trouble re-using much of it.

Thoughts?

Cheers,
Don

> 
>  BUG: workqueue lockup - pool cpus=0 node=0 flags=0x0 nice=0 stuck for 31s!
>  Showing busy workqueues and worker pools:
>  workqueue events: flags=0x0
>    pwq 0: cpus=0 node=0 flags=0x0 nice=0 active=17/256
>      pending: monkey_wrench_fn, e1000_watchdog, cache_reap, vmstat_shepherd, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, cgroup_release_agent
>  workqueue events_power_efficient: flags=0x80
>    pwq 0: cpus=0 node=0 flags=0x0 nice=0 active=2/256
>      pending: check_lifetime, neigh_periodic_work
>  workqueue cgroup_pidlist_destroy: flags=0x0
>    pwq 0: cpus=0 node=0 flags=0x0 nice=0 active=1/1
>      pending: cgroup_pidlist_destroy_work_fn
>  ...
> 
> The detection mechanism is enabled/disabled together with scheduler
> softlockup watchdog and uses the same threshold value; however, it
> currently doesn't trigger panic.  We can do that later once the
> detection mechanism proves to be reliable.
> 
> Signed-off-by: Tejun Heo <tj@kernel.org>
> Cc: Ulrich Obergfell <uobergfe@redhat.com>
> Cc: Michal Hocko <mhocko@suse.com>
> Cc: Chris Mason <clm@fb.com>
> Cc: Andrew Morton <akpm@linux-foundation.org>
> ---
>  include/linux/sched.h     |    1 
>  include/linux/workqueue.h |    6 +
>  kernel/watchdog.c         |   15 +++-
>  kernel/workqueue.c        |  146 +++++++++++++++++++++++++++++++++++++++++++++-
>  4 files changed, 162 insertions(+), 6 deletions(-)
> 
> --- a/include/linux/sched.h
> +++ b/include/linux/sched.h
> @@ -381,6 +381,7 @@ extern void touch_softlockup_watchdog_sc
>  extern void touch_softlockup_watchdog(void);
>  extern void touch_softlockup_watchdog_sync(void);
>  extern void touch_all_softlockup_watchdogs(void);
> +extern int get_softlockup_thresh(void);
>  extern int proc_dowatchdog_thresh(struct ctl_table *table, int write,
>  				  void __user *buffer,
>  				  size_t *lenp, loff_t *ppos);
> --- a/include/linux/workqueue.h
> +++ b/include/linux/workqueue.h
> @@ -618,4 +618,10 @@ static inline int workqueue_sysfs_regist
>  { return 0; }
>  #endif	/* CONFIG_SYSFS */
>  
> +#ifdef CONFIG_LOCKUP_DETECTOR
> +void enable_workqueue_watchdog(void);
> +void disable_workqueue_watchdog(void);
> +void touch_workqueue_watchdog(int cpu);
> +#endif	/* CONFIG_LOCKUP_DETECTOR */
> +
>  #endif
> --- a/kernel/watchdog.c
> +++ b/kernel/watchdog.c
> @@ -20,6 +20,7 @@
>  #include <linux/smpboot.h>
>  #include <linux/sched/rt.h>
>  #include <linux/tick.h>
> +#include <linux/workqueue.h>
>  
>  #include <asm/irq_regs.h>
>  #include <linux/kvm_para.h>
> @@ -192,7 +193,7 @@ __setup("hardlockup_all_cpu_backtrace=",
>   * the thresholds with a factor: we make the soft threshold twice the amount of
>   * time the hard threshold is.
>   */
> -static int get_softlockup_thresh(void)
> +int get_softlockup_thresh(void)
>  {
>  	return watchdog_thresh * 2;
>  }
> @@ -245,6 +246,7 @@ void touch_softlockup_watchdog_sched(voi
>  void touch_softlockup_watchdog(void)
>  {
>  	touch_softlockup_watchdog_sched();
> +	touch_workqueue_watchdog(raw_smp_processor_id());
>  }
>  EXPORT_SYMBOL(touch_softlockup_watchdog);
>  
> @@ -259,6 +261,7 @@ void touch_all_softlockup_watchdogs(void
>  	 */
>  	for_each_watchdog_cpu(cpu)
>  		per_cpu(watchdog_touch_ts, cpu) = 0;
> +	touch_workqueue_watchdog(-1);
>  }
>  
>  #ifdef CONFIG_HARDLOCKUP_DETECTOR
> @@ -795,13 +798,18 @@ static int watchdog_enable_all_cpus(void
>  {
>  	int err = 0;
>  
> +	disable_workqueue_watchdog();
> +
>  	if (!watchdog_running) {
>  		err = smpboot_register_percpu_thread_cpumask(&watchdog_threads,
>  							     &watchdog_cpumask);
> -		if (err)
> +		if (err) {
>  			pr_err("Failed to create watchdog threads, disabled\n");
> -		else
> +		} else {
> +			if (watchdog_enabled & SOFT_WATCHDOG_ENABLED)
> +				enable_workqueue_watchdog();
>  			watchdog_running = 1;
> +		}
>  	} else {
>  		/*
>  		 * Enable/disable the lockup detectors or
> @@ -826,6 +834,7 @@ static void watchdog_disable_all_cpus(vo
>  	if (watchdog_running) {
>  		watchdog_running = 0;
>  		smpboot_unregister_percpu_thread(&watchdog_threads);
> +		disable_workqueue_watchdog();
>  	}
>  }
>  
> --- a/kernel/workqueue.c
> +++ b/kernel/workqueue.c
> @@ -148,6 +148,8 @@ struct worker_pool {
>  	int			id;		/* I: pool ID */
>  	unsigned int		flags;		/* X: flags */
>  
> +	unsigned long		watchdog_ts;	/* L: watchdog timestamp */
> +
>  	struct list_head	worklist;	/* L: list of pending works */
>  	int			nr_workers;	/* L: total number of workers */
>  
> @@ -1083,6 +1085,8 @@ static void pwq_activate_delayed_work(st
>  	struct pool_workqueue *pwq = get_work_pwq(work);
>  
>  	trace_workqueue_activate_work(work);
> +	if (list_empty(&pwq->pool->worklist))
> +		pwq->pool->watchdog_ts = jiffies;
>  	move_linked_works(work, &pwq->pool->worklist, NULL);
>  	__clear_bit(WORK_STRUCT_DELAYED_BIT, work_data_bits(work));
>  	pwq->nr_active++;
> @@ -1385,6 +1389,8 @@ retry:
>  		trace_workqueue_activate_work(work);
>  		pwq->nr_active++;
>  		worklist = &pwq->pool->worklist;
> +		if (list_empty(worklist))
> +			pwq->pool->watchdog_ts = jiffies;
>  	} else {
>  		work_flags |= WORK_STRUCT_DELAYED;
>  		worklist = &pwq->delayed_works;
> @@ -2157,6 +2163,8 @@ recheck:
>  			list_first_entry(&pool->worklist,
>  					 struct work_struct, entry);
>  
> +		pool->watchdog_ts = jiffies;
> +
>  		if (likely(!(*work_data_bits(work) & WORK_STRUCT_LINKED))) {
>  			/* optimization path, not strictly necessary */
>  			process_one_work(worker, work);
> @@ -2240,6 +2248,7 @@ repeat:
>  					struct pool_workqueue, mayday_node);
>  		struct worker_pool *pool = pwq->pool;
>  		struct work_struct *work, *n;
> +		bool first = true;
>  
>  		__set_current_state(TASK_RUNNING);
>  		list_del_init(&pwq->mayday_node);
> @@ -2256,9 +2265,14 @@ repeat:
>  		 * process'em.
>  		 */
>  		WARN_ON_ONCE(!list_empty(scheduled));
> -		list_for_each_entry_safe(work, n, &pool->worklist, entry)
> -			if (get_work_pwq(work) == pwq)
> +		list_for_each_entry_safe(work, n, &pool->worklist, entry) {
> +			if (get_work_pwq(work) == pwq) {
> +				if (first)
> +					pool->watchdog_ts = jiffies;
>  				move_linked_works(work, scheduled, &n);
> +			}
> +			first = false;
> +		}
>  
>  		if (!list_empty(scheduled)) {
>  			process_scheduled_works(rescuer);
> @@ -3069,6 +3083,7 @@ static int init_worker_pool(struct worke
>  	pool->cpu = -1;
>  	pool->node = NUMA_NO_NODE;
>  	pool->flags |= POOL_DISASSOCIATED;
> +	pool->watchdog_ts = jiffies;
>  	INIT_LIST_HEAD(&pool->worklist);
>  	INIT_LIST_HEAD(&pool->idle_list);
>  	hash_init(pool->busy_hash);
> @@ -4308,7 +4323,9 @@ void show_workqueue_state(void)
>  
>  		pr_info("pool %d:", pool->id);
>  		pr_cont_pool_info(pool);
> -		pr_cont(" workers=%d", pool->nr_workers);
> +		pr_cont(" hung=%us workers=%d",
> +			jiffies_to_msecs(jiffies - pool->watchdog_ts) / 1000,
> +			pool->nr_workers);
>  		if (pool->manager)
>  			pr_cont(" manager: %d",
>  				task_pid_nr(pool->manager->task));
> @@ -5167,6 +5184,129 @@ static void workqueue_sysfs_unregister(s
>  static void workqueue_sysfs_unregister(struct workqueue_struct *wq)	{ }
>  #endif	/* CONFIG_SYSFS */
>  
> +/*
> + * Workqueue watchdog.
> + *
> + * Stall may be caused by various bugs - missing WQ_MEM_RECLAIM, illegal
> + * flush dependency, a concurrency managed work item which stays RUNNING
> + * indefinitely.  Workqueue stalls can be very difficult to debug as the
> + * usual warning mechanisms don't trigger and internal workqueue state is
> + * largely opaque.
> + *
> + * Workqueue watchdog monitors all worker pools periodically and dumps
> + * state if some pools failed to make forward progress for a while where
> + * forward progress is defined as the first item on ->worklist changing.
> + *
> + * The mechanism is enabled and disabled together with softlockup watchdog
> + * and uses the same threshold duration; however, it currently doesn't
> + * cause panic even if softlockup_panic is set.  Also, workqueue watchdog
> + * assumes that the usual jiffies and timer mechanisms are working as there
> + * isn't much point in warning about workqueue stalls when timer is broken.
> + */
> +#ifdef CONFIG_LOCKUP_DETECTOR
> +
> +static void wq_watchdog_timer_fn(unsigned long data);
> +
> +static unsigned long wq_watchdog_thresh;
> +static struct timer_list wq_watchdog_timer =
> +	TIMER_DEFERRED_INITIALIZER(wq_watchdog_timer_fn, 0, 0);
> +
> +static unsigned long wq_watchdog_touched = INITIAL_JIFFIES;
> +static DEFINE_PER_CPU(unsigned long, wq_watchdog_touched_cpu) = INITIAL_JIFFIES;
> +
> +static void wq_watchdog_reset_touched(void)
> +{
> +	int cpu;
> +
> +	wq_watchdog_touched = jiffies;
> +	for_each_possible_cpu(cpu)
> +		per_cpu(wq_watchdog_touched_cpu, cpu) = jiffies;
> +}
> +
> +static void wq_watchdog_timer_fn(unsigned long data)
> +{
> +	unsigned long thresh = wq_watchdog_thresh;
> +	bool lockup_detected = false;
> +	struct worker_pool *pool;
> +	int pi;
> +
> +	if (!thresh)
> +		return;
> +
> +	rcu_read_lock();
> +
> +	for_each_pool(pool, pi) {
> +		unsigned long pool_ts, touched, ts;
> +
> +		if (list_empty(&pool->worklist))
> +			continue;
> +
> +		/* get the latest of pool and touched timestamps */
> +		pool_ts = READ_ONCE(pool->watchdog_ts);
> +		touched = READ_ONCE(wq_watchdog_touched);
> +
> +		if (time_after(pool_ts, touched))
> +			ts = pool_ts;
> +		else
> +			ts = touched;
> +
> +		if (pool->cpu >= 0) {
> +			unsigned long cpu_touched =
> +				READ_ONCE(per_cpu(wq_watchdog_touched_cpu,
> +						  pool->cpu));
> +			if (time_after(cpu_touched, ts))
> +				ts = cpu_touched;
> +		}
> +
> +		/* did we stall? */
> +		if (time_after(jiffies, ts + thresh)) {
> +			lockup_detected = true;
> +			pr_emerg("BUG: workqueue lockup - pool");
> +			pr_cont_pool_info(pool);
> +			pr_cont(" stuck for %us!\n",
> +				jiffies_to_msecs(jiffies - pool_ts) / 1000);
> +		}
> +	}
> +
> +	rcu_read_unlock();
> +
> +	if (lockup_detected)
> +		show_workqueue_state();
> +
> +	wq_watchdog_reset_touched();
> +	mod_timer(&wq_watchdog_timer, jiffies + thresh);
> +}
> +
> +void enable_workqueue_watchdog(void)
> +{
> +	wq_watchdog_thresh = get_softlockup_thresh() * HZ;
> +
> +	wq_watchdog_reset_touched();
> +	mod_timer(&wq_watchdog_timer, jiffies + wq_watchdog_thresh);
> +}
> +
> +void disable_workqueue_watchdog(void)
> +{
> +	wq_watchdog_thresh = 0;
> +	del_timer_sync(&wq_watchdog_timer);
> +}
> +
> +void touch_workqueue_watchdog(int cpu)
> +{
> +	/*
> +	 * If not explicitly touched, these stamps are never updated, which
> +	 * means that they may affect stall detection if jiffies wraps;
> +	 * however, it's highly unlikely and the worst that can happen is
> +	 * delaying stall detection by upto one threshold duration.
> +	 */
> +	if (cpu >= 0)
> +		per_cpu(wq_watchdog_touched_cpu, cpu) = jiffies;
> +	else
> +		wq_watchdog_touched = jiffies;
> +}
> +
> +#endif	/* CONFIG_LOCKUP_DETECTOR */
> +
>  static void __init wq_numa_init(void)
>  {
>  	cpumask_var_t *tbl;
> --
> To unsubscribe from this list: send the line "unsubscribe linux-kernel" in
> the body of a message to majordomo@vger.kernel.org
> More majordomo info at  http://vger.kernel.org/majordomo-info.html
> Please read the FAQ at  http://www.tux.org/lkml/

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

* [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue
  2015-12-03 15:06         ` Tejun Heo
@ 2015-12-03 19:26           ` Tejun Heo
  2015-12-03 20:43             ` Peter Zijlstra
                               ` (3 more replies)
  0 siblings, 4 replies; 51+ messages in thread
From: Tejun Heo @ 2015-12-03 19:26 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Ulrich Obergfell, Ingo Molnar, Andrew Morton, linux-kernel, kernel-team

Task or work item involved in memory reclaim trying to flush a
non-WQ_MEM_RECLAIM workqueue or one of its work items can lead to
deadlock.  Trigger WARN_ONCE() if such conditions are detected.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Peter Zijlstra <peterz@infradead.org>
---
Hello,

So, something like this.  Seems to work fine here.  If there's no
objection, I'm gonna push it through wq/for-4.5.

Thanks.

 kernel/workqueue.c |   35 +++++++++++++++++++++++++++++++++++
 1 file changed, 35 insertions(+)

--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -2330,6 +2330,37 @@ repeat:
 	goto repeat;
 }
 
+/**
+ * check_flush_dependency - check for flush dependency sanity
+ * @target_wq: workqueue being flushed
+ * @target_work: work item being flushed (NULL for workqueue flushes)
+ *
+ * %current is trying to flush the whole @target_wq or @target_work on it.
+ * If @target_wq doesn't have %WQ_MEM_RECLAIM, verify that %current is not
+ * reclaiming memory or running on a workqueue which doesn't have
+ * %WQ_MEM_RECLAIM as that can break forward-progress guarantee leading to
+ * a deadlock.
+ */
+static void check_flush_dependency(struct workqueue_struct *target_wq,
+				   struct work_struct *target_work)
+{
+	work_func_t target_func = target_work ? target_work->func : NULL;
+	struct worker *worker;
+
+	if (target_wq->flags & WQ_MEM_RECLAIM)
+		return;
+
+	worker = current_wq_worker();
+
+	WARN_ONCE(current->flags & PF_MEMALLOC,
+		  "workqueue: PF_MEMALLOC task %d(%s) is flushing !WQ_MEM_RECLAIM %s:%pf",
+		  current->pid, current->comm, target_wq->name, target_func);
+	WARN_ONCE(worker && (worker->current_pwq->wq->flags & WQ_MEM_RECLAIM),
+		  "workqueue: WQ_MEM_RECLAIM %s:%pf is flushing !WQ_MEM_RECLAIM %s:%pf",
+		  worker->current_pwq->wq->name, worker->current_func,
+		  target_wq->name, target_func);
+}
+
 struct wq_barrier {
 	struct work_struct	work;
 	struct completion	done;
@@ -2539,6 +2570,8 @@ void flush_workqueue(struct workqueue_st
 		list_add_tail(&this_flusher.list, &wq->flusher_overflow);
 	}
 
+	check_flush_dependency(wq, NULL);
+
 	mutex_unlock(&wq->mutex);
 
 	wait_for_completion(&this_flusher.done);
@@ -2711,6 +2744,8 @@ static bool start_flush_work(struct work
 		pwq = worker->current_pwq;
 	}
 
+	check_flush_dependency(pwq->wq, work);
+
 	insert_wq_barrier(pwq, barr, work, worker);
 	spin_unlock_irq(&pool->lock);
 

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

* Re: [PATCH 2/2] workqueue: implement lockup detector
  2015-12-03 17:50   ` Don Zickus
@ 2015-12-03 19:43     ` Tejun Heo
  2015-12-03 20:12       ` Ulrich Obergfell
  0 siblings, 1 reply; 51+ messages in thread
From: Tejun Heo @ 2015-12-03 19:43 UTC (permalink / raw)
  To: Don Zickus
  Cc: Ulrich Obergfell, Ingo Molnar, Peter Zijlstra, Andrew Morton,
	linux-kernel, kernel-team

Hello, Don.

On Thu, Dec 03, 2015 at 12:50:24PM -0500, Don Zickus wrote:
> This sort of looks like the hung task detector..
> 
> I am a little concerned because we just made a big effort to properly
> separate the hardlockup and softlockup paths and yet retain the flexibility
> to enable/disable them separately.  Now it seems the workqueue detector is
> permanently entwined with the softlockup detector.  I am not entirely sure
> that is correct thing to do.

The only area they get entwined is how it's controlled from userland.
While it isn't quite the same as softlockup detection, I think what it
monitors is close enough that it makes sense to put them under the
same interface.

> It also seems awkward for the lockup code to have to jump to the workqueue
> code to function properly. :-/  Though we have made exceptions for the virt
> stuff and the workqueue code is simple..

Softlockup code doesn't depend on workqueue in any way.  Workqueue
tags on touch_softlockup to detect cases which shouldn't be warned and
its enabledness is controlled together with softlockup and that's it.

> Actually, I am curious, it seems if you just added a
> /proc/sys/kernel/wq_watchdog entry, you could elminiate the entire need for
> modifying the watchdog code to begin with.  As you really aren't using any
> of it other than piggybacking on the touch_softlockup_watchdog stuff, which
> could probably be easily added without all the extra enable/disable changes
> in watchdog.c.

Yeah, except for touch signal, it's purely interface thing.  I don't
feel too strong about this but it seems a bit silly to introduce a
whole different set of interface for this.  e.g. if the user wanted to
disable softlockup detection, it'd be weird to leave wq lockup
detection running.  The same goes for threshold.

> Again, this looks like what the hung task detector is doing, which I
> struggled with years ago to integrate with the lockup code because in the
> end I had trouble re-using much of it.

So, it's a stall detector and there are inherent similarities but the
conditions tested are pretty different and it's a lot lighter.  I'm
not really sure what you're meaning to say.

Thanks.

-- 
tejun

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

* Re: [PATCH 2/2] workqueue: implement lockup detector
  2015-12-03 19:43     ` Tejun Heo
@ 2015-12-03 20:12       ` Ulrich Obergfell
  2015-12-03 20:54         ` Tejun Heo
  0 siblings, 1 reply; 51+ messages in thread
From: Ulrich Obergfell @ 2015-12-03 20:12 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Don Zickus, Ingo Molnar, Peter Zijlstra, Andrew Morton,
	linux-kernel, kernel-team


Tejun,

I share Don's concern about connecting the soft lockup detector and the
workqueue watchdog to the same kernel parameter in /proc. I would feel
more comfortable if the workqueue watchdog had its dedicated parameter.


I also see a scenario that the proposed patch does not handle well: The
watchdog_thresh parameter can be changed 'on the fly' - i.e. it is not
necessary to disable and re-enable the watchdog. The flow of execution
looks like this.

  proc_watchdog_thresh
    proc_watchdog_update
      if (watchdog_enabled && watchdog_thresh)
          watchdog_enable_all_cpus
            if (!watchdog_running) {
                ...
            } else {
                //
                // update 'on the fly'
                //
                update_watchdog_all_cpus()
            }

The patched watchdog_enable_all_cpus() function disables the workqueue watchdog
unconditionally at [1]. However, the workqueue watchdog remains disabled if the
code path [2] is executed (and wq_watchdog_thresh is not updated as well).

static int watchdog_enable_all_cpus(void)
{
        int err = 0;

[1] --> disable_workqueue_watchdog();

        if (!watchdog_running) {
                ...
        } else {
     .-         /*
     |           * Enable/disable the lockup detectors or
     |           * change the sample period 'on the fly'.
     |           */
[2] <            err = update_watchdog_all_cpus();
     |
     |          if (err) {
     |                  watchdog_disable_all_cpus();
     |                  pr_err("Failed to update lockup detectors, disabled\n");
     '-         }
        }

        if (err)
                watchdog_enabled = 0;

        return err;
}


And another question that comes to my mind is: Would the workqueue watchdog
participate in the lockup detector suspend/resume mechanism, and if yes, how
would it be integrated into this ?


Regards,

Uli


----- Original Message -----
From: "Tejun Heo" <tj@kernel.org>
To: "Don Zickus" <dzickus@redhat.com>
Cc: "Ulrich Obergfell" <uobergfe@redhat.com>, "Ingo Molnar" <mingo@redhat.com>, "Peter Zijlstra" <peterz@infradead.org>, "Andrew Morton" <akpm@linux-foundation.org>, linux-kernel@vger.kernel.org, kernel-team@fb.com
Sent: Thursday, December 3, 2015 8:43:58 PM
Subject: Re: [PATCH 2/2] workqueue: implement lockup detector

Hello, Don.

On Thu, Dec 03, 2015 at 12:50:24PM -0500, Don Zickus wrote:
> This sort of looks like the hung task detector..
> 
> I am a little concerned because we just made a big effort to properly
> separate the hardlockup and softlockup paths and yet retain the flexibility
> to enable/disable them separately.  Now it seems the workqueue detector is
> permanently entwined with the softlockup detector.  I am not entirely sure
> that is correct thing to do.

The only area they get entwined is how it's controlled from userland.
While it isn't quite the same as softlockup detection, I think what it
monitors is close enough that it makes sense to put them under the
same interface.

> It also seems awkward for the lockup code to have to jump to the workqueue
> code to function properly. :-/  Though we have made exceptions for the virt
> stuff and the workqueue code is simple..

Softlockup code doesn't depend on workqueue in any way.  Workqueue
tags on touch_softlockup to detect cases which shouldn't be warned and
its enabledness is controlled together with softlockup and that's it.

> Actually, I am curious, it seems if you just added a
> /proc/sys/kernel/wq_watchdog entry, you could elminiate the entire need for
> modifying the watchdog code to begin with.  As you really aren't using any
> of it other than piggybacking on the touch_softlockup_watchdog stuff, which
> could probably be easily added without all the extra enable/disable changes
> in watchdog.c.

Yeah, except for touch signal, it's purely interface thing.  I don't
feel too strong about this but it seems a bit silly to introduce a
whole different set of interface for this.  e.g. if the user wanted to
disable softlockup detection, it'd be weird to leave wq lockup
detection running.  The same goes for threshold.

> Again, this looks like what the hung task detector is doing, which I
> struggled with years ago to integrate with the lockup code because in the
> end I had trouble re-using much of it.

So, it's a stall detector and there are inherent similarities but the
conditions tested are pretty different and it's a lot lighter.  I'm
not really sure what you're meaning to say.

Thanks.

-- 
tejun

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

* Re: [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue
  2015-12-03 19:26           ` [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue Tejun Heo
@ 2015-12-03 20:43             ` Peter Zijlstra
  2015-12-03 20:56               ` Tejun Heo
  2015-12-07 15:58             ` Tejun Heo
                               ` (2 subsequent siblings)
  3 siblings, 1 reply; 51+ messages in thread
From: Peter Zijlstra @ 2015-12-03 20:43 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Ulrich Obergfell, Ingo Molnar, Andrew Morton, linux-kernel, kernel-team

On Thu, Dec 03, 2015 at 02:26:16PM -0500, Tejun Heo wrote:
> +	WARN_ONCE(current->flags & PF_MEMALLOC,

I'm not sure about using PF_MEMALLOC for detecting reclaim. There appear
to be more sites setting this than reclaim. See:

drivers/block/nbd.c:	current->flags |= PF_MEMALLOC;
drivers/mmc/card/queue.c:	current->flags |= PF_MEMALLOC;
drivers/mtd/nand/nandsim.c:	current->flags |= PF_MEMALLOC;
drivers/scsi/iscsi_tcp.c:	current->flags |= PF_MEMALLOC;
drivers/staging/lustre/include/linux/libcfs/linux/linux-mem.h:#define memory_pressure_set() do { current->flags |= PF_MEMALLOC; } while (0)
fs/cifs/connect.c:	current->flags |= PF_MEMALLOC;
fs/xfs/libxfs/xfs_btree.c:		new_pflags |= PF_MEMALLOC | PF_SWAPWRITE | PF_KSWAPD;
fs/xfs/xfs_trans_ail.c:	current->flags |= PF_MEMALLOC;
include/linux/sched.h:	current->flags |= PF_MEMALLOC_NOIO;
mm/page_alloc.c:	current->flags |= PF_MEMALLOC;
mm/page_alloc.c:	current->flags |= PF_MEMALLOC;
mm/vmscan.c:	tsk->flags |= PF_MEMALLOC | PF_SWAPWRITE | PF_KSWAPD;
mm/vmscan.c:	p->flags |= PF_MEMALLOC;
mm/vmscan.c:	p->flags |= PF_MEMALLOC | PF_SWAPWRITE;
net/core/dev.c:		current->flags |= PF_MEMALLOC;
net/core/sock.c:	current->flags |= PF_MEMALLOC;


The actual reclaim sites in page_alloc and vmscan set
current->reclaim_state. So testing against that might be more accurate.

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

* Re: [PATCH 2/2] workqueue: implement lockup detector
  2015-12-03 20:12       ` Ulrich Obergfell
@ 2015-12-03 20:54         ` Tejun Heo
  2015-12-04  8:02           ` Ingo Molnar
  2015-12-04 13:19           ` Ulrich Obergfell
  0 siblings, 2 replies; 51+ messages in thread
From: Tejun Heo @ 2015-12-03 20:54 UTC (permalink / raw)
  To: Ulrich Obergfell
  Cc: Don Zickus, Ingo Molnar, Peter Zijlstra, Andrew Morton,
	linux-kernel, kernel-team

Hello, Ulrich.

On Thu, Dec 03, 2015 at 03:12:20PM -0500, Ulrich Obergfell wrote:
> I share Don's concern about connecting the soft lockup detector and the
> workqueue watchdog to the same kernel parameter in /proc. I would feel
> more comfortable if the workqueue watchdog had its dedicated parameter.

Sure, separating the knobs out isn't difficult.  I still don't like
the idea of having multiple set of similar knobs controlling about the
same thing tho.

For example, let's say there's a user who boots with "nosoftlockup"
explicitly.  I'm pretty sure the user wouldn't be intending to keep
workqueue watchdog running.  The same goes for threshold adjustments,
so here's my question.  What are the reasons for the concern?  What
are we worrying about?

> The patched watchdog_enable_all_cpus() function disables the workqueue watchdog
> unconditionally at [1]. However, the workqueue watchdog remains disabled if the
> code path [2] is executed (and wq_watchdog_thresh is not updated as well).

Oops, you're right.

> And another question that comes to my mind is: Would the workqueue watchdog
> participate in the lockup detector suspend/resume mechanism, and if yes, how
> would it be integrated into this ?

>From the usage, I can't quite tell what the purpose of the mechanism
is.  The only user seems to be fixup_ht_bug() and when it fails it
says "failed to disable PMU erratum BJ122, BV98, HSD29 workaround" so
if you could give me a pointer, it'd be great.  But at any rate, if
shutting down watchdog is all that's necessary, it shouldn't be a
problem to integrate.

Thanks.

-- 
tejun

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

* Re: [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue
  2015-12-03 20:43             ` Peter Zijlstra
@ 2015-12-03 20:56               ` Tejun Heo
  2015-12-03 21:09                 ` Peter Zijlstra
  0 siblings, 1 reply; 51+ messages in thread
From: Tejun Heo @ 2015-12-03 20:56 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Ulrich Obergfell, Ingo Molnar, Andrew Morton, linux-kernel, kernel-team

Hey, Peter.

On Thu, Dec 03, 2015 at 09:43:13PM +0100, Peter Zijlstra wrote:
> I'm not sure about using PF_MEMALLOC for detecting reclaim. There appear
> to be more sites setting this than reclaim. See:
>
> drivers/block/nbd.c:	current->flags |= PF_MEMALLOC;
> drivers/mmc/card/queue.c:	current->flags |= PF_MEMALLOC;
> drivers/mtd/nand/nandsim.c:	current->flags |= PF_MEMALLOC;
> drivers/scsi/iscsi_tcp.c:	current->flags |= PF_MEMALLOC;
> drivers/staging/lustre/include/linux/libcfs/linux/linux-mem.h:#define memory_pressure_set() do { current->flags |= PF_MEMALLOC; } while (0)
> fs/cifs/connect.c:	current->flags |= PF_MEMALLOC;
> fs/xfs/libxfs/xfs_btree.c:		new_pflags |= PF_MEMALLOC | PF_SWAPWRITE | PF_KSWAPD;
> fs/xfs/xfs_trans_ail.c:	current->flags |= PF_MEMALLOC;
> include/linux/sched.h:	current->flags |= PF_MEMALLOC_NOIO;
> mm/page_alloc.c:	current->flags |= PF_MEMALLOC;
> mm/page_alloc.c:	current->flags |= PF_MEMALLOC;
> mm/vmscan.c:	tsk->flags |= PF_MEMALLOC | PF_SWAPWRITE | PF_KSWAPD;
> mm/vmscan.c:	p->flags |= PF_MEMALLOC;
> mm/vmscan.c:	p->flags |= PF_MEMALLOC | PF_SWAPWRITE;
> net/core/dev.c:		current->flags |= PF_MEMALLOC;
> net/core/sock.c:	current->flags |= PF_MEMALLOC;
> 
> 
> The actual reclaim sites in page_alloc and vmscan set
> current->reclaim_state. So testing against that might be more accurate.

So, if I'm not mistaken, those are all marking tasks which can be
depended upon during memory reclaim and we do want to catch them all.
PF_MEMALLOC shouldn't depend on something which require memory to be
reclaimed to guarantee forward progress.

Thanks.

-- 
tejun

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

* Re: [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue
  2015-12-03 20:56               ` Tejun Heo
@ 2015-12-03 21:09                 ` Peter Zijlstra
  2015-12-03 22:04                   ` Tejun Heo
  0 siblings, 1 reply; 51+ messages in thread
From: Peter Zijlstra @ 2015-12-03 21:09 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Ulrich Obergfell, Ingo Molnar, Andrew Morton, linux-kernel, kernel-team

On Thu, Dec 03, 2015 at 03:56:32PM -0500, Tejun Heo wrote:
> So, if I'm not mistaken, those are all marking tasks which can be
> depended upon during memory reclaim and we do want to catch them all.

Up to a point yes, these are things that want to be reliable during
reclaim, but lacking memory reserves and usage bounds (which we
discussed last at lsf/mm) these are just wanna-be.

> PF_MEMALLOC shouldn't depend on something which require memory to be
> reclaimed to guarantee forward progress.

PF_MEMALLOC basically avoids reclaim for any memory allocation while its
set.

The thing is, even if your workqueue has WQ_MEM_RECLAIM set, it will not
hit the mayday button until you're completely full flat out of memory.
At which point you're probably boned anyway, because, as per the above,
all that code assumes there's _some_ memory to be had.

One solution is to always fail maybe_create_worker() when PF_MEMALLOC is
set, thus always hitting the mayday button.

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

* Re: [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue
  2015-12-03 21:09                 ` Peter Zijlstra
@ 2015-12-03 22:04                   ` Tejun Heo
  2015-12-04 12:51                     ` Peter Zijlstra
  0 siblings, 1 reply; 51+ messages in thread
From: Tejun Heo @ 2015-12-03 22:04 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Ulrich Obergfell, Ingo Molnar, Andrew Morton, linux-kernel, kernel-team

Hello, Peter.

On Thu, Dec 03, 2015 at 10:09:11PM +0100, Peter Zijlstra wrote:
> On Thu, Dec 03, 2015 at 03:56:32PM -0500, Tejun Heo wrote:
> > So, if I'm not mistaken, those are all marking tasks which can be
> > depended upon during memory reclaim and we do want to catch them all.
> 
> Up to a point yes, these are things that want to be reliable during
> reclaim, but lacking memory reserves and usage bounds (which we
> discussed last at lsf/mm) these are just wanna-be.

Hmmm... even if buggy in that they can't guarantee forward-progress
even with access to the emergency pool, I think it makes sense to warn
them about creating an extra dependency which doesn't have access to
the emergency pool.

> > PF_MEMALLOC shouldn't depend on something which require memory to be
> > reclaimed to guarantee forward progress.
> 
> PF_MEMALLOC basically avoids reclaim for any memory allocation while its
> set.

So, the assumption is that they're already on the reclaim path and
thus shouldn't recurse into it again.

> The thing is, even if your workqueue has WQ_MEM_RECLAIM set, it will not
> hit the mayday button until you're completely full flat out of memory.

It's more trigger-happy than that.  It's timer based.  If new worker
can't be created for a certain amount of time for whatever reason,
it'll summon the rescuer.

> At which point you're probably boned anyway, because, as per the above,
> all that code assumes there's _some_ memory to be had.

Not really.  PF_MEMALLOC tasks have access to the emergency pool,
creating new workers doesn't, so this really is creating a dependency
which is qualitatively different.

> One solution is to always fail maybe_create_worker() when PF_MEMALLOC is
> set, thus always hitting the mayday button.

I'm not following.  When PF_MEMALLOC is set where?

Thanks.

-- 
tejun

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

* Re: [PATCH 2/2] workqueue: implement lockup detector
  2015-12-03 20:54         ` Tejun Heo
@ 2015-12-04  8:02           ` Ingo Molnar
  2015-12-04 16:52             ` Don Zickus
  2015-12-04 13:19           ` Ulrich Obergfell
  1 sibling, 1 reply; 51+ messages in thread
From: Ingo Molnar @ 2015-12-04  8:02 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Ulrich Obergfell, Don Zickus, Ingo Molnar, Peter Zijlstra,
	Andrew Morton, linux-kernel, kernel-team


* Tejun Heo <tj@kernel.org> wrote:

> Hello, Ulrich.
> 
> On Thu, Dec 03, 2015 at 03:12:20PM -0500, Ulrich Obergfell wrote:
> > I share Don's concern about connecting the soft lockup detector and the
> > workqueue watchdog to the same kernel parameter in /proc. I would feel
> > more comfortable if the workqueue watchdog had its dedicated parameter.
> 
> Sure, separating the knobs out isn't difficult.  I still don't like
> the idea of having multiple set of similar knobs controlling about the
> same thing tho.
> 
> For example, let's say there's a user who boots with "nosoftlockup"
> explicitly.  I'm pretty sure the user wouldn't be intending to keep
> workqueue watchdog running.  The same goes for threshold adjustments,
> so here's my question.  What are the reasons for the concern?  What
> are we worrying about?

As Don mentioned it already, we went through similar arguments (and pain) with the 
hard/soft lockup detectors and its various control knobs, it would be better to 
have new control knobs separated.

As for the ease of use argument, we can add a new, obviously named control knob 
that controls _all_ lockup detectors:

  boot param: nolockupdetectors
  matching Kconfig knob: CONFIG_BOOTPARAM_NO_LOCKUP_DETECTORS=0

but please don't artificially couple the control knobs of these various lockup 
detectors, as these internal assumptions are less than obvious to users. With 
(effectively) 4 lockup detectors such coupling of interfaces is even more 
confusing and damaging.

but ease of use is not a big concern in any case, since new debug features are 
disabled by default, so only those people will see it (and may want to disable it 
via a boot parameter, hopefully only temporarily) who enable it intentionally.

Thanks,

	Ingo

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

* Re: [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue
  2015-12-03 22:04                   ` Tejun Heo
@ 2015-12-04 12:51                     ` Peter Zijlstra
  0 siblings, 0 replies; 51+ messages in thread
From: Peter Zijlstra @ 2015-12-04 12:51 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Ulrich Obergfell, Ingo Molnar, Andrew Morton, linux-kernel, kernel-team

On Thu, Dec 03, 2015 at 05:04:06PM -0500, Tejun Heo wrote:
> > One solution is to always fail maybe_create_worker() when PF_MEMALLOC is
> > set, thus always hitting the mayday button.
> 
> I'm not following.  When PF_MEMALLOC is set where?

It seems I made a false assumption. I was thinking the worker creation
was done from queue/flush context, but its done by other workers, at a
time when PF_MEMALLOC cannot be set.

In any case, no objections to the proposed patch.

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

* Re: [PATCH 2/2] workqueue: implement lockup detector
  2015-12-03 20:54         ` Tejun Heo
  2015-12-04  8:02           ` Ingo Molnar
@ 2015-12-04 13:19           ` Ulrich Obergfell
  1 sibling, 0 replies; 51+ messages in thread
From: Ulrich Obergfell @ 2015-12-04 13:19 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Don Zickus, Ingo Molnar, Peter Zijlstra, Andrew Morton,
	linux-kernel, kernel-team


Tejun,

> Sure, separating the knobs out isn't difficult.  I still don't like
> the idea of having multiple set of similar knobs controlling about the
> same thing tho.
>
> For example, let's say there's a user who boots with "nosoftlockup"
> explicitly.  I'm pretty sure the user wouldn't be intending to keep
> workqueue watchdog running.  The same goes for threshold adjustments,
> so here's my question.  What are the reasons for the concern?  What
> are we worrying about?

I'm not sure if it is obvious to a user that a stall of workqueues is
"about the same thing" as a soft lockup, and that one could thus argue
that both should be controlled by the same knob. Looking at this from
perspective of usability, I would still vote for having separate knobs
for each lockup detector. For example

  /proc/sys/kernel/wq_watchdog_thresh

could control the on|off state of the workqueue watchdog and the timeout
at the same time (0 means off, > 0 means on and specifies the timeout).
Separating wq_watchdog_thresh from watchdog_thresh might also be useful
for diagnostic purposes for example, if during the investigation of a
problem one would want to explicitly increase or lower one threshold
without impacting the other.


>> And another question that comes to my mind is: Would the workqueue watchdog
>> participate in the lockup detector suspend/resume mechanism, and if yes, how
>> would it be integrated into this ?
>
> From the usage, I can't quite tell what the purpose of the mechanism
> is.  The only user seems to be fixup_ht_bug() and when it fails it
> says "failed to disable PMU erratum BJ122, BV98, HSD29 workaround" so
> if you could give me a pointer, it'd be great.  But at any rate, if
> shutting down watchdog is all that's necessary, it shouldn't be a
> problem to integrate.

The patch post that introduced the mechanism is here:

  http://marc.info/?l=linux-kernel&m=143843318208917&w=2

The watchdog_{suspend|resume} functions were later renamed:

  http://marc.info/?l=linux-kernel&m=143894132129982&w=2

At the moment I don't see a reason why the workqueue watchdog would have to
participate in that mechanism. However, if the workqueue watchdog would be
connected to the soft lockup detector as you proposed, I think it should be
participating for the 'sake of consistency' (it would seem hard to under-
stand if the interface would only suspend parts of the lockup detector).


Regards,

Uli

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

* Re: [PATCH 2/2] workqueue: implement lockup detector
  2015-12-04  8:02           ` Ingo Molnar
@ 2015-12-04 16:52             ` Don Zickus
  0 siblings, 0 replies; 51+ messages in thread
From: Don Zickus @ 2015-12-04 16:52 UTC (permalink / raw)
  To: Ingo Molnar
  Cc: Tejun Heo, Ulrich Obergfell, Ingo Molnar, Peter Zijlstra,
	Andrew Morton, linux-kernel, kernel-team

On Fri, Dec 04, 2015 at 09:02:26AM +0100, Ingo Molnar wrote:
> 
> * Tejun Heo <tj@kernel.org> wrote:
> 
> > Hello, Ulrich.
> > 
> > On Thu, Dec 03, 2015 at 03:12:20PM -0500, Ulrich Obergfell wrote:
> > > I share Don's concern about connecting the soft lockup detector and the
> > > workqueue watchdog to the same kernel parameter in /proc. I would feel
> > > more comfortable if the workqueue watchdog had its dedicated parameter.
> > 
> > Sure, separating the knobs out isn't difficult.  I still don't like
> > the idea of having multiple set of similar knobs controlling about the
> > same thing tho.
> > 
> > For example, let's say there's a user who boots with "nosoftlockup"
> > explicitly.  I'm pretty sure the user wouldn't be intending to keep
> > workqueue watchdog running.  The same goes for threshold adjustments,
> > so here's my question.  What are the reasons for the concern?  What
> > are we worrying about?
> 
> As Don mentioned it already, we went through similar arguments (and pain) with the 
> hard/soft lockup detectors and its various control knobs, it would be better to 
> have new control knobs separated.
> 
> As for the ease of use argument, we can add a new, obviously named control knob 
> that controls _all_ lockup detectors:
> 
>   boot param: nolockupdetectors
>   matching Kconfig knob: CONFIG_BOOTPARAM_NO_LOCKUP_DETECTORS=0
> 
> but please don't artificially couple the control knobs of these various lockup 
> detectors, as these internal assumptions are less than obvious to users. With 
> (effectively) 4 lockup detectors such coupling of interfaces is even more 
> confusing and damaging.

It might be worth tying them together with a generic knob and expanding the
bit mask for the 'watchdog' variable.  I can't figure out an easy way to do
that right now.

I don't think we want to go down the route of 'registering' detectors yet.
:-)

Cheers,
Don

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

* Re: [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue
  2015-12-03 19:26           ` [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue Tejun Heo
  2015-12-03 20:43             ` Peter Zijlstra
@ 2015-12-07 15:58             ` Tejun Heo
       [not found]             ` <20151203192616.GJ27463-qYNAdHglDFBN0TnZuCh8vA@public.gmane.org>
  2016-03-10 15:12             ` [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue Adrian Hunter
  3 siblings, 0 replies; 51+ messages in thread
From: Tejun Heo @ 2015-12-07 15:58 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Ulrich Obergfell, Ingo Molnar, Andrew Morton, linux-kernel, kernel-team

On Thu, Dec 03, 2015 at 02:26:16PM -0500, Tejun Heo wrote:
> Task or work item involved in memory reclaim trying to flush a
> non-WQ_MEM_RECLAIM workqueue or one of its work items can lead to
> deadlock.  Trigger WARN_ONCE() if such conditions are detected.
> 
> Signed-off-by: Tejun Heo <tj@kernel.org>
> Cc: Peter Zijlstra <peterz@infradead.org>

Applied to wq/for-4.5.

-- 
tejun

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

* [PATCH v2 2/2] workqueue: implement lockup detector
  2015-12-03  0:28 ` [PATCH 2/2] workqueue: implement lockup detector Tejun Heo
  2015-12-03 14:49   ` Tejun Heo
  2015-12-03 17:50   ` Don Zickus
@ 2015-12-07 19:06   ` Tejun Heo
  2015-12-07 21:38     ` Don Zickus
  2 siblings, 1 reply; 51+ messages in thread
From: Tejun Heo @ 2015-12-07 19:06 UTC (permalink / raw)
  To: Ulrich Obergfell, Ingo Molnar, Peter Zijlstra, Andrew Morton
  Cc: linux-kernel, kernel-team

Hello,

Decoupled the control knobs from softlockup.  It's now workqueue
module param which can be updated at runtime.  If there's no
objection, I'll push the two patches through wq/for-4.5.

Thanks.
------ 8< ------
Workqueue stalls can happen from a variety of usage bugs such as
missing WQ_MEM_RECLAIM flag or concurrency managed work item
indefinitely staying RUNNING.  These stalls can be extremely difficult
to hunt down because the usual warning mechanisms can't detect
workqueue stalls and the internal state is pretty opaque.

To alleviate the situation, this patch implements workqueue lockup
detector.  It periodically monitors all worker_pools periodically and,
if any pool failed to make forward progress longer than the threshold
duration, triggers warning and dumps workqueue state as follows.

 BUG: workqueue lockup - pool cpus=0 node=0 flags=0x0 nice=0 stuck for 31s!
 Showing busy workqueues and worker pools:
 workqueue events: flags=0x0
   pwq 0: cpus=0 node=0 flags=0x0 nice=0 active=17/256
     pending: monkey_wrench_fn, e1000_watchdog, cache_reap, vmstat_shepherd, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, cgroup_release_agent
 workqueue events_power_efficient: flags=0x80
   pwq 0: cpus=0 node=0 flags=0x0 nice=0 active=2/256
     pending: check_lifetime, neigh_periodic_work
 workqueue cgroup_pidlist_destroy: flags=0x0
   pwq 0: cpus=0 node=0 flags=0x0 nice=0 active=1/1
     pending: cgroup_pidlist_destroy_work_fn
 ...

The detection mechanism is controller through kernel parameter
workqueue.watchdog_thresh and can be updated at runtime through the
sysfs module parameter file.

v2: Decoupled from softlockup control knobs.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Ulrich Obergfell <uobergfe@redhat.com>
Cc: Michal Hocko <mhocko@suse.com>
Cc: Chris Mason <clm@fb.com>
Cc: Andrew Morton <akpm@linux-foundation.org>
---
 Documentation/kernel-parameters.txt |    9 +
 include/linux/workqueue.h           |    6 +
 kernel/watchdog.c                   |    3 
 kernel/workqueue.c                  |  174 +++++++++++++++++++++++++++++++++++-
 lib/Kconfig.debug                   |   11 ++
 5 files changed, 200 insertions(+), 3 deletions(-)

--- a/Documentation/kernel-parameters.txt
+++ b/Documentation/kernel-parameters.txt
@@ -4114,6 +4114,15 @@ bytes respectively. Such letter suffixes
 			or other driver-specific files in the
 			Documentation/watchdog/ directory.
 
+	workqueue.watchdog_thresh=
+			If CONFIG_WQ_WATCHDOG is configured, workqueue can
+			warn stall conditions and dump internal state to
+			help debugging.  0 disables workqueue stall
+			detection; otherwise, it's the stall threshold
+			duration in seconds.  The default value is 30 and
+			it can be updated at runtime by writing to the
+			corresponding sysfs file.
+
 	workqueue.disable_numa
 			By default, all work items queued to unbound
 			workqueues are affine to the NUMA nodes they're
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -618,4 +618,10 @@ static inline int workqueue_sysfs_regist
 { return 0; }
 #endif	/* CONFIG_SYSFS */
 
+#ifdef CONFIG_WQ_WATCHDOG
+void wq_watchdog_touch(int cpu);
+#else	/* CONFIG_WQ_WATCHDOG */
+static inline void wq_watchdog_touch(int cpu) { }
+#endif	/* CONFIG_WQ_WATCHDOG */
+
 #endif
--- a/kernel/watchdog.c
+++ b/kernel/watchdog.c
@@ -20,6 +20,7 @@
 #include <linux/smpboot.h>
 #include <linux/sched/rt.h>
 #include <linux/tick.h>
+#include <linux/workqueue.h>
 
 #include <asm/irq_regs.h>
 #include <linux/kvm_para.h>
@@ -245,6 +246,7 @@ void touch_softlockup_watchdog_sched(voi
 void touch_softlockup_watchdog(void)
 {
 	touch_softlockup_watchdog_sched();
+	wq_watchdog_touch(raw_smp_processor_id());
 }
 EXPORT_SYMBOL(touch_softlockup_watchdog);
 
@@ -259,6 +261,7 @@ void touch_all_softlockup_watchdogs(void
 	 */
 	for_each_watchdog_cpu(cpu)
 		per_cpu(watchdog_touch_ts, cpu) = 0;
+	wq_watchdog_touch(-1);
 }
 
 #ifdef CONFIG_HARDLOCKUP_DETECTOR
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -148,6 +148,8 @@ struct worker_pool {
 	int			id;		/* I: pool ID */
 	unsigned int		flags;		/* X: flags */
 
+	unsigned long		watchdog_ts;	/* L: watchdog timestamp */
+
 	struct list_head	worklist;	/* L: list of pending works */
 	int			nr_workers;	/* L: total number of workers */
 
@@ -1083,6 +1085,8 @@ static void pwq_activate_delayed_work(st
 	struct pool_workqueue *pwq = get_work_pwq(work);
 
 	trace_workqueue_activate_work(work);
+	if (list_empty(&pwq->pool->worklist))
+		pwq->pool->watchdog_ts = jiffies;
 	move_linked_works(work, &pwq->pool->worklist, NULL);
 	__clear_bit(WORK_STRUCT_DELAYED_BIT, work_data_bits(work));
 	pwq->nr_active++;
@@ -1385,6 +1389,8 @@ retry:
 		trace_workqueue_activate_work(work);
 		pwq->nr_active++;
 		worklist = &pwq->pool->worklist;
+		if (list_empty(worklist))
+			pwq->pool->watchdog_ts = jiffies;
 	} else {
 		work_flags |= WORK_STRUCT_DELAYED;
 		worklist = &pwq->delayed_works;
@@ -2157,6 +2163,8 @@ recheck:
 			list_first_entry(&pool->worklist,
 					 struct work_struct, entry);
 
+		pool->watchdog_ts = jiffies;
+
 		if (likely(!(*work_data_bits(work) & WORK_STRUCT_LINKED))) {
 			/* optimization path, not strictly necessary */
 			process_one_work(worker, work);
@@ -2240,6 +2248,7 @@ repeat:
 					struct pool_workqueue, mayday_node);
 		struct worker_pool *pool = pwq->pool;
 		struct work_struct *work, *n;
+		bool first = true;
 
 		__set_current_state(TASK_RUNNING);
 		list_del_init(&pwq->mayday_node);
@@ -2256,9 +2265,14 @@ repeat:
 		 * process'em.
 		 */
 		WARN_ON_ONCE(!list_empty(scheduled));
-		list_for_each_entry_safe(work, n, &pool->worklist, entry)
-			if (get_work_pwq(work) == pwq)
+		list_for_each_entry_safe(work, n, &pool->worklist, entry) {
+			if (get_work_pwq(work) == pwq) {
+				if (first)
+					pool->watchdog_ts = jiffies;
 				move_linked_works(work, scheduled, &n);
+			}
+			first = false;
+		}
 
 		if (!list_empty(scheduled)) {
 			process_scheduled_works(rescuer);
@@ -3104,6 +3118,7 @@ static int init_worker_pool(struct worke
 	pool->cpu = -1;
 	pool->node = NUMA_NO_NODE;
 	pool->flags |= POOL_DISASSOCIATED;
+	pool->watchdog_ts = jiffies;
 	INIT_LIST_HEAD(&pool->worklist);
 	INIT_LIST_HEAD(&pool->idle_list);
 	hash_init(pool->busy_hash);
@@ -4343,7 +4358,9 @@ void show_workqueue_state(void)
 
 		pr_info("pool %d:", pool->id);
 		pr_cont_pool_info(pool);
-		pr_cont(" workers=%d", pool->nr_workers);
+		pr_cont(" hung=%us workers=%d",
+			jiffies_to_msecs(jiffies - pool->watchdog_ts) / 1000,
+			pool->nr_workers);
 		if (pool->manager)
 			pr_cont(" manager: %d",
 				task_pid_nr(pool->manager->task));
@@ -5202,6 +5219,154 @@ static void workqueue_sysfs_unregister(s
 static void workqueue_sysfs_unregister(struct workqueue_struct *wq)	{ }
 #endif	/* CONFIG_SYSFS */
 
+/*
+ * Workqueue watchdog.
+ *
+ * Stall may be caused by various bugs - missing WQ_MEM_RECLAIM, illegal
+ * flush dependency, a concurrency managed work item which stays RUNNING
+ * indefinitely.  Workqueue stalls can be very difficult to debug as the
+ * usual warning mechanisms don't trigger and internal workqueue state is
+ * largely opaque.
+ *
+ * Workqueue watchdog monitors all worker pools periodically and dumps
+ * state if some pools failed to make forward progress for a while where
+ * forward progress is defined as the first item on ->worklist changing.
+ *
+ * This mechanism is controlled through the kernel parameter
+ * "workqueue.watchdog_thresh" which can be updated at runtime through the
+ * corresponding sysfs parameter file.
+ */
+#ifdef CONFIG_WQ_WATCHDOG
+
+static void wq_watchdog_timer_fn(unsigned long data);
+
+static unsigned long wq_watchdog_thresh = 30;
+static struct timer_list wq_watchdog_timer =
+	TIMER_DEFERRED_INITIALIZER(wq_watchdog_timer_fn, 0, 0);
+
+static unsigned long wq_watchdog_touched = INITIAL_JIFFIES;
+static DEFINE_PER_CPU(unsigned long, wq_watchdog_touched_cpu) = INITIAL_JIFFIES;
+
+static void wq_watchdog_reset_touched(void)
+{
+	int cpu;
+
+	wq_watchdog_touched = jiffies;
+	for_each_possible_cpu(cpu)
+		per_cpu(wq_watchdog_touched_cpu, cpu) = jiffies;
+}
+
+static void wq_watchdog_timer_fn(unsigned long data)
+{
+	unsigned long thresh = READ_ONCE(wq_watchdog_thresh) * HZ;
+	bool lockup_detected = false;
+	struct worker_pool *pool;
+	int pi;
+
+	if (!thresh)
+		return;
+
+	rcu_read_lock();
+
+	for_each_pool(pool, pi) {
+		unsigned long pool_ts, touched, ts;
+
+		if (list_empty(&pool->worklist))
+			continue;
+
+		/* get the latest of pool and touched timestamps */
+		pool_ts = READ_ONCE(pool->watchdog_ts);
+		touched = READ_ONCE(wq_watchdog_touched);
+
+		if (time_after(pool_ts, touched))
+			ts = pool_ts;
+		else
+			ts = touched;
+
+		if (pool->cpu >= 0) {
+			unsigned long cpu_touched =
+				READ_ONCE(per_cpu(wq_watchdog_touched_cpu,
+						  pool->cpu));
+			if (time_after(cpu_touched, ts))
+				ts = cpu_touched;
+		}
+
+		/* did we stall? */
+		if (time_after(jiffies, ts + thresh)) {
+			lockup_detected = true;
+			pr_emerg("BUG: workqueue lockup - pool");
+			pr_cont_pool_info(pool);
+			pr_cont(" stuck for %us!\n",
+				jiffies_to_msecs(jiffies - pool_ts) / 1000);
+		}
+	}
+
+	rcu_read_unlock();
+
+	if (lockup_detected)
+		show_workqueue_state();
+
+	wq_watchdog_reset_touched();
+	mod_timer(&wq_watchdog_timer, jiffies + thresh);
+}
+
+void wq_watchdog_touch(int cpu)
+{
+	if (cpu >= 0)
+		per_cpu(wq_watchdog_touched_cpu, cpu) = jiffies;
+	else
+		wq_watchdog_touched = jiffies;
+}
+
+static void wq_watchdog_set_thresh(unsigned long thresh)
+{
+	wq_watchdog_thresh = 0;
+	del_timer_sync(&wq_watchdog_timer);
+
+	if (thresh) {
+		wq_watchdog_thresh = thresh;
+		wq_watchdog_reset_touched();
+		mod_timer(&wq_watchdog_timer, jiffies + thresh * HZ);
+	}
+}
+
+static int wq_watchdog_param_set_thresh(const char *val,
+					const struct kernel_param *kp)
+{
+	unsigned long thresh;
+	int ret;
+
+	ret = kstrtoul(val, 0, &thresh);
+	if (ret)
+		return ret;
+
+	if (system_wq)
+		wq_watchdog_set_thresh(thresh);
+	else
+		wq_watchdog_thresh = thresh;
+
+	return 0;
+}
+
+static const struct kernel_param_ops wq_watchdog_thresh_ops = {
+	.set	= wq_watchdog_param_set_thresh,
+	.get	= param_get_ulong,
+};
+
+module_param_cb(watchdog_thresh, &wq_watchdog_thresh_ops, &wq_watchdog_thresh,
+		0644);
+
+static void wq_watchdog_init(void)
+{
+	wq_watchdog_set_thresh(wq_watchdog_thresh);
+}
+
+#else	/* CONFIG_WQ_WATCHDOG */
+
+static inline void wq_watchdog_init(void) { }
+
+#endif	/* CONFIG_WQ_WATCHDOG */
+
 static void __init wq_numa_init(void)
 {
 	cpumask_var_t *tbl;
@@ -5325,6 +5490,9 @@ static int __init init_workqueues(void)
 	       !system_unbound_wq || !system_freezable_wq ||
 	       !system_power_efficient_wq ||
 	       !system_freezable_power_efficient_wq);
+
+	wq_watchdog_init();
+
 	return 0;
 }
 early_initcall(init_workqueues);
--- a/lib/Kconfig.debug
+++ b/lib/Kconfig.debug
@@ -812,6 +812,17 @@ config BOOTPARAM_HUNG_TASK_PANIC_VALUE
 	default 0 if !BOOTPARAM_HUNG_TASK_PANIC
 	default 1 if BOOTPARAM_HUNG_TASK_PANIC
 
+config WQ_WATCHDOG
+	bool "Detect Workqueue Stalls"
+	depends on DEBUG_KERNEL
+	help
+	  Say Y here to enable stall detection on workqueues.  If a
+	  worker pool doesn't make forward progress on a pending work
+	  item for over a given amount of time, 30s by default, a
+	  warning message is printed along with dump of workqueue
+	  state.  This can be configured through kernel parameter
+	  "workqueue.watchdog_thresh" and its sysfs counterpart.
+
 endmenu # "Debug lockups and hangs"
 
 config PANIC_ON_OOPS

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

* Re: [PATCH v2 2/2] workqueue: implement lockup detector
  2015-12-07 19:06   ` [PATCH v2 " Tejun Heo
@ 2015-12-07 21:38     ` Don Zickus
  2015-12-07 21:39       ` Tejun Heo
  0 siblings, 1 reply; 51+ messages in thread
From: Don Zickus @ 2015-12-07 21:38 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Ulrich Obergfell, Ingo Molnar, Peter Zijlstra, Andrew Morton,
	linux-kernel, kernel-team

On Mon, Dec 07, 2015 at 02:06:17PM -0500, Tejun Heo wrote:
> Hello,
> 
> Decoupled the control knobs from softlockup.  It's now workqueue
> module param which can be updated at runtime.  If there's no
> objection, I'll push the two patches through wq/for-4.5.

Does this still compile correctly with CONFIG_WQ_WATCHDOG disabled?

Cheers,
Don

> 
> Thanks.
> ------ 8< ------
> Workqueue stalls can happen from a variety of usage bugs such as
> missing WQ_MEM_RECLAIM flag or concurrency managed work item
> indefinitely staying RUNNING.  These stalls can be extremely difficult
> to hunt down because the usual warning mechanisms can't detect
> workqueue stalls and the internal state is pretty opaque.
> 
> To alleviate the situation, this patch implements workqueue lockup
> detector.  It periodically monitors all worker_pools periodically and,
> if any pool failed to make forward progress longer than the threshold
> duration, triggers warning and dumps workqueue state as follows.
> 
>  BUG: workqueue lockup - pool cpus=0 node=0 flags=0x0 nice=0 stuck for 31s!
>  Showing busy workqueues and worker pools:
>  workqueue events: flags=0x0
>    pwq 0: cpus=0 node=0 flags=0x0 nice=0 active=17/256
>      pending: monkey_wrench_fn, e1000_watchdog, cache_reap, vmstat_shepherd, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, cgroup_release_agent
>  workqueue events_power_efficient: flags=0x80
>    pwq 0: cpus=0 node=0 flags=0x0 nice=0 active=2/256
>      pending: check_lifetime, neigh_periodic_work
>  workqueue cgroup_pidlist_destroy: flags=0x0
>    pwq 0: cpus=0 node=0 flags=0x0 nice=0 active=1/1
>      pending: cgroup_pidlist_destroy_work_fn
>  ...
> 
> The detection mechanism is controller through kernel parameter
> workqueue.watchdog_thresh and can be updated at runtime through the
> sysfs module parameter file.
> 
> v2: Decoupled from softlockup control knobs.
> 
> Signed-off-by: Tejun Heo <tj@kernel.org>
> Cc: Ulrich Obergfell <uobergfe@redhat.com>
> Cc: Michal Hocko <mhocko@suse.com>
> Cc: Chris Mason <clm@fb.com>
> Cc: Andrew Morton <akpm@linux-foundation.org>
> ---
>  Documentation/kernel-parameters.txt |    9 +
>  include/linux/workqueue.h           |    6 +
>  kernel/watchdog.c                   |    3 
>  kernel/workqueue.c                  |  174 +++++++++++++++++++++++++++++++++++-
>  lib/Kconfig.debug                   |   11 ++
>  5 files changed, 200 insertions(+), 3 deletions(-)
> 
> --- a/Documentation/kernel-parameters.txt
> +++ b/Documentation/kernel-parameters.txt
> @@ -4114,6 +4114,15 @@ bytes respectively. Such letter suffixes
>  			or other driver-specific files in the
>  			Documentation/watchdog/ directory.
>  
> +	workqueue.watchdog_thresh=
> +			If CONFIG_WQ_WATCHDOG is configured, workqueue can
> +			warn stall conditions and dump internal state to
> +			help debugging.  0 disables workqueue stall
> +			detection; otherwise, it's the stall threshold
> +			duration in seconds.  The default value is 30 and
> +			it can be updated at runtime by writing to the
> +			corresponding sysfs file.
> +
>  	workqueue.disable_numa
>  			By default, all work items queued to unbound
>  			workqueues are affine to the NUMA nodes they're
> --- a/include/linux/workqueue.h
> +++ b/include/linux/workqueue.h
> @@ -618,4 +618,10 @@ static inline int workqueue_sysfs_regist
>  { return 0; }
>  #endif	/* CONFIG_SYSFS */
>  
> +#ifdef CONFIG_WQ_WATCHDOG
> +void wq_watchdog_touch(int cpu);
> +#else	/* CONFIG_WQ_WATCHDOG */
> +static inline void wq_watchdog_touch(int cpu) { }
> +#endif	/* CONFIG_WQ_WATCHDOG */
> +
>  #endif
> --- a/kernel/watchdog.c
> +++ b/kernel/watchdog.c
> @@ -20,6 +20,7 @@
>  #include <linux/smpboot.h>
>  #include <linux/sched/rt.h>
>  #include <linux/tick.h>
> +#include <linux/workqueue.h>
>  
>  #include <asm/irq_regs.h>
>  #include <linux/kvm_para.h>
> @@ -245,6 +246,7 @@ void touch_softlockup_watchdog_sched(voi
>  void touch_softlockup_watchdog(void)
>  {
>  	touch_softlockup_watchdog_sched();
> +	wq_watchdog_touch(raw_smp_processor_id());
>  }
>  EXPORT_SYMBOL(touch_softlockup_watchdog);
>  
> @@ -259,6 +261,7 @@ void touch_all_softlockup_watchdogs(void
>  	 */
>  	for_each_watchdog_cpu(cpu)
>  		per_cpu(watchdog_touch_ts, cpu) = 0;
> +	wq_watchdog_touch(-1);
>  }
>  
>  #ifdef CONFIG_HARDLOCKUP_DETECTOR
> --- a/kernel/workqueue.c
> +++ b/kernel/workqueue.c
> @@ -148,6 +148,8 @@ struct worker_pool {
>  	int			id;		/* I: pool ID */
>  	unsigned int		flags;		/* X: flags */
>  
> +	unsigned long		watchdog_ts;	/* L: watchdog timestamp */
> +
>  	struct list_head	worklist;	/* L: list of pending works */
>  	int			nr_workers;	/* L: total number of workers */
>  
> @@ -1083,6 +1085,8 @@ static void pwq_activate_delayed_work(st
>  	struct pool_workqueue *pwq = get_work_pwq(work);
>  
>  	trace_workqueue_activate_work(work);
> +	if (list_empty(&pwq->pool->worklist))
> +		pwq->pool->watchdog_ts = jiffies;
>  	move_linked_works(work, &pwq->pool->worklist, NULL);
>  	__clear_bit(WORK_STRUCT_DELAYED_BIT, work_data_bits(work));
>  	pwq->nr_active++;
> @@ -1385,6 +1389,8 @@ retry:
>  		trace_workqueue_activate_work(work);
>  		pwq->nr_active++;
>  		worklist = &pwq->pool->worklist;
> +		if (list_empty(worklist))
> +			pwq->pool->watchdog_ts = jiffies;
>  	} else {
>  		work_flags |= WORK_STRUCT_DELAYED;
>  		worklist = &pwq->delayed_works;
> @@ -2157,6 +2163,8 @@ recheck:
>  			list_first_entry(&pool->worklist,
>  					 struct work_struct, entry);
>  
> +		pool->watchdog_ts = jiffies;
> +
>  		if (likely(!(*work_data_bits(work) & WORK_STRUCT_LINKED))) {
>  			/* optimization path, not strictly necessary */
>  			process_one_work(worker, work);
> @@ -2240,6 +2248,7 @@ repeat:
>  					struct pool_workqueue, mayday_node);
>  		struct worker_pool *pool = pwq->pool;
>  		struct work_struct *work, *n;
> +		bool first = true;
>  
>  		__set_current_state(TASK_RUNNING);
>  		list_del_init(&pwq->mayday_node);
> @@ -2256,9 +2265,14 @@ repeat:
>  		 * process'em.
>  		 */
>  		WARN_ON_ONCE(!list_empty(scheduled));
> -		list_for_each_entry_safe(work, n, &pool->worklist, entry)
> -			if (get_work_pwq(work) == pwq)
> +		list_for_each_entry_safe(work, n, &pool->worklist, entry) {
> +			if (get_work_pwq(work) == pwq) {
> +				if (first)
> +					pool->watchdog_ts = jiffies;
>  				move_linked_works(work, scheduled, &n);
> +			}
> +			first = false;
> +		}
>  
>  		if (!list_empty(scheduled)) {
>  			process_scheduled_works(rescuer);
> @@ -3104,6 +3118,7 @@ static int init_worker_pool(struct worke
>  	pool->cpu = -1;
>  	pool->node = NUMA_NO_NODE;
>  	pool->flags |= POOL_DISASSOCIATED;
> +	pool->watchdog_ts = jiffies;
>  	INIT_LIST_HEAD(&pool->worklist);
>  	INIT_LIST_HEAD(&pool->idle_list);
>  	hash_init(pool->busy_hash);
> @@ -4343,7 +4358,9 @@ void show_workqueue_state(void)
>  
>  		pr_info("pool %d:", pool->id);
>  		pr_cont_pool_info(pool);
> -		pr_cont(" workers=%d", pool->nr_workers);
> +		pr_cont(" hung=%us workers=%d",
> +			jiffies_to_msecs(jiffies - pool->watchdog_ts) / 1000,
> +			pool->nr_workers);
>  		if (pool->manager)
>  			pr_cont(" manager: %d",
>  				task_pid_nr(pool->manager->task));
> @@ -5202,6 +5219,154 @@ static void workqueue_sysfs_unregister(s
>  static void workqueue_sysfs_unregister(struct workqueue_struct *wq)	{ }
>  #endif	/* CONFIG_SYSFS */
>  
> +/*
> + * Workqueue watchdog.
> + *
> + * Stall may be caused by various bugs - missing WQ_MEM_RECLAIM, illegal
> + * flush dependency, a concurrency managed work item which stays RUNNING
> + * indefinitely.  Workqueue stalls can be very difficult to debug as the
> + * usual warning mechanisms don't trigger and internal workqueue state is
> + * largely opaque.
> + *
> + * Workqueue watchdog monitors all worker pools periodically and dumps
> + * state if some pools failed to make forward progress for a while where
> + * forward progress is defined as the first item on ->worklist changing.
> + *
> + * This mechanism is controlled through the kernel parameter
> + * "workqueue.watchdog_thresh" which can be updated at runtime through the
> + * corresponding sysfs parameter file.
> + */
> +#ifdef CONFIG_WQ_WATCHDOG
> +
> +static void wq_watchdog_timer_fn(unsigned long data);
> +
> +static unsigned long wq_watchdog_thresh = 30;
> +static struct timer_list wq_watchdog_timer =
> +	TIMER_DEFERRED_INITIALIZER(wq_watchdog_timer_fn, 0, 0);
> +
> +static unsigned long wq_watchdog_touched = INITIAL_JIFFIES;
> +static DEFINE_PER_CPU(unsigned long, wq_watchdog_touched_cpu) = INITIAL_JIFFIES;
> +
> +static void wq_watchdog_reset_touched(void)
> +{
> +	int cpu;
> +
> +	wq_watchdog_touched = jiffies;
> +	for_each_possible_cpu(cpu)
> +		per_cpu(wq_watchdog_touched_cpu, cpu) = jiffies;
> +}
> +
> +static void wq_watchdog_timer_fn(unsigned long data)
> +{
> +	unsigned long thresh = READ_ONCE(wq_watchdog_thresh) * HZ;
> +	bool lockup_detected = false;
> +	struct worker_pool *pool;
> +	int pi;
> +
> +	if (!thresh)
> +		return;
> +
> +	rcu_read_lock();
> +
> +	for_each_pool(pool, pi) {
> +		unsigned long pool_ts, touched, ts;
> +
> +		if (list_empty(&pool->worklist))
> +			continue;
> +
> +		/* get the latest of pool and touched timestamps */
> +		pool_ts = READ_ONCE(pool->watchdog_ts);
> +		touched = READ_ONCE(wq_watchdog_touched);
> +
> +		if (time_after(pool_ts, touched))
> +			ts = pool_ts;
> +		else
> +			ts = touched;
> +
> +		if (pool->cpu >= 0) {
> +			unsigned long cpu_touched =
> +				READ_ONCE(per_cpu(wq_watchdog_touched_cpu,
> +						  pool->cpu));
> +			if (time_after(cpu_touched, ts))
> +				ts = cpu_touched;
> +		}
> +
> +		/* did we stall? */
> +		if (time_after(jiffies, ts + thresh)) {
> +			lockup_detected = true;
> +			pr_emerg("BUG: workqueue lockup - pool");
> +			pr_cont_pool_info(pool);
> +			pr_cont(" stuck for %us!\n",
> +				jiffies_to_msecs(jiffies - pool_ts) / 1000);
> +		}
> +	}
> +
> +	rcu_read_unlock();
> +
> +	if (lockup_detected)
> +		show_workqueue_state();
> +
> +	wq_watchdog_reset_touched();
> +	mod_timer(&wq_watchdog_timer, jiffies + thresh);
> +}
> +
> +void wq_watchdog_touch(int cpu)
> +{
> +	if (cpu >= 0)
> +		per_cpu(wq_watchdog_touched_cpu, cpu) = jiffies;
> +	else
> +		wq_watchdog_touched = jiffies;
> +}
> +
> +static void wq_watchdog_set_thresh(unsigned long thresh)
> +{
> +	wq_watchdog_thresh = 0;
> +	del_timer_sync(&wq_watchdog_timer);
> +
> +	if (thresh) {
> +		wq_watchdog_thresh = thresh;
> +		wq_watchdog_reset_touched();
> +		mod_timer(&wq_watchdog_timer, jiffies + thresh * HZ);
> +	}
> +}
> +
> +static int wq_watchdog_param_set_thresh(const char *val,
> +					const struct kernel_param *kp)
> +{
> +	unsigned long thresh;
> +	int ret;
> +
> +	ret = kstrtoul(val, 0, &thresh);
> +	if (ret)
> +		return ret;
> +
> +	if (system_wq)
> +		wq_watchdog_set_thresh(thresh);
> +	else
> +		wq_watchdog_thresh = thresh;
> +
> +	return 0;
> +}
> +
> +static const struct kernel_param_ops wq_watchdog_thresh_ops = {
> +	.set	= wq_watchdog_param_set_thresh,
> +	.get	= param_get_ulong,
> +};
> +
> +module_param_cb(watchdog_thresh, &wq_watchdog_thresh_ops, &wq_watchdog_thresh,
> +		0644);
> +
> +static void wq_watchdog_init(void)
> +{
> +	wq_watchdog_set_thresh(wq_watchdog_thresh);
> +}
> +
> +#else	/* CONFIG_WQ_WATCHDOG */
> +
> +static inline void wq_watchdog_init(void) { }
> +
> +#endif	/* CONFIG_WQ_WATCHDOG */
> +
>  static void __init wq_numa_init(void)
>  {
>  	cpumask_var_t *tbl;
> @@ -5325,6 +5490,9 @@ static int __init init_workqueues(void)
>  	       !system_unbound_wq || !system_freezable_wq ||
>  	       !system_power_efficient_wq ||
>  	       !system_freezable_power_efficient_wq);
> +
> +	wq_watchdog_init();
> +
>  	return 0;
>  }
>  early_initcall(init_workqueues);
> --- a/lib/Kconfig.debug
> +++ b/lib/Kconfig.debug
> @@ -812,6 +812,17 @@ config BOOTPARAM_HUNG_TASK_PANIC_VALUE
>  	default 0 if !BOOTPARAM_HUNG_TASK_PANIC
>  	default 1 if BOOTPARAM_HUNG_TASK_PANIC
>  
> +config WQ_WATCHDOG
> +	bool "Detect Workqueue Stalls"
> +	depends on DEBUG_KERNEL
> +	help
> +	  Say Y here to enable stall detection on workqueues.  If a
> +	  worker pool doesn't make forward progress on a pending work
> +	  item for over a given amount of time, 30s by default, a
> +	  warning message is printed along with dump of workqueue
> +	  state.  This can be configured through kernel parameter
> +	  "workqueue.watchdog_thresh" and its sysfs counterpart.
> +
>  endmenu # "Debug lockups and hangs"
>  
>  config PANIC_ON_OOPS
> --
> To unsubscribe from this list: send the line "unsubscribe linux-kernel" in
> the body of a message to majordomo@vger.kernel.org
> More majordomo info at  http://vger.kernel.org/majordomo-info.html
> Please read the FAQ at  http://www.tux.org/lkml/

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

* Re: [PATCH v2 2/2] workqueue: implement lockup detector
  2015-12-07 21:38     ` Don Zickus
@ 2015-12-07 21:39       ` Tejun Heo
  2015-12-08 16:00         ` Don Zickus
  0 siblings, 1 reply; 51+ messages in thread
From: Tejun Heo @ 2015-12-07 21:39 UTC (permalink / raw)
  To: Don Zickus
  Cc: Ulrich Obergfell, Ingo Molnar, Peter Zijlstra, Andrew Morton,
	linux-kernel, kernel-team

On Mon, Dec 07, 2015 at 04:38:16PM -0500, Don Zickus wrote:
> On Mon, Dec 07, 2015 at 02:06:17PM -0500, Tejun Heo wrote:
> > Hello,
> > 
> > Decoupled the control knobs from softlockup.  It's now workqueue
> > module param which can be updated at runtime.  If there's no
> > objection, I'll push the two patches through wq/for-4.5.
> 
> Does this still compile correctly with CONFIG_WQ_WATCHDOG disabled?

Yeah and if some config combo breaks I'm gonna catch it while it goes
through wq tree.

Thanks.

-- 
tejun

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

* Re: [PATCH v2 2/2] workqueue: implement lockup detector
  2015-12-07 21:39       ` Tejun Heo
@ 2015-12-08 16:00         ` Don Zickus
  2015-12-08 16:31           ` Tejun Heo
  0 siblings, 1 reply; 51+ messages in thread
From: Don Zickus @ 2015-12-08 16:00 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Ulrich Obergfell, Ingo Molnar, Peter Zijlstra, Andrew Morton,
	linux-kernel, kernel-team

On Mon, Dec 07, 2015 at 04:39:52PM -0500, Tejun Heo wrote:
> On Mon, Dec 07, 2015 at 04:38:16PM -0500, Don Zickus wrote:
> > On Mon, Dec 07, 2015 at 02:06:17PM -0500, Tejun Heo wrote:
> > > Hello,
> > > 
> > > Decoupled the control knobs from softlockup.  It's now workqueue
> > > module param which can be updated at runtime.  If there's no
> > > objection, I'll push the two patches through wq/for-4.5.
> > 
> > Does this still compile correctly with CONFIG_WQ_WATCHDOG disabled?
> 
> Yeah and if some config combo breaks I'm gonna catch it while it goes
> through wq tree.

I think I am fine with this approach.

Acked-by: Don Zickus <dzickus@redhat.com>

Cheers,
Don

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

* Re: [PATCH v2 2/2] workqueue: implement lockup detector
  2015-12-08 16:00         ` Don Zickus
@ 2015-12-08 16:31           ` Tejun Heo
  0 siblings, 0 replies; 51+ messages in thread
From: Tejun Heo @ 2015-12-08 16:31 UTC (permalink / raw)
  To: Don Zickus
  Cc: Ulrich Obergfell, Ingo Molnar, Peter Zijlstra, Andrew Morton,
	linux-kernel, kernel-team

On Tue, Dec 08, 2015 at 11:00:54AM -0500, Don Zickus wrote:
> On Mon, Dec 07, 2015 at 04:39:52PM -0500, Tejun Heo wrote:
> > On Mon, Dec 07, 2015 at 04:38:16PM -0500, Don Zickus wrote:
> > > On Mon, Dec 07, 2015 at 02:06:17PM -0500, Tejun Heo wrote:
> > > > Hello,
> > > > 
> > > > Decoupled the control knobs from softlockup.  It's now workqueue
> > > > module param which can be updated at runtime.  If there's no
> > > > objection, I'll push the two patches through wq/for-4.5.
> > > 
> > > Does this still compile correctly with CONFIG_WQ_WATCHDOG disabled?
> > 
> > Yeah and if some config combo breaks I'm gonna catch it while it goes
> > through wq tree.
> 
> I think I am fine with this approach.
> 
> Acked-by: Don Zickus <dzickus@redhat.com>

Applied the two patches to wq/for-4.5.

Thanks.

-- 
tejun

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

* Re: [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue
  2015-12-03 19:26           ` [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue Tejun Heo
@ 2016-01-26 17:38                 ` Thierry Reding
  2015-12-07 15:58             ` Tejun Heo
                                   ` (2 subsequent siblings)
  3 siblings, 0 replies; 51+ messages in thread
From: Thierry Reding @ 2016-01-26 17:38 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Peter Zijlstra, Ulrich Obergfell, Ingo Molnar, Andrew Morton,
	linux-kernel-u79uwXL29TY76Z2rM5mHXA, kernel-team-b10kYP2dOMg,
	Jon Hunter, linux-tegra-u79uwXL29TY76Z2rM5mHXA

[-- Attachment #1: Type: text/plain, Size: 6332 bytes --]

On Thu, Dec 03, 2015 at 02:26:16PM -0500, Tejun Heo wrote:
> Task or work item involved in memory reclaim trying to flush a
> non-WQ_MEM_RECLAIM workqueue or one of its work items can lead to
> deadlock.  Trigger WARN_ONCE() if such conditions are detected.
> 
> Signed-off-by: Tejun Heo <tj-DgEjT+Ai2ygdnm+yROfE0A@public.gmane.org>
> Cc: Peter Zijlstra <peterz-wEGCiKHe2LqWVfeAwA7xHQ@public.gmane.org>
> ---
> Hello,
> 
> So, something like this.  Seems to work fine here.  If there's no
> objection, I'm gonna push it through wq/for-4.5.
> 
> Thanks.
> 
>  kernel/workqueue.c |   35 +++++++++++++++++++++++++++++++++++
>  1 file changed, 35 insertions(+)
> 
> --- a/kernel/workqueue.c
> +++ b/kernel/workqueue.c
> @@ -2330,6 +2330,37 @@ repeat:
>  	goto repeat;
>  }
>  
> +/**
> + * check_flush_dependency - check for flush dependency sanity
> + * @target_wq: workqueue being flushed
> + * @target_work: work item being flushed (NULL for workqueue flushes)
> + *
> + * %current is trying to flush the whole @target_wq or @target_work on it.
> + * If @target_wq doesn't have %WQ_MEM_RECLAIM, verify that %current is not
> + * reclaiming memory or running on a workqueue which doesn't have
> + * %WQ_MEM_RECLAIM as that can break forward-progress guarantee leading to
> + * a deadlock.
> + */
> +static void check_flush_dependency(struct workqueue_struct *target_wq,
> +				   struct work_struct *target_work)
> +{
> +	work_func_t target_func = target_work ? target_work->func : NULL;
> +	struct worker *worker;
> +
> +	if (target_wq->flags & WQ_MEM_RECLAIM)
> +		return;
> +
> +	worker = current_wq_worker();
> +
> +	WARN_ONCE(current->flags & PF_MEMALLOC,
> +		  "workqueue: PF_MEMALLOC task %d(%s) is flushing !WQ_MEM_RECLAIM %s:%pf",
> +		  current->pid, current->comm, target_wq->name, target_func);
> +	WARN_ONCE(worker && (worker->current_pwq->wq->flags & WQ_MEM_RECLAIM),
> +		  "workqueue: WQ_MEM_RECLAIM %s:%pf is flushing !WQ_MEM_RECLAIM %s:%pf",
> +		  worker->current_pwq->wq->name, worker->current_func,
> +		  target_wq->name, target_func);
> +}
> +
>  struct wq_barrier {
>  	struct work_struct	work;
>  	struct completion	done;
> @@ -2539,6 +2570,8 @@ void flush_workqueue(struct workqueue_st
>  		list_add_tail(&this_flusher.list, &wq->flusher_overflow);
>  	}
>  
> +	check_flush_dependency(wq, NULL);
> +
>  	mutex_unlock(&wq->mutex);
>  
>  	wait_for_completion(&this_flusher.done);
> @@ -2711,6 +2744,8 @@ static bool start_flush_work(struct work
>  		pwq = worker->current_pwq;
>  	}
>  
> +	check_flush_dependency(pwq->wq, work);
> +
>  	insert_wq_barrier(pwq, barr, work, worker);
>  	spin_unlock_irq(&pool->lock);
>  

I've started noticing the following during boot on some of the devices I
work with:

[    4.723705] WARNING: CPU: 0 PID: 6 at kernel/workqueue.c:2361 check_flush_dependency+0x138/0x144()
[    4.736818] workqueue: WQ_MEM_RECLAIM deferwq:deferred_probe_work_func is flushing !WQ_MEM_RECLAIM events:lru_add_drain_per_cpu
[    4.748099] Modules linked in:
[    4.751342] CPU: 0 PID: 6 Comm: kworker/u8:0 Not tainted 4.5.0-rc1-00018-g420fc292d9c7 #1
[    4.759504] Hardware name: NVIDIA Tegra SoC (Flattened Device Tree)
[    4.765762] Workqueue: deferwq deferred_probe_work_func
[    4.771004] [<c0017acc>] (unwind_backtrace) from [<c0013134>] (show_stack+0x10/0x14)
[    4.778746] [<c0013134>] (show_stack) from [<c0245f18>] (dump_stack+0x94/0xd4)
[    4.785966] [<c0245f18>] (dump_stack) from [<c0026f9c>] (warn_slowpath_common+0x80/0xb0)
[    4.794048] [<c0026f9c>] (warn_slowpath_common) from [<c0026ffc>] (warn_slowpath_fmt+0x30/0x40)
[    4.802736] [<c0026ffc>] (warn_slowpath_fmt) from [<c00390b8>] (check_flush_dependency+0x138/0x144)
[    4.811769] [<c00390b8>] (check_flush_dependency) from [<c0039ca0>] (flush_work+0x50/0x15c)
[    4.820112] [<c0039ca0>] (flush_work) from [<c00c51b0>] (lru_add_drain_all+0x130/0x180)
[    4.828110] [<c00c51b0>] (lru_add_drain_all) from [<c00f728c>] (migrate_prep+0x8/0x10)
[    4.836018] [<c00f728c>] (migrate_prep) from [<c00bfbc4>] (alloc_contig_range+0xd8/0x338)
[    4.844186] [<c00bfbc4>] (alloc_contig_range) from [<c00f8f18>] (cma_alloc+0xe0/0x1ac)
[    4.852093] [<c00f8f18>] (cma_alloc) from [<c001cac4>] (__alloc_from_contiguous+0x38/0xd8)
[    4.860346] [<c001cac4>] (__alloc_from_contiguous) from [<c001ceb4>] (__dma_alloc+0x240/0x278)
[    4.868944] [<c001ceb4>] (__dma_alloc) from [<c001cf78>] (arm_dma_alloc+0x54/0x5c)
[    4.876506] [<c001cf78>] (arm_dma_alloc) from [<c0355ea4>] (dmam_alloc_coherent+0xc0/0xec)
[    4.884764] [<c0355ea4>] (dmam_alloc_coherent) from [<c039cc4c>] (ahci_port_start+0x150/0x1dc)
[    4.893367] [<c039cc4c>] (ahci_port_start) from [<c0384734>] (ata_host_start.part.3+0xc8/0x1c8)
[    4.902055] [<c0384734>] (ata_host_start.part.3) from [<c03898dc>] (ata_host_activate+0x50/0x148)
[    4.910919] [<c03898dc>] (ata_host_activate) from [<c039d558>] (ahci_host_activate+0x44/0x114)
[    4.919523] [<c039d558>] (ahci_host_activate) from [<c039f05c>] (ahci_platform_init_host+0x1d8/0x3c8)
[    4.928733] [<c039f05c>] (ahci_platform_init_host) from [<c039e6bc>] (tegra_ahci_probe+0x448/0x4e8)
[    4.937770] [<c039e6bc>] (tegra_ahci_probe) from [<c0347058>] (platform_drv_probe+0x50/0xac)
[    4.946197] [<c0347058>] (platform_drv_probe) from [<c03458cc>] (driver_probe_device+0x214/0x2c0)
[    4.955061] [<c03458cc>] (driver_probe_device) from [<c0343cc0>] (bus_for_each_drv+0x60/0x94)
[    4.963575] [<c0343cc0>] (bus_for_each_drv) from [<c03455d8>] (__device_attach+0xb0/0x114)
[    4.971828] [<c03455d8>] (__device_attach) from [<c0344ab8>] (bus_probe_device+0x84/0x8c)
[    4.979994] [<c0344ab8>] (bus_probe_device) from [<c0344f48>] (deferred_probe_work_func+0x68/0x98)
[    4.988941] [<c0344f48>] (deferred_probe_work_func) from [<c003b738>] (process_one_work+0x120/0x3f8)
[    4.998062] [<c003b738>] (process_one_work) from [<c003ba48>] (worker_thread+0x38/0x55c)
[    5.006144] [<c003ba48>] (worker_thread) from [<c0040f14>] (kthread+0xdc/0xf4)
[    5.013362] [<c0040f14>] (kthread) from [<c000f778>] (ret_from_fork+0x14/0x3c)

This seems to be caused by the interaction of the probe deferral
workqueue with WQ_MEM_RECLAIM workqueue. Any ideas on how to solve this?

Thierry

[-- Attachment #2: signature.asc --]
[-- Type: application/pgp-signature, Size: 819 bytes --]

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

* Re: [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue
@ 2016-01-26 17:38                 ` Thierry Reding
  0 siblings, 0 replies; 51+ messages in thread
From: Thierry Reding @ 2016-01-26 17:38 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Peter Zijlstra, Ulrich Obergfell, Ingo Molnar, Andrew Morton,
	linux-kernel, kernel-team, Jon Hunter, linux-tegra

[-- Attachment #1: Type: text/plain, Size: 6277 bytes --]

On Thu, Dec 03, 2015 at 02:26:16PM -0500, Tejun Heo wrote:
> Task or work item involved in memory reclaim trying to flush a
> non-WQ_MEM_RECLAIM workqueue or one of its work items can lead to
> deadlock.  Trigger WARN_ONCE() if such conditions are detected.
> 
> Signed-off-by: Tejun Heo <tj@kernel.org>
> Cc: Peter Zijlstra <peterz@infradead.org>
> ---
> Hello,
> 
> So, something like this.  Seems to work fine here.  If there's no
> objection, I'm gonna push it through wq/for-4.5.
> 
> Thanks.
> 
>  kernel/workqueue.c |   35 +++++++++++++++++++++++++++++++++++
>  1 file changed, 35 insertions(+)
> 
> --- a/kernel/workqueue.c
> +++ b/kernel/workqueue.c
> @@ -2330,6 +2330,37 @@ repeat:
>  	goto repeat;
>  }
>  
> +/**
> + * check_flush_dependency - check for flush dependency sanity
> + * @target_wq: workqueue being flushed
> + * @target_work: work item being flushed (NULL for workqueue flushes)
> + *
> + * %current is trying to flush the whole @target_wq or @target_work on it.
> + * If @target_wq doesn't have %WQ_MEM_RECLAIM, verify that %current is not
> + * reclaiming memory or running on a workqueue which doesn't have
> + * %WQ_MEM_RECLAIM as that can break forward-progress guarantee leading to
> + * a deadlock.
> + */
> +static void check_flush_dependency(struct workqueue_struct *target_wq,
> +				   struct work_struct *target_work)
> +{
> +	work_func_t target_func = target_work ? target_work->func : NULL;
> +	struct worker *worker;
> +
> +	if (target_wq->flags & WQ_MEM_RECLAIM)
> +		return;
> +
> +	worker = current_wq_worker();
> +
> +	WARN_ONCE(current->flags & PF_MEMALLOC,
> +		  "workqueue: PF_MEMALLOC task %d(%s) is flushing !WQ_MEM_RECLAIM %s:%pf",
> +		  current->pid, current->comm, target_wq->name, target_func);
> +	WARN_ONCE(worker && (worker->current_pwq->wq->flags & WQ_MEM_RECLAIM),
> +		  "workqueue: WQ_MEM_RECLAIM %s:%pf is flushing !WQ_MEM_RECLAIM %s:%pf",
> +		  worker->current_pwq->wq->name, worker->current_func,
> +		  target_wq->name, target_func);
> +}
> +
>  struct wq_barrier {
>  	struct work_struct	work;
>  	struct completion	done;
> @@ -2539,6 +2570,8 @@ void flush_workqueue(struct workqueue_st
>  		list_add_tail(&this_flusher.list, &wq->flusher_overflow);
>  	}
>  
> +	check_flush_dependency(wq, NULL);
> +
>  	mutex_unlock(&wq->mutex);
>  
>  	wait_for_completion(&this_flusher.done);
> @@ -2711,6 +2744,8 @@ static bool start_flush_work(struct work
>  		pwq = worker->current_pwq;
>  	}
>  
> +	check_flush_dependency(pwq->wq, work);
> +
>  	insert_wq_barrier(pwq, barr, work, worker);
>  	spin_unlock_irq(&pool->lock);
>  

I've started noticing the following during boot on some of the devices I
work with:

[    4.723705] WARNING: CPU: 0 PID: 6 at kernel/workqueue.c:2361 check_flush_dependency+0x138/0x144()
[    4.736818] workqueue: WQ_MEM_RECLAIM deferwq:deferred_probe_work_func is flushing !WQ_MEM_RECLAIM events:lru_add_drain_per_cpu
[    4.748099] Modules linked in:
[    4.751342] CPU: 0 PID: 6 Comm: kworker/u8:0 Not tainted 4.5.0-rc1-00018-g420fc292d9c7 #1
[    4.759504] Hardware name: NVIDIA Tegra SoC (Flattened Device Tree)
[    4.765762] Workqueue: deferwq deferred_probe_work_func
[    4.771004] [<c0017acc>] (unwind_backtrace) from [<c0013134>] (show_stack+0x10/0x14)
[    4.778746] [<c0013134>] (show_stack) from [<c0245f18>] (dump_stack+0x94/0xd4)
[    4.785966] [<c0245f18>] (dump_stack) from [<c0026f9c>] (warn_slowpath_common+0x80/0xb0)
[    4.794048] [<c0026f9c>] (warn_slowpath_common) from [<c0026ffc>] (warn_slowpath_fmt+0x30/0x40)
[    4.802736] [<c0026ffc>] (warn_slowpath_fmt) from [<c00390b8>] (check_flush_dependency+0x138/0x144)
[    4.811769] [<c00390b8>] (check_flush_dependency) from [<c0039ca0>] (flush_work+0x50/0x15c)
[    4.820112] [<c0039ca0>] (flush_work) from [<c00c51b0>] (lru_add_drain_all+0x130/0x180)
[    4.828110] [<c00c51b0>] (lru_add_drain_all) from [<c00f728c>] (migrate_prep+0x8/0x10)
[    4.836018] [<c00f728c>] (migrate_prep) from [<c00bfbc4>] (alloc_contig_range+0xd8/0x338)
[    4.844186] [<c00bfbc4>] (alloc_contig_range) from [<c00f8f18>] (cma_alloc+0xe0/0x1ac)
[    4.852093] [<c00f8f18>] (cma_alloc) from [<c001cac4>] (__alloc_from_contiguous+0x38/0xd8)
[    4.860346] [<c001cac4>] (__alloc_from_contiguous) from [<c001ceb4>] (__dma_alloc+0x240/0x278)
[    4.868944] [<c001ceb4>] (__dma_alloc) from [<c001cf78>] (arm_dma_alloc+0x54/0x5c)
[    4.876506] [<c001cf78>] (arm_dma_alloc) from [<c0355ea4>] (dmam_alloc_coherent+0xc0/0xec)
[    4.884764] [<c0355ea4>] (dmam_alloc_coherent) from [<c039cc4c>] (ahci_port_start+0x150/0x1dc)
[    4.893367] [<c039cc4c>] (ahci_port_start) from [<c0384734>] (ata_host_start.part.3+0xc8/0x1c8)
[    4.902055] [<c0384734>] (ata_host_start.part.3) from [<c03898dc>] (ata_host_activate+0x50/0x148)
[    4.910919] [<c03898dc>] (ata_host_activate) from [<c039d558>] (ahci_host_activate+0x44/0x114)
[    4.919523] [<c039d558>] (ahci_host_activate) from [<c039f05c>] (ahci_platform_init_host+0x1d8/0x3c8)
[    4.928733] [<c039f05c>] (ahci_platform_init_host) from [<c039e6bc>] (tegra_ahci_probe+0x448/0x4e8)
[    4.937770] [<c039e6bc>] (tegra_ahci_probe) from [<c0347058>] (platform_drv_probe+0x50/0xac)
[    4.946197] [<c0347058>] (platform_drv_probe) from [<c03458cc>] (driver_probe_device+0x214/0x2c0)
[    4.955061] [<c03458cc>] (driver_probe_device) from [<c0343cc0>] (bus_for_each_drv+0x60/0x94)
[    4.963575] [<c0343cc0>] (bus_for_each_drv) from [<c03455d8>] (__device_attach+0xb0/0x114)
[    4.971828] [<c03455d8>] (__device_attach) from [<c0344ab8>] (bus_probe_device+0x84/0x8c)
[    4.979994] [<c0344ab8>] (bus_probe_device) from [<c0344f48>] (deferred_probe_work_func+0x68/0x98)
[    4.988941] [<c0344f48>] (deferred_probe_work_func) from [<c003b738>] (process_one_work+0x120/0x3f8)
[    4.998062] [<c003b738>] (process_one_work) from [<c003ba48>] (worker_thread+0x38/0x55c)
[    5.006144] [<c003ba48>] (worker_thread) from [<c0040f14>] (kthread+0xdc/0xf4)
[    5.013362] [<c0040f14>] (kthread) from [<c000f778>] (ret_from_fork+0x14/0x3c)

This seems to be caused by the interaction of the probe deferral
workqueue with WQ_MEM_RECLAIM workqueue. Any ideas on how to solve this?

Thierry

[-- Attachment #2: signature.asc --]
[-- Type: application/pgp-signature, Size: 819 bytes --]

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

* Re: [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue
  2016-01-26 17:38                 ` Thierry Reding
@ 2016-01-28 10:12                     ` Peter Zijlstra
  -1 siblings, 0 replies; 51+ messages in thread
From: Peter Zijlstra @ 2016-01-28 10:12 UTC (permalink / raw)
  To: Thierry Reding
  Cc: Tejun Heo, Ulrich Obergfell, Ingo Molnar, Andrew Morton,
	linux-kernel-u79uwXL29TY76Z2rM5mHXA, kernel-team-b10kYP2dOMg,
	Jon Hunter, linux-tegra-u79uwXL29TY76Z2rM5mHXA,
	rmk+kernel-lFZ/pmaqli7XmaaqVzeoHQ

On Tue, Jan 26, 2016 at 06:38:43PM +0100, Thierry Reding wrote:
> > Task or work item involved in memory reclaim trying to flush a
> > non-WQ_MEM_RECLAIM workqueue or one of its work items can lead to
> > deadlock.  Trigger WARN_ONCE() if such conditions are detected.
> I've started noticing the following during boot on some of the devices I
> work with:
> 
> [    4.723705] WARNING: CPU: 0 PID: 6 at kernel/workqueue.c:2361 check_flush_dependency+0x138/0x144()
> [    4.736818] workqueue: WQ_MEM_RECLAIM deferwq:deferred_probe_work_func is flushing !WQ_MEM_RECLAIM events:lru_add_drain_per_cpu
> [    4.748099] Modules linked in:
> [    4.751342] CPU: 0 PID: 6 Comm: kworker/u8:0 Not tainted 4.5.0-rc1-00018-g420fc292d9c7 #1
> [    4.759504] Hardware name: NVIDIA Tegra SoC (Flattened Device Tree)
> [    4.765762] Workqueue: deferwq deferred_probe_work_func
> [    4.771004] [<c0017acc>] (unwind_backtrace) from [<c0013134>] (show_stack+0x10/0x14)
> [    4.778746] [<c0013134>] (show_stack) from [<c0245f18>] (dump_stack+0x94/0xd4)
> [    4.785966] [<c0245f18>] (dump_stack) from [<c0026f9c>] (warn_slowpath_common+0x80/0xb0)
> [    4.794048] [<c0026f9c>] (warn_slowpath_common) from [<c0026ffc>] (warn_slowpath_fmt+0x30/0x40)
> [    4.802736] [<c0026ffc>] (warn_slowpath_fmt) from [<c00390b8>] (check_flush_dependency+0x138/0x144)
> [    4.811769] [<c00390b8>] (check_flush_dependency) from [<c0039ca0>] (flush_work+0x50/0x15c)
> [    4.820112] [<c0039ca0>] (flush_work) from [<c00c51b0>] (lru_add_drain_all+0x130/0x180)
> [    4.828110] [<c00c51b0>] (lru_add_drain_all) from [<c00f728c>] (migrate_prep+0x8/0x10)

Right, also, I think it makes sense to do lru_add_drain_all() from a
WQ_MEM_RECLAIM workqueue, it is, after all, aiding in getting memory
freed.

Does something like the below cure things?

TJ does this make sense to you?

---
 mm/swap.c | 11 ++++++++++-
 1 file changed, 10 insertions(+), 1 deletion(-)

diff --git a/mm/swap.c b/mm/swap.c
index 09fe5e97714a..a3de016b2a9d 100644
--- a/mm/swap.c
+++ b/mm/swap.c
@@ -666,6 +666,15 @@ static void lru_add_drain_per_cpu(struct work_struct *dummy)
 
 static DEFINE_PER_CPU(struct work_struct, lru_add_drain_work);
 
+static struct workqueue_struct *lru_wq;
+
+static int __init lru_init(void)
+{
+	lru_wq = create_workqueue("lru");
+	return 0;
+}
+early_initcall(lru_init);
+
 void lru_add_drain_all(void)
 {
 	static DEFINE_MUTEX(lock);
@@ -685,7 +694,7 @@ void lru_add_drain_all(void)
 		    pagevec_count(&per_cpu(lru_deactivate_pvecs, cpu)) ||
 		    need_activate_page_drain(cpu)) {
 			INIT_WORK(work, lru_add_drain_per_cpu);
-			schedule_work_on(cpu, work);
+			queue_work_on(cpu, &lru_wq, work);
 			cpumask_set_cpu(cpu, &has_work);
 		}
 	}

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

* Re: [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue
@ 2016-01-28 10:12                     ` Peter Zijlstra
  0 siblings, 0 replies; 51+ messages in thread
From: Peter Zijlstra @ 2016-01-28 10:12 UTC (permalink / raw)
  To: Thierry Reding
  Cc: Tejun Heo, Ulrich Obergfell, Ingo Molnar, Andrew Morton,
	linux-kernel, kernel-team, Jon Hunter, linux-tegra, rmk+kernel

On Tue, Jan 26, 2016 at 06:38:43PM +0100, Thierry Reding wrote:
> > Task or work item involved in memory reclaim trying to flush a
> > non-WQ_MEM_RECLAIM workqueue or one of its work items can lead to
> > deadlock.  Trigger WARN_ONCE() if such conditions are detected.
> I've started noticing the following during boot on some of the devices I
> work with:
> 
> [    4.723705] WARNING: CPU: 0 PID: 6 at kernel/workqueue.c:2361 check_flush_dependency+0x138/0x144()
> [    4.736818] workqueue: WQ_MEM_RECLAIM deferwq:deferred_probe_work_func is flushing !WQ_MEM_RECLAIM events:lru_add_drain_per_cpu
> [    4.748099] Modules linked in:
> [    4.751342] CPU: 0 PID: 6 Comm: kworker/u8:0 Not tainted 4.5.0-rc1-00018-g420fc292d9c7 #1
> [    4.759504] Hardware name: NVIDIA Tegra SoC (Flattened Device Tree)
> [    4.765762] Workqueue: deferwq deferred_probe_work_func
> [    4.771004] [<c0017acc>] (unwind_backtrace) from [<c0013134>] (show_stack+0x10/0x14)
> [    4.778746] [<c0013134>] (show_stack) from [<c0245f18>] (dump_stack+0x94/0xd4)
> [    4.785966] [<c0245f18>] (dump_stack) from [<c0026f9c>] (warn_slowpath_common+0x80/0xb0)
> [    4.794048] [<c0026f9c>] (warn_slowpath_common) from [<c0026ffc>] (warn_slowpath_fmt+0x30/0x40)
> [    4.802736] [<c0026ffc>] (warn_slowpath_fmt) from [<c00390b8>] (check_flush_dependency+0x138/0x144)
> [    4.811769] [<c00390b8>] (check_flush_dependency) from [<c0039ca0>] (flush_work+0x50/0x15c)
> [    4.820112] [<c0039ca0>] (flush_work) from [<c00c51b0>] (lru_add_drain_all+0x130/0x180)
> [    4.828110] [<c00c51b0>] (lru_add_drain_all) from [<c00f728c>] (migrate_prep+0x8/0x10)

Right, also, I think it makes sense to do lru_add_drain_all() from a
WQ_MEM_RECLAIM workqueue, it is, after all, aiding in getting memory
freed.

Does something like the below cure things?

TJ does this make sense to you?

---
 mm/swap.c | 11 ++++++++++-
 1 file changed, 10 insertions(+), 1 deletion(-)

diff --git a/mm/swap.c b/mm/swap.c
index 09fe5e97714a..a3de016b2a9d 100644
--- a/mm/swap.c
+++ b/mm/swap.c
@@ -666,6 +666,15 @@ static void lru_add_drain_per_cpu(struct work_struct *dummy)
 
 static DEFINE_PER_CPU(struct work_struct, lru_add_drain_work);
 
+static struct workqueue_struct *lru_wq;
+
+static int __init lru_init(void)
+{
+	lru_wq = create_workqueue("lru");
+	return 0;
+}
+early_initcall(lru_init);
+
 void lru_add_drain_all(void)
 {
 	static DEFINE_MUTEX(lock);
@@ -685,7 +694,7 @@ void lru_add_drain_all(void)
 		    pagevec_count(&per_cpu(lru_deactivate_pvecs, cpu)) ||
 		    need_activate_page_drain(cpu)) {
 			INIT_WORK(work, lru_add_drain_per_cpu);
-			schedule_work_on(cpu, work);
+			queue_work_on(cpu, &lru_wq, work);
 			cpumask_set_cpu(cpu, &has_work);
 		}
 	}

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

* Re: [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue
  2016-01-28 10:12                     ` Peter Zijlstra
@ 2016-01-28 12:47                         ` Thierry Reding
  -1 siblings, 0 replies; 51+ messages in thread
From: Thierry Reding @ 2016-01-28 12:47 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Tejun Heo, Ulrich Obergfell, Ingo Molnar, Andrew Morton,
	linux-kernel-u79uwXL29TY76Z2rM5mHXA, kernel-team-b10kYP2dOMg,
	Jon Hunter, linux-tegra-u79uwXL29TY76Z2rM5mHXA,
	rmk+kernel-lFZ/pmaqli7XmaaqVzeoHQ

[-- Attachment #1: Type: text/plain, Size: 3192 bytes --]

On Thu, Jan 28, 2016 at 11:12:10AM +0100, Peter Zijlstra wrote:
> On Tue, Jan 26, 2016 at 06:38:43PM +0100, Thierry Reding wrote:
> > > Task or work item involved in memory reclaim trying to flush a
> > > non-WQ_MEM_RECLAIM workqueue or one of its work items can lead to
> > > deadlock.  Trigger WARN_ONCE() if such conditions are detected.
> > I've started noticing the following during boot on some of the devices I
> > work with:
> > 
> > [    4.723705] WARNING: CPU: 0 PID: 6 at kernel/workqueue.c:2361 check_flush_dependency+0x138/0x144()
> > [    4.736818] workqueue: WQ_MEM_RECLAIM deferwq:deferred_probe_work_func is flushing !WQ_MEM_RECLAIM events:lru_add_drain_per_cpu
> > [    4.748099] Modules linked in:
> > [    4.751342] CPU: 0 PID: 6 Comm: kworker/u8:0 Not tainted 4.5.0-rc1-00018-g420fc292d9c7 #1
> > [    4.759504] Hardware name: NVIDIA Tegra SoC (Flattened Device Tree)
> > [    4.765762] Workqueue: deferwq deferred_probe_work_func
> > [    4.771004] [<c0017acc>] (unwind_backtrace) from [<c0013134>] (show_stack+0x10/0x14)
> > [    4.778746] [<c0013134>] (show_stack) from [<c0245f18>] (dump_stack+0x94/0xd4)
> > [    4.785966] [<c0245f18>] (dump_stack) from [<c0026f9c>] (warn_slowpath_common+0x80/0xb0)
> > [    4.794048] [<c0026f9c>] (warn_slowpath_common) from [<c0026ffc>] (warn_slowpath_fmt+0x30/0x40)
> > [    4.802736] [<c0026ffc>] (warn_slowpath_fmt) from [<c00390b8>] (check_flush_dependency+0x138/0x144)
> > [    4.811769] [<c00390b8>] (check_flush_dependency) from [<c0039ca0>] (flush_work+0x50/0x15c)
> > [    4.820112] [<c0039ca0>] (flush_work) from [<c00c51b0>] (lru_add_drain_all+0x130/0x180)
> > [    4.828110] [<c00c51b0>] (lru_add_drain_all) from [<c00f728c>] (migrate_prep+0x8/0x10)
> 
> Right, also, I think it makes sense to do lru_add_drain_all() from a
> WQ_MEM_RECLAIM workqueue, it is, after all, aiding in getting memory
> freed.
> 
> Does something like the below cure things?
> 
> TJ does this make sense to you?
> 
> ---
>  mm/swap.c | 11 ++++++++++-
>  1 file changed, 10 insertions(+), 1 deletion(-)
> 
> diff --git a/mm/swap.c b/mm/swap.c
> index 09fe5e97714a..a3de016b2a9d 100644
> --- a/mm/swap.c
> +++ b/mm/swap.c
> @@ -666,6 +666,15 @@ static void lru_add_drain_per_cpu(struct work_struct *dummy)
>  
>  static DEFINE_PER_CPU(struct work_struct, lru_add_drain_work);
>  
> +static struct workqueue_struct *lru_wq;
> +
> +static int __init lru_init(void)
> +{
> +	lru_wq = create_workqueue("lru");
> +	return 0;
> +}
> +early_initcall(lru_init);
> +
>  void lru_add_drain_all(void)
>  {
>  	static DEFINE_MUTEX(lock);
> @@ -685,7 +694,7 @@ void lru_add_drain_all(void)
>  		    pagevec_count(&per_cpu(lru_deactivate_pvecs, cpu)) ||
>  		    need_activate_page_drain(cpu)) {
>  			INIT_WORK(work, lru_add_drain_per_cpu);
> -			schedule_work_on(cpu, work);
> +			queue_work_on(cpu, &lru_wq, work);
                                           ^

This ampersand is too much here and causes a compile-time warning.
Removing it and booting the resulting kernel doesn't trigger the
WQ_MEM_RECLAIM warning anymore, though.

Tested on top of next-20160128.

Thanks,
Thierry

[-- Attachment #2: signature.asc --]
[-- Type: application/pgp-signature, Size: 819 bytes --]

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

* Re: [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue
@ 2016-01-28 12:47                         ` Thierry Reding
  0 siblings, 0 replies; 51+ messages in thread
From: Thierry Reding @ 2016-01-28 12:47 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Tejun Heo, Ulrich Obergfell, Ingo Molnar, Andrew Morton,
	linux-kernel, kernel-team, Jon Hunter, linux-tegra, rmk+kernel

[-- Attachment #1: Type: text/plain, Size: 3192 bytes --]

On Thu, Jan 28, 2016 at 11:12:10AM +0100, Peter Zijlstra wrote:
> On Tue, Jan 26, 2016 at 06:38:43PM +0100, Thierry Reding wrote:
> > > Task or work item involved in memory reclaim trying to flush a
> > > non-WQ_MEM_RECLAIM workqueue or one of its work items can lead to
> > > deadlock.  Trigger WARN_ONCE() if such conditions are detected.
> > I've started noticing the following during boot on some of the devices I
> > work with:
> > 
> > [    4.723705] WARNING: CPU: 0 PID: 6 at kernel/workqueue.c:2361 check_flush_dependency+0x138/0x144()
> > [    4.736818] workqueue: WQ_MEM_RECLAIM deferwq:deferred_probe_work_func is flushing !WQ_MEM_RECLAIM events:lru_add_drain_per_cpu
> > [    4.748099] Modules linked in:
> > [    4.751342] CPU: 0 PID: 6 Comm: kworker/u8:0 Not tainted 4.5.0-rc1-00018-g420fc292d9c7 #1
> > [    4.759504] Hardware name: NVIDIA Tegra SoC (Flattened Device Tree)
> > [    4.765762] Workqueue: deferwq deferred_probe_work_func
> > [    4.771004] [<c0017acc>] (unwind_backtrace) from [<c0013134>] (show_stack+0x10/0x14)
> > [    4.778746] [<c0013134>] (show_stack) from [<c0245f18>] (dump_stack+0x94/0xd4)
> > [    4.785966] [<c0245f18>] (dump_stack) from [<c0026f9c>] (warn_slowpath_common+0x80/0xb0)
> > [    4.794048] [<c0026f9c>] (warn_slowpath_common) from [<c0026ffc>] (warn_slowpath_fmt+0x30/0x40)
> > [    4.802736] [<c0026ffc>] (warn_slowpath_fmt) from [<c00390b8>] (check_flush_dependency+0x138/0x144)
> > [    4.811769] [<c00390b8>] (check_flush_dependency) from [<c0039ca0>] (flush_work+0x50/0x15c)
> > [    4.820112] [<c0039ca0>] (flush_work) from [<c00c51b0>] (lru_add_drain_all+0x130/0x180)
> > [    4.828110] [<c00c51b0>] (lru_add_drain_all) from [<c00f728c>] (migrate_prep+0x8/0x10)
> 
> Right, also, I think it makes sense to do lru_add_drain_all() from a
> WQ_MEM_RECLAIM workqueue, it is, after all, aiding in getting memory
> freed.
> 
> Does something like the below cure things?
> 
> TJ does this make sense to you?
> 
> ---
>  mm/swap.c | 11 ++++++++++-
>  1 file changed, 10 insertions(+), 1 deletion(-)
> 
> diff --git a/mm/swap.c b/mm/swap.c
> index 09fe5e97714a..a3de016b2a9d 100644
> --- a/mm/swap.c
> +++ b/mm/swap.c
> @@ -666,6 +666,15 @@ static void lru_add_drain_per_cpu(struct work_struct *dummy)
>  
>  static DEFINE_PER_CPU(struct work_struct, lru_add_drain_work);
>  
> +static struct workqueue_struct *lru_wq;
> +
> +static int __init lru_init(void)
> +{
> +	lru_wq = create_workqueue("lru");
> +	return 0;
> +}
> +early_initcall(lru_init);
> +
>  void lru_add_drain_all(void)
>  {
>  	static DEFINE_MUTEX(lock);
> @@ -685,7 +694,7 @@ void lru_add_drain_all(void)
>  		    pagevec_count(&per_cpu(lru_deactivate_pvecs, cpu)) ||
>  		    need_activate_page_drain(cpu)) {
>  			INIT_WORK(work, lru_add_drain_per_cpu);
> -			schedule_work_on(cpu, work);
> +			queue_work_on(cpu, &lru_wq, work);
                                           ^

This ampersand is too much here and causes a compile-time warning.
Removing it and booting the resulting kernel doesn't trigger the
WQ_MEM_RECLAIM warning anymore, though.

Tested on top of next-20160128.

Thanks,
Thierry

[-- Attachment #2: signature.asc --]
[-- Type: application/pgp-signature, Size: 819 bytes --]

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

* Re: [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue
  2016-01-28 12:47                         ` Thierry Reding
@ 2016-01-28 12:48                           ` Thierry Reding
  -1 siblings, 0 replies; 51+ messages in thread
From: Thierry Reding @ 2016-01-28 12:48 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Tejun Heo, Ulrich Obergfell, Ingo Molnar, Andrew Morton,
	linux-kernel-u79uwXL29TY76Z2rM5mHXA, kernel-team-b10kYP2dOMg,
	Jon Hunter, linux-tegra-u79uwXL29TY76Z2rM5mHXA,
	rmk+kernel-lFZ/pmaqli7XmaaqVzeoHQ

[-- Attachment #1: Type: text/plain, Size: 3652 bytes --]

On Thu, Jan 28, 2016 at 01:47:00PM +0100, Thierry Reding wrote:
> On Thu, Jan 28, 2016 at 11:12:10AM +0100, Peter Zijlstra wrote:
> > On Tue, Jan 26, 2016 at 06:38:43PM +0100, Thierry Reding wrote:
> > > > Task or work item involved in memory reclaim trying to flush a
> > > > non-WQ_MEM_RECLAIM workqueue or one of its work items can lead to
> > > > deadlock.  Trigger WARN_ONCE() if such conditions are detected.
> > > I've started noticing the following during boot on some of the devices I
> > > work with:
> > > 
> > > [    4.723705] WARNING: CPU: 0 PID: 6 at kernel/workqueue.c:2361 check_flush_dependency+0x138/0x144()
> > > [    4.736818] workqueue: WQ_MEM_RECLAIM deferwq:deferred_probe_work_func is flushing !WQ_MEM_RECLAIM events:lru_add_drain_per_cpu
> > > [    4.748099] Modules linked in:
> > > [    4.751342] CPU: 0 PID: 6 Comm: kworker/u8:0 Not tainted 4.5.0-rc1-00018-g420fc292d9c7 #1
> > > [    4.759504] Hardware name: NVIDIA Tegra SoC (Flattened Device Tree)
> > > [    4.765762] Workqueue: deferwq deferred_probe_work_func
> > > [    4.771004] [<c0017acc>] (unwind_backtrace) from [<c0013134>] (show_stack+0x10/0x14)
> > > [    4.778746] [<c0013134>] (show_stack) from [<c0245f18>] (dump_stack+0x94/0xd4)
> > > [    4.785966] [<c0245f18>] (dump_stack) from [<c0026f9c>] (warn_slowpath_common+0x80/0xb0)
> > > [    4.794048] [<c0026f9c>] (warn_slowpath_common) from [<c0026ffc>] (warn_slowpath_fmt+0x30/0x40)
> > > [    4.802736] [<c0026ffc>] (warn_slowpath_fmt) from [<c00390b8>] (check_flush_dependency+0x138/0x144)
> > > [    4.811769] [<c00390b8>] (check_flush_dependency) from [<c0039ca0>] (flush_work+0x50/0x15c)
> > > [    4.820112] [<c0039ca0>] (flush_work) from [<c00c51b0>] (lru_add_drain_all+0x130/0x180)
> > > [    4.828110] [<c00c51b0>] (lru_add_drain_all) from [<c00f728c>] (migrate_prep+0x8/0x10)
> > 
> > Right, also, I think it makes sense to do lru_add_drain_all() from a
> > WQ_MEM_RECLAIM workqueue, it is, after all, aiding in getting memory
> > freed.
> > 
> > Does something like the below cure things?
> > 
> > TJ does this make sense to you?
> > 
> > ---
> >  mm/swap.c | 11 ++++++++++-
> >  1 file changed, 10 insertions(+), 1 deletion(-)
> > 
> > diff --git a/mm/swap.c b/mm/swap.c
> > index 09fe5e97714a..a3de016b2a9d 100644
> > --- a/mm/swap.c
> > +++ b/mm/swap.c
> > @@ -666,6 +666,15 @@ static void lru_add_drain_per_cpu(struct work_struct *dummy)
> >  
> >  static DEFINE_PER_CPU(struct work_struct, lru_add_drain_work);
> >  
> > +static struct workqueue_struct *lru_wq;
> > +
> > +static int __init lru_init(void)
> > +{
> > +	lru_wq = create_workqueue("lru");
> > +	return 0;
> > +}
> > +early_initcall(lru_init);
> > +
> >  void lru_add_drain_all(void)
> >  {
> >  	static DEFINE_MUTEX(lock);
> > @@ -685,7 +694,7 @@ void lru_add_drain_all(void)
> >  		    pagevec_count(&per_cpu(lru_deactivate_pvecs, cpu)) ||
> >  		    need_activate_page_drain(cpu)) {
> >  			INIT_WORK(work, lru_add_drain_per_cpu);
> > -			schedule_work_on(cpu, work);
> > +			queue_work_on(cpu, &lru_wq, work);
>                                            ^
> 
> This ampersand is too much here and causes a compile-time warning.
> Removing it and booting the resulting kernel doesn't trigger the
> WQ_MEM_RECLAIM warning anymore, though.
> 
> Tested on top of next-20160128.

This implies that if you want to turn this into a proper patch:

Tested-by: Thierry Reding <treding-DDmLM1+adcrQT0dZR+AlfA@public.gmane.org>

Alternatively, if you come up with a different way to fix things,
please let me know and I'll be happy to test again.

Thierry

[-- Attachment #2: signature.asc --]
[-- Type: application/pgp-signature, Size: 819 bytes --]

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

* Re: [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue
@ 2016-01-28 12:48                           ` Thierry Reding
  0 siblings, 0 replies; 51+ messages in thread
From: Thierry Reding @ 2016-01-28 12:48 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Tejun Heo, Ulrich Obergfell, Ingo Molnar, Andrew Morton,
	linux-kernel, kernel-team, Jon Hunter, linux-tegra, rmk+kernel

[-- Attachment #1: Type: text/plain, Size: 3623 bytes --]

On Thu, Jan 28, 2016 at 01:47:00PM +0100, Thierry Reding wrote:
> On Thu, Jan 28, 2016 at 11:12:10AM +0100, Peter Zijlstra wrote:
> > On Tue, Jan 26, 2016 at 06:38:43PM +0100, Thierry Reding wrote:
> > > > Task or work item involved in memory reclaim trying to flush a
> > > > non-WQ_MEM_RECLAIM workqueue or one of its work items can lead to
> > > > deadlock.  Trigger WARN_ONCE() if such conditions are detected.
> > > I've started noticing the following during boot on some of the devices I
> > > work with:
> > > 
> > > [    4.723705] WARNING: CPU: 0 PID: 6 at kernel/workqueue.c:2361 check_flush_dependency+0x138/0x144()
> > > [    4.736818] workqueue: WQ_MEM_RECLAIM deferwq:deferred_probe_work_func is flushing !WQ_MEM_RECLAIM events:lru_add_drain_per_cpu
> > > [    4.748099] Modules linked in:
> > > [    4.751342] CPU: 0 PID: 6 Comm: kworker/u8:0 Not tainted 4.5.0-rc1-00018-g420fc292d9c7 #1
> > > [    4.759504] Hardware name: NVIDIA Tegra SoC (Flattened Device Tree)
> > > [    4.765762] Workqueue: deferwq deferred_probe_work_func
> > > [    4.771004] [<c0017acc>] (unwind_backtrace) from [<c0013134>] (show_stack+0x10/0x14)
> > > [    4.778746] [<c0013134>] (show_stack) from [<c0245f18>] (dump_stack+0x94/0xd4)
> > > [    4.785966] [<c0245f18>] (dump_stack) from [<c0026f9c>] (warn_slowpath_common+0x80/0xb0)
> > > [    4.794048] [<c0026f9c>] (warn_slowpath_common) from [<c0026ffc>] (warn_slowpath_fmt+0x30/0x40)
> > > [    4.802736] [<c0026ffc>] (warn_slowpath_fmt) from [<c00390b8>] (check_flush_dependency+0x138/0x144)
> > > [    4.811769] [<c00390b8>] (check_flush_dependency) from [<c0039ca0>] (flush_work+0x50/0x15c)
> > > [    4.820112] [<c0039ca0>] (flush_work) from [<c00c51b0>] (lru_add_drain_all+0x130/0x180)
> > > [    4.828110] [<c00c51b0>] (lru_add_drain_all) from [<c00f728c>] (migrate_prep+0x8/0x10)
> > 
> > Right, also, I think it makes sense to do lru_add_drain_all() from a
> > WQ_MEM_RECLAIM workqueue, it is, after all, aiding in getting memory
> > freed.
> > 
> > Does something like the below cure things?
> > 
> > TJ does this make sense to you?
> > 
> > ---
> >  mm/swap.c | 11 ++++++++++-
> >  1 file changed, 10 insertions(+), 1 deletion(-)
> > 
> > diff --git a/mm/swap.c b/mm/swap.c
> > index 09fe5e97714a..a3de016b2a9d 100644
> > --- a/mm/swap.c
> > +++ b/mm/swap.c
> > @@ -666,6 +666,15 @@ static void lru_add_drain_per_cpu(struct work_struct *dummy)
> >  
> >  static DEFINE_PER_CPU(struct work_struct, lru_add_drain_work);
> >  
> > +static struct workqueue_struct *lru_wq;
> > +
> > +static int __init lru_init(void)
> > +{
> > +	lru_wq = create_workqueue("lru");
> > +	return 0;
> > +}
> > +early_initcall(lru_init);
> > +
> >  void lru_add_drain_all(void)
> >  {
> >  	static DEFINE_MUTEX(lock);
> > @@ -685,7 +694,7 @@ void lru_add_drain_all(void)
> >  		    pagevec_count(&per_cpu(lru_deactivate_pvecs, cpu)) ||
> >  		    need_activate_page_drain(cpu)) {
> >  			INIT_WORK(work, lru_add_drain_per_cpu);
> > -			schedule_work_on(cpu, work);
> > +			queue_work_on(cpu, &lru_wq, work);
>                                            ^
> 
> This ampersand is too much here and causes a compile-time warning.
> Removing it and booting the resulting kernel doesn't trigger the
> WQ_MEM_RECLAIM warning anymore, though.
> 
> Tested on top of next-20160128.

This implies that if you want to turn this into a proper patch:

Tested-by: Thierry Reding <treding@nvidia.com>

Alternatively, if you come up with a different way to fix things,
please let me know and I'll be happy to test again.

Thierry

[-- Attachment #2: signature.asc --]
[-- Type: application/pgp-signature, Size: 819 bytes --]

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

* [PATCH wq/for-4.5-fixes] workqueue: skip flush dependency checks for legacy workqueues
  2016-01-26 17:38                 ` Thierry Reding
@ 2016-01-29 10:59                     ` Tejun Heo
  -1 siblings, 0 replies; 51+ messages in thread
From: Tejun Heo @ 2016-01-29 10:59 UTC (permalink / raw)
  To: Thierry Reding
  Cc: Peter Zijlstra, Ulrich Obergfell, Ingo Molnar, Andrew Morton,
	linux-kernel-u79uwXL29TY76Z2rM5mHXA, kernel-team-b10kYP2dOMg,
	Jon Hunter, linux-tegra-u79uwXL29TY76Z2rM5mHXA

fca839c00a12 ("workqueue: warn if memory reclaim tries to flush
!WQ_MEM_RECLAIM workqueue") implemented flush dependency warning which
triggers if a PF_MEMALLOC task or WQ_MEM_RECLAIM workqueue tries to
flush a !WQ_MEM_RECLAIM workquee.

This assumes that workqueues marked with WQ_MEM_RECLAIM sit in memory
reclaim path and making it depend on something which may need more
memory to make forward progress can lead to deadlocks.  Unfortunately,
workqueues created with the legacy create*_workqueue() interface
always have WQ_MEM_RECLAIM regardless of whether they are depended
upon memory reclaim or not.  These spurious WQ_MEM_RECLAIM markings
cause spurious triggering of the flush dependency checks.

  WARNING: CPU: 0 PID: 6 at kernel/workqueue.c:2361 check_flush_dependency+0x138/0x144()
  workqueue: WQ_MEM_RECLAIM deferwq:deferred_probe_work_func is flushing !WQ_MEM_RECLAIM events:lru_add_drain_per_cpu
  ...
  Workqueue: deferwq deferred_probe_work_func
  [<c0017acc>] (unwind_backtrace) from [<c0013134>] (show_stack+0x10/0x14)
  [<c0013134>] (show_stack) from [<c0245f18>] (dump_stack+0x94/0xd4)
  [<c0245f18>] (dump_stack) from [<c0026f9c>] (warn_slowpath_common+0x80/0xb0)
  [<c0026f9c>] (warn_slowpath_common) from [<c0026ffc>] (warn_slowpath_fmt+0x30/0x40)
  [<c0026ffc>] (warn_slowpath_fmt) from [<c00390b8>] (check_flush_dependency+0x138/0x144)
  [<c00390b8>] (check_flush_dependency) from [<c0039ca0>] (flush_work+0x50/0x15c)
  [<c0039ca0>] (flush_work) from [<c00c51b0>] (lru_add_drain_all+0x130/0x180)
  [<c00c51b0>] (lru_add_drain_all) from [<c00f728c>] (migrate_prep+0x8/0x10)
  [<c00f728c>] (migrate_prep) from [<c00bfbc4>] (alloc_contig_range+0xd8/0x338)
  [<c00bfbc4>] (alloc_contig_range) from [<c00f8f18>] (cma_alloc+0xe0/0x1ac)
  [<c00f8f18>] (cma_alloc) from [<c001cac4>] (__alloc_from_contiguous+0x38/0xd8)
  [<c001cac4>] (__alloc_from_contiguous) from [<c001ceb4>] (__dma_alloc+0x240/0x278)
  [<c001ceb4>] (__dma_alloc) from [<c001cf78>] (arm_dma_alloc+0x54/0x5c)
  [<c001cf78>] (arm_dma_alloc) from [<c0355ea4>] (dmam_alloc_coherent+0xc0/0xec)
  [<c0355ea4>] (dmam_alloc_coherent) from [<c039cc4c>] (ahci_port_start+0x150/0x1dc)
  [<c039cc4c>] (ahci_port_start) from [<c0384734>] (ata_host_start.part.3+0xc8/0x1c8)
  [<c0384734>] (ata_host_start.part.3) from [<c03898dc>] (ata_host_activate+0x50/0x148)
  [<c03898dc>] (ata_host_activate) from [<c039d558>] (ahci_host_activate+0x44/0x114)
  [<c039d558>] (ahci_host_activate) from [<c039f05c>] (ahci_platform_init_host+0x1d8/0x3c8)
  [<c039f05c>] (ahci_platform_init_host) from [<c039e6bc>] (tegra_ahci_probe+0x448/0x4e8)
  [<c039e6bc>] (tegra_ahci_probe) from [<c0347058>] (platform_drv_probe+0x50/0xac)
  [<c0347058>] (platform_drv_probe) from [<c03458cc>] (driver_probe_device+0x214/0x2c0)
  [<c03458cc>] (driver_probe_device) from [<c0343cc0>] (bus_for_each_drv+0x60/0x94)
  [<c0343cc0>] (bus_for_each_drv) from [<c03455d8>] (__device_attach+0xb0/0x114)
  [<c03455d8>] (__device_attach) from [<c0344ab8>] (bus_probe_device+0x84/0x8c)
  [<c0344ab8>] (bus_probe_device) from [<c0344f48>] (deferred_probe_work_func+0x68/0x98)
  [<c0344f48>] (deferred_probe_work_func) from [<c003b738>] (process_one_work+0x120/0x3f8)
  [<c003b738>] (process_one_work) from [<c003ba48>] (worker_thread+0x38/0x55c)
  [<c003ba48>] (worker_thread) from [<c0040f14>] (kthread+0xdc/0xf4)
  [<c0040f14>] (kthread) from [<c000f778>] (ret_from_fork+0x14/0x3c)

Fix it by marking workqueues created via create*_workqueue() with
__WQ_LEGACY and disabling flush dependency checks on them.

Signed-off-by: Tejun Heo <tj-DgEjT+Ai2ygdnm+yROfE0A@public.gmane.org>
Reported-by: Thierry Reding <thierry.reding-Re5JQEeQqe8AvxtiuMwx3w@public.gmane.org>
Link: http://lkml.kernel.org/g/20160126173843.GA11115-AwZRO8vwLAwmlAP/+Wk3EA@public.gmane.org
---
Hello, Thierry.

Can youp please verify the fix?

Thanks.

 include/linux/workqueue.h |    9 +++++----
 kernel/workqueue.c        |    3 ++-
 2 files changed, 7 insertions(+), 5 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index 0e32bc7..ca73c50 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -311,6 +311,7 @@ enum {
 
 	__WQ_DRAINING		= 1 << 16, /* internal: workqueue is draining */
 	__WQ_ORDERED		= 1 << 17, /* internal: workqueue is ordered */
+	__WQ_LEGACY		= 1 << 18, /* internal: create*_workqueue() */
 
 	WQ_MAX_ACTIVE		= 512,	  /* I like 512, better ideas? */
 	WQ_MAX_UNBOUND_PER_CPU	= 4,	  /* 4 * #cpus for unbound wq */
@@ -411,12 +412,12 @@ __alloc_workqueue_key(const char *fmt, unsigned int flags, int max_active,
 	alloc_workqueue(fmt, WQ_UNBOUND | __WQ_ORDERED | (flags), 1, ##args)
 
 #define create_workqueue(name)						\
-	alloc_workqueue("%s", WQ_MEM_RECLAIM, 1, (name))
+	alloc_workqueue("%s", __WQ_LEGACY | WQ_MEM_RECLAIM, 1, (name))
 #define create_freezable_workqueue(name)				\
-	alloc_workqueue("%s", WQ_FREEZABLE | WQ_UNBOUND | WQ_MEM_RECLAIM, \
-			1, (name))
+	alloc_workqueue("%s", __WQ_LEGACY | WQ_FREEZABLE | WQ_UNBOUND |	\
+			WQ_MEM_RECLAIM, 1, (name))
 #define create_singlethread_workqueue(name)				\
-	alloc_ordered_workqueue("%s", WQ_MEM_RECLAIM, name)
+	alloc_ordered_workqueue("%s", __WQ_LEGACY | WQ_MEM_RECLAIM, name)
 
 extern void destroy_workqueue(struct workqueue_struct *wq);
 
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 61a0264..dc7faad 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -2355,7 +2355,8 @@ static void check_flush_dependency(struct workqueue_struct *target_wq,
 	WARN_ONCE(current->flags & PF_MEMALLOC,
 		  "workqueue: PF_MEMALLOC task %d(%s) is flushing !WQ_MEM_RECLAIM %s:%pf",
 		  current->pid, current->comm, target_wq->name, target_func);
-	WARN_ONCE(worker && (worker->current_pwq->wq->flags & WQ_MEM_RECLAIM),
+	WARN_ONCE(worker && ((worker->current_pwq->wq->flags &
+			      (WQ_MEM_RECLAIM | __WQ_LEGACY)) == WQ_MEM_RECLAIM),
 		  "workqueue: WQ_MEM_RECLAIM %s:%pf is flushing !WQ_MEM_RECLAIM %s:%pf",
 		  worker->current_pwq->wq->name, worker->current_func,
 		  target_wq->name, target_func);

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

* [PATCH wq/for-4.5-fixes] workqueue: skip flush dependency checks for legacy workqueues
@ 2016-01-29 10:59                     ` Tejun Heo
  0 siblings, 0 replies; 51+ messages in thread
From: Tejun Heo @ 2016-01-29 10:59 UTC (permalink / raw)
  To: Thierry Reding
  Cc: Peter Zijlstra, Ulrich Obergfell, Ingo Molnar, Andrew Morton,
	linux-kernel, kernel-team, Jon Hunter, linux-tegra

fca839c00a12 ("workqueue: warn if memory reclaim tries to flush
!WQ_MEM_RECLAIM workqueue") implemented flush dependency warning which
triggers if a PF_MEMALLOC task or WQ_MEM_RECLAIM workqueue tries to
flush a !WQ_MEM_RECLAIM workquee.

This assumes that workqueues marked with WQ_MEM_RECLAIM sit in memory
reclaim path and making it depend on something which may need more
memory to make forward progress can lead to deadlocks.  Unfortunately,
workqueues created with the legacy create*_workqueue() interface
always have WQ_MEM_RECLAIM regardless of whether they are depended
upon memory reclaim or not.  These spurious WQ_MEM_RECLAIM markings
cause spurious triggering of the flush dependency checks.

  WARNING: CPU: 0 PID: 6 at kernel/workqueue.c:2361 check_flush_dependency+0x138/0x144()
  workqueue: WQ_MEM_RECLAIM deferwq:deferred_probe_work_func is flushing !WQ_MEM_RECLAIM events:lru_add_drain_per_cpu
  ...
  Workqueue: deferwq deferred_probe_work_func
  [<c0017acc>] (unwind_backtrace) from [<c0013134>] (show_stack+0x10/0x14)
  [<c0013134>] (show_stack) from [<c0245f18>] (dump_stack+0x94/0xd4)
  [<c0245f18>] (dump_stack) from [<c0026f9c>] (warn_slowpath_common+0x80/0xb0)
  [<c0026f9c>] (warn_slowpath_common) from [<c0026ffc>] (warn_slowpath_fmt+0x30/0x40)
  [<c0026ffc>] (warn_slowpath_fmt) from [<c00390b8>] (check_flush_dependency+0x138/0x144)
  [<c00390b8>] (check_flush_dependency) from [<c0039ca0>] (flush_work+0x50/0x15c)
  [<c0039ca0>] (flush_work) from [<c00c51b0>] (lru_add_drain_all+0x130/0x180)
  [<c00c51b0>] (lru_add_drain_all) from [<c00f728c>] (migrate_prep+0x8/0x10)
  [<c00f728c>] (migrate_prep) from [<c00bfbc4>] (alloc_contig_range+0xd8/0x338)
  [<c00bfbc4>] (alloc_contig_range) from [<c00f8f18>] (cma_alloc+0xe0/0x1ac)
  [<c00f8f18>] (cma_alloc) from [<c001cac4>] (__alloc_from_contiguous+0x38/0xd8)
  [<c001cac4>] (__alloc_from_contiguous) from [<c001ceb4>] (__dma_alloc+0x240/0x278)
  [<c001ceb4>] (__dma_alloc) from [<c001cf78>] (arm_dma_alloc+0x54/0x5c)
  [<c001cf78>] (arm_dma_alloc) from [<c0355ea4>] (dmam_alloc_coherent+0xc0/0xec)
  [<c0355ea4>] (dmam_alloc_coherent) from [<c039cc4c>] (ahci_port_start+0x150/0x1dc)
  [<c039cc4c>] (ahci_port_start) from [<c0384734>] (ata_host_start.part.3+0xc8/0x1c8)
  [<c0384734>] (ata_host_start.part.3) from [<c03898dc>] (ata_host_activate+0x50/0x148)
  [<c03898dc>] (ata_host_activate) from [<c039d558>] (ahci_host_activate+0x44/0x114)
  [<c039d558>] (ahci_host_activate) from [<c039f05c>] (ahci_platform_init_host+0x1d8/0x3c8)
  [<c039f05c>] (ahci_platform_init_host) from [<c039e6bc>] (tegra_ahci_probe+0x448/0x4e8)
  [<c039e6bc>] (tegra_ahci_probe) from [<c0347058>] (platform_drv_probe+0x50/0xac)
  [<c0347058>] (platform_drv_probe) from [<c03458cc>] (driver_probe_device+0x214/0x2c0)
  [<c03458cc>] (driver_probe_device) from [<c0343cc0>] (bus_for_each_drv+0x60/0x94)
  [<c0343cc0>] (bus_for_each_drv) from [<c03455d8>] (__device_attach+0xb0/0x114)
  [<c03455d8>] (__device_attach) from [<c0344ab8>] (bus_probe_device+0x84/0x8c)
  [<c0344ab8>] (bus_probe_device) from [<c0344f48>] (deferred_probe_work_func+0x68/0x98)
  [<c0344f48>] (deferred_probe_work_func) from [<c003b738>] (process_one_work+0x120/0x3f8)
  [<c003b738>] (process_one_work) from [<c003ba48>] (worker_thread+0x38/0x55c)
  [<c003ba48>] (worker_thread) from [<c0040f14>] (kthread+0xdc/0xf4)
  [<c0040f14>] (kthread) from [<c000f778>] (ret_from_fork+0x14/0x3c)

Fix it by marking workqueues created via create*_workqueue() with
__WQ_LEGACY and disabling flush dependency checks on them.

Signed-off-by: Tejun Heo <tj@kernel.org>
Reported-by: Thierry Reding <thierry.reding@gmail.com>
Link: http://lkml.kernel.org/g/20160126173843.GA11115@ulmo.nvidia.com
---
Hello, Thierry.

Can youp please verify the fix?

Thanks.

 include/linux/workqueue.h |    9 +++++----
 kernel/workqueue.c        |    3 ++-
 2 files changed, 7 insertions(+), 5 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index 0e32bc7..ca73c50 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -311,6 +311,7 @@ enum {
 
 	__WQ_DRAINING		= 1 << 16, /* internal: workqueue is draining */
 	__WQ_ORDERED		= 1 << 17, /* internal: workqueue is ordered */
+	__WQ_LEGACY		= 1 << 18, /* internal: create*_workqueue() */
 
 	WQ_MAX_ACTIVE		= 512,	  /* I like 512, better ideas? */
 	WQ_MAX_UNBOUND_PER_CPU	= 4,	  /* 4 * #cpus for unbound wq */
@@ -411,12 +412,12 @@ __alloc_workqueue_key(const char *fmt, unsigned int flags, int max_active,
 	alloc_workqueue(fmt, WQ_UNBOUND | __WQ_ORDERED | (flags), 1, ##args)
 
 #define create_workqueue(name)						\
-	alloc_workqueue("%s", WQ_MEM_RECLAIM, 1, (name))
+	alloc_workqueue("%s", __WQ_LEGACY | WQ_MEM_RECLAIM, 1, (name))
 #define create_freezable_workqueue(name)				\
-	alloc_workqueue("%s", WQ_FREEZABLE | WQ_UNBOUND | WQ_MEM_RECLAIM, \
-			1, (name))
+	alloc_workqueue("%s", __WQ_LEGACY | WQ_FREEZABLE | WQ_UNBOUND |	\
+			WQ_MEM_RECLAIM, 1, (name))
 #define create_singlethread_workqueue(name)				\
-	alloc_ordered_workqueue("%s", WQ_MEM_RECLAIM, name)
+	alloc_ordered_workqueue("%s", __WQ_LEGACY | WQ_MEM_RECLAIM, name)
 
 extern void destroy_workqueue(struct workqueue_struct *wq);
 
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 61a0264..dc7faad 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -2355,7 +2355,8 @@ static void check_flush_dependency(struct workqueue_struct *target_wq,
 	WARN_ONCE(current->flags & PF_MEMALLOC,
 		  "workqueue: PF_MEMALLOC task %d(%s) is flushing !WQ_MEM_RECLAIM %s:%pf",
 		  current->pid, current->comm, target_wq->name, target_func);
-	WARN_ONCE(worker && (worker->current_pwq->wq->flags & WQ_MEM_RECLAIM),
+	WARN_ONCE(worker && ((worker->current_pwq->wq->flags &
+			      (WQ_MEM_RECLAIM | __WQ_LEGACY)) == WQ_MEM_RECLAIM),
 		  "workqueue: WQ_MEM_RECLAIM %s:%pf is flushing !WQ_MEM_RECLAIM %s:%pf",
 		  worker->current_pwq->wq->name, worker->current_func,
 		  target_wq->name, target_func);

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

* Re: [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue
  2016-01-28 10:12                     ` Peter Zijlstra
  (?)
@ 2016-01-29 11:09                         ` Tejun Heo
  -1 siblings, 0 replies; 51+ messages in thread
From: Tejun Heo @ 2016-01-29 11:09 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Thierry Reding, Ulrich Obergfell, Ingo Molnar, Andrew Morton,
	linux-kernel-u79uwXL29TY76Z2rM5mHXA, kernel-team-b10kYP2dOMg,
	Jon Hunter, linux-tegra-u79uwXL29TY76Z2rM5mHXA,
	rmk+kernel-lFZ/pmaqli7XmaaqVzeoHQ, Johannes Weiner,
	linux-mm-Bw31MaZKKs3YtjvyW6yDsg

Hello, Peter.

On Thu, Jan 28, 2016 at 11:12:10AM +0100, Peter Zijlstra wrote:
> On Tue, Jan 26, 2016 at 06:38:43PM +0100, Thierry Reding wrote:
> > > Task or work item involved in memory reclaim trying to flush a
> > > non-WQ_MEM_RECLAIM workqueue or one of its work items can lead to
> > > deadlock.  Trigger WARN_ONCE() if such conditions are detected.
> > I've started noticing the following during boot on some of the devices I
> > work with:
> > 
> > [    4.723705] WARNING: CPU: 0 PID: 6 at kernel/workqueue.c:2361 check_flush_dependency+0x138/0x144()
> > [    4.736818] workqueue: WQ_MEM_RECLAIM deferwq:deferred_probe_work_func is flushing !WQ_MEM_RECLAIM events:lru_add_drain_per_cpu
...
> Right, also, I think it makes sense to do lru_add_drain_all() from a
> WQ_MEM_RECLAIM workqueue, it is, after all, aiding in getting memory
> freed.
> 
> Does something like the below cure things?
> 
> TJ does this make sense to you?

The problem here is that deferwq which has nothing to do with memory
reclaim is marked with WQ_MEM_RECLAIM because it was created the old
create_singlethread_workqueue() which doesn't distinguish workqueues
which may be used on mem reclaim path and thus has to mark all as
needing forward progress guarantee.  I posted a patch to disable
disable flush dependency checks on those workqueues and there's a
outreachy project to weed out the users of the old interface, so
hopefully this won't be an issue soon.

As for whether lru drain should have WQ_MEM_RECLAIM, I'm not sure.
Cc'ing linux-mm.  The rule here is that any workquee which is depended
upon during memory reclaim should have WQ_MEM_RECLAIM set.  IOW, if a
work item failing to start execution under memory pressure can prevent
memory from being reclaimed, it should be scheduled on a
WQ_MEM_RECLAIM workqueue.  Would this be the case for
lru_add_drain_per_cpu()?

Thanks.

-- 
tejun

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

* Re: [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue
@ 2016-01-29 11:09                         ` Tejun Heo
  0 siblings, 0 replies; 51+ messages in thread
From: Tejun Heo @ 2016-01-29 11:09 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Thierry Reding, Ulrich Obergfell, Ingo Molnar, Andrew Morton,
	linux-kernel, kernel-team, Jon Hunter, linux-tegra, rmk+kernel,
	Johannes Weiner, linux-mm

Hello, Peter.

On Thu, Jan 28, 2016 at 11:12:10AM +0100, Peter Zijlstra wrote:
> On Tue, Jan 26, 2016 at 06:38:43PM +0100, Thierry Reding wrote:
> > > Task or work item involved in memory reclaim trying to flush a
> > > non-WQ_MEM_RECLAIM workqueue or one of its work items can lead to
> > > deadlock.  Trigger WARN_ONCE() if such conditions are detected.
> > I've started noticing the following during boot on some of the devices I
> > work with:
> > 
> > [    4.723705] WARNING: CPU: 0 PID: 6 at kernel/workqueue.c:2361 check_flush_dependency+0x138/0x144()
> > [    4.736818] workqueue: WQ_MEM_RECLAIM deferwq:deferred_probe_work_func is flushing !WQ_MEM_RECLAIM events:lru_add_drain_per_cpu
...
> Right, also, I think it makes sense to do lru_add_drain_all() from a
> WQ_MEM_RECLAIM workqueue, it is, after all, aiding in getting memory
> freed.
> 
> Does something like the below cure things?
> 
> TJ does this make sense to you?

The problem here is that deferwq which has nothing to do with memory
reclaim is marked with WQ_MEM_RECLAIM because it was created the old
create_singlethread_workqueue() which doesn't distinguish workqueues
which may be used on mem reclaim path and thus has to mark all as
needing forward progress guarantee.  I posted a patch to disable
disable flush dependency checks on those workqueues and there's a
outreachy project to weed out the users of the old interface, so
hopefully this won't be an issue soon.

As for whether lru drain should have WQ_MEM_RECLAIM, I'm not sure.
Cc'ing linux-mm.  The rule here is that any workquee which is depended
upon during memory reclaim should have WQ_MEM_RECLAIM set.  IOW, if a
work item failing to start execution under memory pressure can prevent
memory from being reclaimed, it should be scheduled on a
WQ_MEM_RECLAIM workqueue.  Would this be the case for
lru_add_drain_per_cpu()?

Thanks.

-- 
tejun

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

* Re: [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue
@ 2016-01-29 11:09                         ` Tejun Heo
  0 siblings, 0 replies; 51+ messages in thread
From: Tejun Heo @ 2016-01-29 11:09 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Thierry Reding, Ulrich Obergfell, Ingo Molnar, Andrew Morton,
	linux-kernel, kernel-team, Jon Hunter, linux-tegra, rmk+kernel,
	Johannes Weiner, linux-mm

Hello, Peter.

On Thu, Jan 28, 2016 at 11:12:10AM +0100, Peter Zijlstra wrote:
> On Tue, Jan 26, 2016 at 06:38:43PM +0100, Thierry Reding wrote:
> > > Task or work item involved in memory reclaim trying to flush a
> > > non-WQ_MEM_RECLAIM workqueue or one of its work items can lead to
> > > deadlock.  Trigger WARN_ONCE() if such conditions are detected.
> > I've started noticing the following during boot on some of the devices I
> > work with:
> > 
> > [    4.723705] WARNING: CPU: 0 PID: 6 at kernel/workqueue.c:2361 check_flush_dependency+0x138/0x144()
> > [    4.736818] workqueue: WQ_MEM_RECLAIM deferwq:deferred_probe_work_func is flushing !WQ_MEM_RECLAIM events:lru_add_drain_per_cpu
...
> Right, also, I think it makes sense to do lru_add_drain_all() from a
> WQ_MEM_RECLAIM workqueue, it is, after all, aiding in getting memory
> freed.
> 
> Does something like the below cure things?
> 
> TJ does this make sense to you?

The problem here is that deferwq which has nothing to do with memory
reclaim is marked with WQ_MEM_RECLAIM because it was created the old
create_singlethread_workqueue() which doesn't distinguish workqueues
which may be used on mem reclaim path and thus has to mark all as
needing forward progress guarantee.  I posted a patch to disable
disable flush dependency checks on those workqueues and there's a
outreachy project to weed out the users of the old interface, so
hopefully this won't be an issue soon.

As for whether lru drain should have WQ_MEM_RECLAIM, I'm not sure.
Cc'ing linux-mm.  The rule here is that any workquee which is depended
upon during memory reclaim should have WQ_MEM_RECLAIM set.  IOW, if a
work item failing to start execution under memory pressure can prevent
memory from being reclaimed, it should be scheduled on a
WQ_MEM_RECLAIM workqueue.  Would this be the case for
lru_add_drain_per_cpu()?

Thanks.

-- 
tejun

--
To unsubscribe, send a message with 'unsubscribe linux-mm' in
the body to majordomo@kvack.org.  For more info on Linux MM,
see: http://www.linux-mm.org/ .
Don't email: <a href=mailto:"dont@kvack.org"> email@kvack.org </a>

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

* Re: [PATCH wq/for-4.5-fixes] workqueue: skip flush dependency checks for legacy workqueues
  2016-01-29 10:59                     ` Tejun Heo
  (?)
@ 2016-01-29 15:07                     ` Thierry Reding
  -1 siblings, 0 replies; 51+ messages in thread
From: Thierry Reding @ 2016-01-29 15:07 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Peter Zijlstra, Ulrich Obergfell, Ingo Molnar, Andrew Morton,
	linux-kernel, kernel-team, Jon Hunter, linux-tegra

[-- Attachment #1: Type: text/plain, Size: 4038 bytes --]

On Fri, Jan 29, 2016 at 05:59:46AM -0500, Tejun Heo wrote:
> fca839c00a12 ("workqueue: warn if memory reclaim tries to flush
> !WQ_MEM_RECLAIM workqueue") implemented flush dependency warning which
> triggers if a PF_MEMALLOC task or WQ_MEM_RECLAIM workqueue tries to
> flush a !WQ_MEM_RECLAIM workquee.
> 
> This assumes that workqueues marked with WQ_MEM_RECLAIM sit in memory
> reclaim path and making it depend on something which may need more
> memory to make forward progress can lead to deadlocks.  Unfortunately,
> workqueues created with the legacy create*_workqueue() interface
> always have WQ_MEM_RECLAIM regardless of whether they are depended
> upon memory reclaim or not.  These spurious WQ_MEM_RECLAIM markings
> cause spurious triggering of the flush dependency checks.
> 
>   WARNING: CPU: 0 PID: 6 at kernel/workqueue.c:2361 check_flush_dependency+0x138/0x144()
>   workqueue: WQ_MEM_RECLAIM deferwq:deferred_probe_work_func is flushing !WQ_MEM_RECLAIM events:lru_add_drain_per_cpu
>   ...
>   Workqueue: deferwq deferred_probe_work_func
>   [<c0017acc>] (unwind_backtrace) from [<c0013134>] (show_stack+0x10/0x14)
>   [<c0013134>] (show_stack) from [<c0245f18>] (dump_stack+0x94/0xd4)
>   [<c0245f18>] (dump_stack) from [<c0026f9c>] (warn_slowpath_common+0x80/0xb0)
>   [<c0026f9c>] (warn_slowpath_common) from [<c0026ffc>] (warn_slowpath_fmt+0x30/0x40)
>   [<c0026ffc>] (warn_slowpath_fmt) from [<c00390b8>] (check_flush_dependency+0x138/0x144)
>   [<c00390b8>] (check_flush_dependency) from [<c0039ca0>] (flush_work+0x50/0x15c)
>   [<c0039ca0>] (flush_work) from [<c00c51b0>] (lru_add_drain_all+0x130/0x180)
>   [<c00c51b0>] (lru_add_drain_all) from [<c00f728c>] (migrate_prep+0x8/0x10)
>   [<c00f728c>] (migrate_prep) from [<c00bfbc4>] (alloc_contig_range+0xd8/0x338)
>   [<c00bfbc4>] (alloc_contig_range) from [<c00f8f18>] (cma_alloc+0xe0/0x1ac)
>   [<c00f8f18>] (cma_alloc) from [<c001cac4>] (__alloc_from_contiguous+0x38/0xd8)
>   [<c001cac4>] (__alloc_from_contiguous) from [<c001ceb4>] (__dma_alloc+0x240/0x278)
>   [<c001ceb4>] (__dma_alloc) from [<c001cf78>] (arm_dma_alloc+0x54/0x5c)
>   [<c001cf78>] (arm_dma_alloc) from [<c0355ea4>] (dmam_alloc_coherent+0xc0/0xec)
>   [<c0355ea4>] (dmam_alloc_coherent) from [<c039cc4c>] (ahci_port_start+0x150/0x1dc)
>   [<c039cc4c>] (ahci_port_start) from [<c0384734>] (ata_host_start.part.3+0xc8/0x1c8)
>   [<c0384734>] (ata_host_start.part.3) from [<c03898dc>] (ata_host_activate+0x50/0x148)
>   [<c03898dc>] (ata_host_activate) from [<c039d558>] (ahci_host_activate+0x44/0x114)
>   [<c039d558>] (ahci_host_activate) from [<c039f05c>] (ahci_platform_init_host+0x1d8/0x3c8)
>   [<c039f05c>] (ahci_platform_init_host) from [<c039e6bc>] (tegra_ahci_probe+0x448/0x4e8)
>   [<c039e6bc>] (tegra_ahci_probe) from [<c0347058>] (platform_drv_probe+0x50/0xac)
>   [<c0347058>] (platform_drv_probe) from [<c03458cc>] (driver_probe_device+0x214/0x2c0)
>   [<c03458cc>] (driver_probe_device) from [<c0343cc0>] (bus_for_each_drv+0x60/0x94)
>   [<c0343cc0>] (bus_for_each_drv) from [<c03455d8>] (__device_attach+0xb0/0x114)
>   [<c03455d8>] (__device_attach) from [<c0344ab8>] (bus_probe_device+0x84/0x8c)
>   [<c0344ab8>] (bus_probe_device) from [<c0344f48>] (deferred_probe_work_func+0x68/0x98)
>   [<c0344f48>] (deferred_probe_work_func) from [<c003b738>] (process_one_work+0x120/0x3f8)
>   [<c003b738>] (process_one_work) from [<c003ba48>] (worker_thread+0x38/0x55c)
>   [<c003ba48>] (worker_thread) from [<c0040f14>] (kthread+0xdc/0xf4)
>   [<c0040f14>] (kthread) from [<c000f778>] (ret_from_fork+0x14/0x3c)
> 
> Fix it by marking workqueues created via create*_workqueue() with
> __WQ_LEGACY and disabling flush dependency checks on them.
> 
> Signed-off-by: Tejun Heo <tj@kernel.org>
> Reported-by: Thierry Reding <thierry.reding@gmail.com>
> Link: http://lkml.kernel.org/g/20160126173843.GA11115@ulmo.nvidia.com

Thanks for fixing this, everything is back to normal:

Tested-by: Thierry Reding <treding@nvidia.com>

[-- Attachment #2: signature.asc --]
[-- Type: application/pgp-signature, Size: 819 bytes --]

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

* Re: [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue
  2016-01-29 11:09                         ` Tejun Heo
  (?)
@ 2016-01-29 15:17                             ` Peter Zijlstra
  -1 siblings, 0 replies; 51+ messages in thread
From: Peter Zijlstra @ 2016-01-29 15:17 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Thierry Reding, Ulrich Obergfell, Ingo Molnar, Andrew Morton,
	linux-kernel-u79uwXL29TY76Z2rM5mHXA, kernel-team-b10kYP2dOMg,
	Jon Hunter, linux-tegra-u79uwXL29TY76Z2rM5mHXA,
	rmk+kernel-lFZ/pmaqli7XmaaqVzeoHQ, Johannes Weiner,
	linux-mm-Bw31MaZKKs3YtjvyW6yDsg

On Fri, Jan 29, 2016 at 06:09:41AM -0500, Tejun Heo wrote:
>  I posted a patch to disable
> disable flush dependency checks on those workqueues and there's a
> outreachy project to weed out the users of the old interface, so
> hopefully this won't be an issue soon.

Will that same project review all workqueue users for the strict per-cpu
stuff, so we can finally kill that weird stuff you do on hotplug?

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

* Re: [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue
@ 2016-01-29 15:17                             ` Peter Zijlstra
  0 siblings, 0 replies; 51+ messages in thread
From: Peter Zijlstra @ 2016-01-29 15:17 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Thierry Reding, Ulrich Obergfell, Ingo Molnar, Andrew Morton,
	linux-kernel, kernel-team, Jon Hunter, linux-tegra, rmk+kernel,
	Johannes Weiner, linux-mm

On Fri, Jan 29, 2016 at 06:09:41AM -0500, Tejun Heo wrote:
>  I posted a patch to disable
> disable flush dependency checks on those workqueues and there's a
> outreachy project to weed out the users of the old interface, so
> hopefully this won't be an issue soon.

Will that same project review all workqueue users for the strict per-cpu
stuff, so we can finally kill that weird stuff you do on hotplug?

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

* Re: [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue
@ 2016-01-29 15:17                             ` Peter Zijlstra
  0 siblings, 0 replies; 51+ messages in thread
From: Peter Zijlstra @ 2016-01-29 15:17 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Thierry Reding, Ulrich Obergfell, Ingo Molnar, Andrew Morton,
	linux-kernel, kernel-team, Jon Hunter, linux-tegra, rmk+kernel,
	Johannes Weiner, linux-mm

On Fri, Jan 29, 2016 at 06:09:41AM -0500, Tejun Heo wrote:
>  I posted a patch to disable
> disable flush dependency checks on those workqueues and there's a
> outreachy project to weed out the users of the old interface, so
> hopefully this won't be an issue soon.

Will that same project review all workqueue users for the strict per-cpu
stuff, so we can finally kill that weird stuff you do on hotplug?

--
To unsubscribe, send a message with 'unsubscribe linux-mm' in
the body to majordomo@kvack.org.  For more info on Linux MM,
see: http://www.linux-mm.org/ .
Don't email: <a href=mailto:"dont@kvack.org"> email@kvack.org </a>

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

* Re: [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue
  2016-01-29 15:17                             ` Peter Zijlstra
@ 2016-01-29 18:28                               ` Tejun Heo
  -1 siblings, 0 replies; 51+ messages in thread
From: Tejun Heo @ 2016-01-29 18:28 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Thierry Reding, Ulrich Obergfell, Ingo Molnar, Andrew Morton,
	linux-kernel, kernel-team, Jon Hunter, linux-tegra, rmk+kernel,
	Johannes Weiner, linux-mm

Hey, Peter.

On Fri, Jan 29, 2016 at 04:17:39PM +0100, Peter Zijlstra wrote:
> On Fri, Jan 29, 2016 at 06:09:41AM -0500, Tejun Heo wrote:
> >  I posted a patch to disable
> > disable flush dependency checks on those workqueues and there's a
> > outreachy project to weed out the users of the old interface, so
> > hopefully this won't be an issue soon.
> 
> Will that same project review all workqueue users for the strict per-cpu
> stuff, so we can finally kill that weird stuff you do on hotplug?

Unfortunately not.  We do want to distinguish cpu-affine for
correctness and as an optimization; however, making that distinction
is unlikely to make the dynamic worker affinity binding go away.  We
can't forcifully shut down workers which are executing work items
which are affine as an optimization when the CPU goes down.

Thanks.

-- 
tejun

--
To unsubscribe, send a message with 'unsubscribe linux-mm' in
the body to majordomo@kvack.org.  For more info on Linux MM,
see: http://www.linux-mm.org/ .
Don't email: <a href=mailto:"dont@kvack.org"> email@kvack.org </a>

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

* Re: [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue
@ 2016-01-29 18:28                               ` Tejun Heo
  0 siblings, 0 replies; 51+ messages in thread
From: Tejun Heo @ 2016-01-29 18:28 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Thierry Reding, Ulrich Obergfell, Ingo Molnar, Andrew Morton,
	linux-kernel, kernel-team, Jon Hunter, linux-tegra, rmk+kernel,
	Johannes Weiner, linux-mm

Hey, Peter.

On Fri, Jan 29, 2016 at 04:17:39PM +0100, Peter Zijlstra wrote:
> On Fri, Jan 29, 2016 at 06:09:41AM -0500, Tejun Heo wrote:
> >  I posted a patch to disable
> > disable flush dependency checks on those workqueues and there's a
> > outreachy project to weed out the users of the old interface, so
> > hopefully this won't be an issue soon.
> 
> Will that same project review all workqueue users for the strict per-cpu
> stuff, so we can finally kill that weird stuff you do on hotplug?

Unfortunately not.  We do want to distinguish cpu-affine for
correctness and as an optimization; however, making that distinction
is unlikely to make the dynamic worker affinity binding go away.  We
can't forcifully shut down workers which are executing work items
which are affine as an optimization when the CPU goes down.

Thanks.

-- 
tejun

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

* Re: [PATCH wq/for-4.5-fixes] workqueue: skip flush dependency checks for legacy workqueues
  2016-01-29 10:59                     ` Tejun Heo
  (?)
  (?)
@ 2016-01-29 18:32                     ` Tejun Heo
  -1 siblings, 0 replies; 51+ messages in thread
From: Tejun Heo @ 2016-01-29 18:32 UTC (permalink / raw)
  To: Thierry Reding
  Cc: Peter Zijlstra, Ulrich Obergfell, Ingo Molnar, Andrew Morton,
	linux-kernel, kernel-team, Jon Hunter, linux-tegra

On Fri, Jan 29, 2016 at 05:59:46AM -0500, Tejun Heo wrote:
> fca839c00a12 ("workqueue: warn if memory reclaim tries to flush
> !WQ_MEM_RECLAIM workqueue") implemented flush dependency warning which
> triggers if a PF_MEMALLOC task or WQ_MEM_RECLAIM workqueue tries to
> flush a !WQ_MEM_RECLAIM workquee.
> 
> This assumes that workqueues marked with WQ_MEM_RECLAIM sit in memory
> reclaim path and making it depend on something which may need more
> memory to make forward progress can lead to deadlocks.  Unfortunately,
> workqueues created with the legacy create*_workqueue() interface
> always have WQ_MEM_RECLAIM regardless of whether they are depended
> upon memory reclaim or not.  These spurious WQ_MEM_RECLAIM markings
> cause spurious triggering of the flush dependency checks.
...

Applied to wq/for-4.5-fixes.

Thanks.

-- 
tejun

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

* Re: [PATCH wq/for-4.5-fixes] workqueue: skip flush dependency checks for legacy workqueues
  2016-01-29 10:59                     ` Tejun Heo
@ 2016-02-02  6:54                         ` Archit Taneja
  -1 siblings, 0 replies; 51+ messages in thread
From: Archit Taneja @ 2016-02-02  6:54 UTC (permalink / raw)
  To: Tejun Heo, Thierry Reding
  Cc: Peter Zijlstra, Ulrich Obergfell, Ingo Molnar, Andrew Morton,
	linux-kernel-u79uwXL29TY76Z2rM5mHXA, kernel-team-b10kYP2dOMg,
	Jon Hunter, linux-tegra-u79uwXL29TY76Z2rM5mHXA



On 01/29/2016 04:29 PM, Tejun Heo wrote:
> fca839c00a12 ("workqueue: warn if memory reclaim tries to flush
> !WQ_MEM_RECLAIM workqueue") implemented flush dependency warning which
> triggers if a PF_MEMALLOC task or WQ_MEM_RECLAIM workqueue tries to
> flush a !WQ_MEM_RECLAIM workquee.
>
> This assumes that workqueues marked with WQ_MEM_RECLAIM sit in memory
> reclaim path and making it depend on something which may need more
> memory to make forward progress can lead to deadlocks.  Unfortunately,
> workqueues created with the legacy create*_workqueue() interface
> always have WQ_MEM_RECLAIM regardless of whether they are depended
> upon memory reclaim or not.  These spurious WQ_MEM_RECLAIM markings
> cause spurious triggering of the flush dependency checks.
>
>    WARNING: CPU: 0 PID: 6 at kernel/workqueue.c:2361 check_flush_dependency+0x138/0x144()
>    workqueue: WQ_MEM_RECLAIM deferwq:deferred_probe_work_func is flushing !WQ_MEM_RECLAIM events:lru_add_drain_per_cpu
>    ...
>    Workqueue: deferwq deferred_probe_work_func
>    [<c0017acc>] (unwind_backtrace) from [<c0013134>] (show_stack+0x10/0x14)
>    [<c0013134>] (show_stack) from [<c0245f18>] (dump_stack+0x94/0xd4)
>    [<c0245f18>] (dump_stack) from [<c0026f9c>] (warn_slowpath_common+0x80/0xb0)
>    [<c0026f9c>] (warn_slowpath_common) from [<c0026ffc>] (warn_slowpath_fmt+0x30/0x40)
>    [<c0026ffc>] (warn_slowpath_fmt) from [<c00390b8>] (check_flush_dependency+0x138/0x144)
>    [<c00390b8>] (check_flush_dependency) from [<c0039ca0>] (flush_work+0x50/0x15c)
>    [<c0039ca0>] (flush_work) from [<c00c51b0>] (lru_add_drain_all+0x130/0x180)
>    [<c00c51b0>] (lru_add_drain_all) from [<c00f728c>] (migrate_prep+0x8/0x10)
>    [<c00f728c>] (migrate_prep) from [<c00bfbc4>] (alloc_contig_range+0xd8/0x338)
>    [<c00bfbc4>] (alloc_contig_range) from [<c00f8f18>] (cma_alloc+0xe0/0x1ac)
>    [<c00f8f18>] (cma_alloc) from [<c001cac4>] (__alloc_from_contiguous+0x38/0xd8)
>    [<c001cac4>] (__alloc_from_contiguous) from [<c001ceb4>] (__dma_alloc+0x240/0x278)
>    [<c001ceb4>] (__dma_alloc) from [<c001cf78>] (arm_dma_alloc+0x54/0x5c)
>    [<c001cf78>] (arm_dma_alloc) from [<c0355ea4>] (dmam_alloc_coherent+0xc0/0xec)
>    [<c0355ea4>] (dmam_alloc_coherent) from [<c039cc4c>] (ahci_port_start+0x150/0x1dc)
>    [<c039cc4c>] (ahci_port_start) from [<c0384734>] (ata_host_start.part.3+0xc8/0x1c8)
>    [<c0384734>] (ata_host_start.part.3) from [<c03898dc>] (ata_host_activate+0x50/0x148)
>    [<c03898dc>] (ata_host_activate) from [<c039d558>] (ahci_host_activate+0x44/0x114)
>    [<c039d558>] (ahci_host_activate) from [<c039f05c>] (ahci_platform_init_host+0x1d8/0x3c8)
>    [<c039f05c>] (ahci_platform_init_host) from [<c039e6bc>] (tegra_ahci_probe+0x448/0x4e8)
>    [<c039e6bc>] (tegra_ahci_probe) from [<c0347058>] (platform_drv_probe+0x50/0xac)
>    [<c0347058>] (platform_drv_probe) from [<c03458cc>] (driver_probe_device+0x214/0x2c0)
>    [<c03458cc>] (driver_probe_device) from [<c0343cc0>] (bus_for_each_drv+0x60/0x94)
>    [<c0343cc0>] (bus_for_each_drv) from [<c03455d8>] (__device_attach+0xb0/0x114)
>    [<c03455d8>] (__device_attach) from [<c0344ab8>] (bus_probe_device+0x84/0x8c)
>    [<c0344ab8>] (bus_probe_device) from [<c0344f48>] (deferred_probe_work_func+0x68/0x98)
>    [<c0344f48>] (deferred_probe_work_func) from [<c003b738>] (process_one_work+0x120/0x3f8)
>    [<c003b738>] (process_one_work) from [<c003ba48>] (worker_thread+0x38/0x55c)
>    [<c003ba48>] (worker_thread) from [<c0040f14>] (kthread+0xdc/0xf4)
>    [<c0040f14>] (kthread) from [<c000f778>] (ret_from_fork+0x14/0x3c)
>
> Fix it by marking workqueues created via create*_workqueue() with
> __WQ_LEGACY and disabling flush dependency checks on them.
>
> Signed-off-by: Tejun Heo <tj-DgEjT+Ai2ygdnm+yROfE0A@public.gmane.org>
> Reported-by: Thierry Reding <thierry.reding-Re5JQEeQqe8AvxtiuMwx3w@public.gmane.org>
> Link: http://lkml.kernel.org/g/20160126173843.GA11115-AwZRO8vwLAwmlAP/+Wk3EA@public.gmane.org
> ---
> Hello, Thierry.
>
> Can youp please verify the fix?

This fixes a similar backtrace observed when the drm/msm driver
tries to allocate a vram buffer via cma.

Thanks,
Archit

>
> Thanks.
>
>   include/linux/workqueue.h |    9 +++++----
>   kernel/workqueue.c        |    3 ++-
>   2 files changed, 7 insertions(+), 5 deletions(-)
>
> diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
> index 0e32bc7..ca73c50 100644
> --- a/include/linux/workqueue.h
> +++ b/include/linux/workqueue.h
> @@ -311,6 +311,7 @@ enum {
>
>   	__WQ_DRAINING		= 1 << 16, /* internal: workqueue is draining */
>   	__WQ_ORDERED		= 1 << 17, /* internal: workqueue is ordered */
> +	__WQ_LEGACY		= 1 << 18, /* internal: create*_workqueue() */
>
>   	WQ_MAX_ACTIVE		= 512,	  /* I like 512, better ideas? */
>   	WQ_MAX_UNBOUND_PER_CPU	= 4,	  /* 4 * #cpus for unbound wq */
> @@ -411,12 +412,12 @@ __alloc_workqueue_key(const char *fmt, unsigned int flags, int max_active,
>   	alloc_workqueue(fmt, WQ_UNBOUND | __WQ_ORDERED | (flags), 1, ##args)
>
>   #define create_workqueue(name)						\
> -	alloc_workqueue("%s", WQ_MEM_RECLAIM, 1, (name))
> +	alloc_workqueue("%s", __WQ_LEGACY | WQ_MEM_RECLAIM, 1, (name))
>   #define create_freezable_workqueue(name)				\
> -	alloc_workqueue("%s", WQ_FREEZABLE | WQ_UNBOUND | WQ_MEM_RECLAIM, \
> -			1, (name))
> +	alloc_workqueue("%s", __WQ_LEGACY | WQ_FREEZABLE | WQ_UNBOUND |	\
> +			WQ_MEM_RECLAIM, 1, (name))
>   #define create_singlethread_workqueue(name)				\
> -	alloc_ordered_workqueue("%s", WQ_MEM_RECLAIM, name)
> +	alloc_ordered_workqueue("%s", __WQ_LEGACY | WQ_MEM_RECLAIM, name)
>
>   extern void destroy_workqueue(struct workqueue_struct *wq);
>
> diff --git a/kernel/workqueue.c b/kernel/workqueue.c
> index 61a0264..dc7faad 100644
> --- a/kernel/workqueue.c
> +++ b/kernel/workqueue.c
> @@ -2355,7 +2355,8 @@ static void check_flush_dependency(struct workqueue_struct *target_wq,
>   	WARN_ONCE(current->flags & PF_MEMALLOC,
>   		  "workqueue: PF_MEMALLOC task %d(%s) is flushing !WQ_MEM_RECLAIM %s:%pf",
>   		  current->pid, current->comm, target_wq->name, target_func);
> -	WARN_ONCE(worker && (worker->current_pwq->wq->flags & WQ_MEM_RECLAIM),
> +	WARN_ONCE(worker && ((worker->current_pwq->wq->flags &
> +			      (WQ_MEM_RECLAIM | __WQ_LEGACY)) == WQ_MEM_RECLAIM),
>   		  "workqueue: WQ_MEM_RECLAIM %s:%pf is flushing !WQ_MEM_RECLAIM %s:%pf",
>   		  worker->current_pwq->wq->name, worker->current_func,
>   		  target_wq->name, target_func);
>

-- 
The Qualcomm Innovation Center, Inc. is a member of the Code Aurora 
Forum, hosted by The Linux Foundation

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

* Re: [PATCH wq/for-4.5-fixes] workqueue: skip flush dependency checks for legacy workqueues
@ 2016-02-02  6:54                         ` Archit Taneja
  0 siblings, 0 replies; 51+ messages in thread
From: Archit Taneja @ 2016-02-02  6:54 UTC (permalink / raw)
  To: Tejun Heo, Thierry Reding
  Cc: Peter Zijlstra, Ulrich Obergfell, Ingo Molnar, Andrew Morton,
	linux-kernel, kernel-team, Jon Hunter, linux-tegra



On 01/29/2016 04:29 PM, Tejun Heo wrote:
> fca839c00a12 ("workqueue: warn if memory reclaim tries to flush
> !WQ_MEM_RECLAIM workqueue") implemented flush dependency warning which
> triggers if a PF_MEMALLOC task or WQ_MEM_RECLAIM workqueue tries to
> flush a !WQ_MEM_RECLAIM workquee.
>
> This assumes that workqueues marked with WQ_MEM_RECLAIM sit in memory
> reclaim path and making it depend on something which may need more
> memory to make forward progress can lead to deadlocks.  Unfortunately,
> workqueues created with the legacy create*_workqueue() interface
> always have WQ_MEM_RECLAIM regardless of whether they are depended
> upon memory reclaim or not.  These spurious WQ_MEM_RECLAIM markings
> cause spurious triggering of the flush dependency checks.
>
>    WARNING: CPU: 0 PID: 6 at kernel/workqueue.c:2361 check_flush_dependency+0x138/0x144()
>    workqueue: WQ_MEM_RECLAIM deferwq:deferred_probe_work_func is flushing !WQ_MEM_RECLAIM events:lru_add_drain_per_cpu
>    ...
>    Workqueue: deferwq deferred_probe_work_func
>    [<c0017acc>] (unwind_backtrace) from [<c0013134>] (show_stack+0x10/0x14)
>    [<c0013134>] (show_stack) from [<c0245f18>] (dump_stack+0x94/0xd4)
>    [<c0245f18>] (dump_stack) from [<c0026f9c>] (warn_slowpath_common+0x80/0xb0)
>    [<c0026f9c>] (warn_slowpath_common) from [<c0026ffc>] (warn_slowpath_fmt+0x30/0x40)
>    [<c0026ffc>] (warn_slowpath_fmt) from [<c00390b8>] (check_flush_dependency+0x138/0x144)
>    [<c00390b8>] (check_flush_dependency) from [<c0039ca0>] (flush_work+0x50/0x15c)
>    [<c0039ca0>] (flush_work) from [<c00c51b0>] (lru_add_drain_all+0x130/0x180)
>    [<c00c51b0>] (lru_add_drain_all) from [<c00f728c>] (migrate_prep+0x8/0x10)
>    [<c00f728c>] (migrate_prep) from [<c00bfbc4>] (alloc_contig_range+0xd8/0x338)
>    [<c00bfbc4>] (alloc_contig_range) from [<c00f8f18>] (cma_alloc+0xe0/0x1ac)
>    [<c00f8f18>] (cma_alloc) from [<c001cac4>] (__alloc_from_contiguous+0x38/0xd8)
>    [<c001cac4>] (__alloc_from_contiguous) from [<c001ceb4>] (__dma_alloc+0x240/0x278)
>    [<c001ceb4>] (__dma_alloc) from [<c001cf78>] (arm_dma_alloc+0x54/0x5c)
>    [<c001cf78>] (arm_dma_alloc) from [<c0355ea4>] (dmam_alloc_coherent+0xc0/0xec)
>    [<c0355ea4>] (dmam_alloc_coherent) from [<c039cc4c>] (ahci_port_start+0x150/0x1dc)
>    [<c039cc4c>] (ahci_port_start) from [<c0384734>] (ata_host_start.part.3+0xc8/0x1c8)
>    [<c0384734>] (ata_host_start.part.3) from [<c03898dc>] (ata_host_activate+0x50/0x148)
>    [<c03898dc>] (ata_host_activate) from [<c039d558>] (ahci_host_activate+0x44/0x114)
>    [<c039d558>] (ahci_host_activate) from [<c039f05c>] (ahci_platform_init_host+0x1d8/0x3c8)
>    [<c039f05c>] (ahci_platform_init_host) from [<c039e6bc>] (tegra_ahci_probe+0x448/0x4e8)
>    [<c039e6bc>] (tegra_ahci_probe) from [<c0347058>] (platform_drv_probe+0x50/0xac)
>    [<c0347058>] (platform_drv_probe) from [<c03458cc>] (driver_probe_device+0x214/0x2c0)
>    [<c03458cc>] (driver_probe_device) from [<c0343cc0>] (bus_for_each_drv+0x60/0x94)
>    [<c0343cc0>] (bus_for_each_drv) from [<c03455d8>] (__device_attach+0xb0/0x114)
>    [<c03455d8>] (__device_attach) from [<c0344ab8>] (bus_probe_device+0x84/0x8c)
>    [<c0344ab8>] (bus_probe_device) from [<c0344f48>] (deferred_probe_work_func+0x68/0x98)
>    [<c0344f48>] (deferred_probe_work_func) from [<c003b738>] (process_one_work+0x120/0x3f8)
>    [<c003b738>] (process_one_work) from [<c003ba48>] (worker_thread+0x38/0x55c)
>    [<c003ba48>] (worker_thread) from [<c0040f14>] (kthread+0xdc/0xf4)
>    [<c0040f14>] (kthread) from [<c000f778>] (ret_from_fork+0x14/0x3c)
>
> Fix it by marking workqueues created via create*_workqueue() with
> __WQ_LEGACY and disabling flush dependency checks on them.
>
> Signed-off-by: Tejun Heo <tj@kernel.org>
> Reported-by: Thierry Reding <thierry.reding@gmail.com>
> Link: http://lkml.kernel.org/g/20160126173843.GA11115@ulmo.nvidia.com
> ---
> Hello, Thierry.
>
> Can youp please verify the fix?

This fixes a similar backtrace observed when the drm/msm driver
tries to allocate a vram buffer via cma.

Thanks,
Archit

>
> Thanks.
>
>   include/linux/workqueue.h |    9 +++++----
>   kernel/workqueue.c        |    3 ++-
>   2 files changed, 7 insertions(+), 5 deletions(-)
>
> diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
> index 0e32bc7..ca73c50 100644
> --- a/include/linux/workqueue.h
> +++ b/include/linux/workqueue.h
> @@ -311,6 +311,7 @@ enum {
>
>   	__WQ_DRAINING		= 1 << 16, /* internal: workqueue is draining */
>   	__WQ_ORDERED		= 1 << 17, /* internal: workqueue is ordered */
> +	__WQ_LEGACY		= 1 << 18, /* internal: create*_workqueue() */
>
>   	WQ_MAX_ACTIVE		= 512,	  /* I like 512, better ideas? */
>   	WQ_MAX_UNBOUND_PER_CPU	= 4,	  /* 4 * #cpus for unbound wq */
> @@ -411,12 +412,12 @@ __alloc_workqueue_key(const char *fmt, unsigned int flags, int max_active,
>   	alloc_workqueue(fmt, WQ_UNBOUND | __WQ_ORDERED | (flags), 1, ##args)
>
>   #define create_workqueue(name)						\
> -	alloc_workqueue("%s", WQ_MEM_RECLAIM, 1, (name))
> +	alloc_workqueue("%s", __WQ_LEGACY | WQ_MEM_RECLAIM, 1, (name))
>   #define create_freezable_workqueue(name)				\
> -	alloc_workqueue("%s", WQ_FREEZABLE | WQ_UNBOUND | WQ_MEM_RECLAIM, \
> -			1, (name))
> +	alloc_workqueue("%s", __WQ_LEGACY | WQ_FREEZABLE | WQ_UNBOUND |	\
> +			WQ_MEM_RECLAIM, 1, (name))
>   #define create_singlethread_workqueue(name)				\
> -	alloc_ordered_workqueue("%s", WQ_MEM_RECLAIM, name)
> +	alloc_ordered_workqueue("%s", __WQ_LEGACY | WQ_MEM_RECLAIM, name)
>
>   extern void destroy_workqueue(struct workqueue_struct *wq);
>
> diff --git a/kernel/workqueue.c b/kernel/workqueue.c
> index 61a0264..dc7faad 100644
> --- a/kernel/workqueue.c
> +++ b/kernel/workqueue.c
> @@ -2355,7 +2355,8 @@ static void check_flush_dependency(struct workqueue_struct *target_wq,
>   	WARN_ONCE(current->flags & PF_MEMALLOC,
>   		  "workqueue: PF_MEMALLOC task %d(%s) is flushing !WQ_MEM_RECLAIM %s:%pf",
>   		  current->pid, current->comm, target_wq->name, target_func);
> -	WARN_ONCE(worker && (worker->current_pwq->wq->flags & WQ_MEM_RECLAIM),
> +	WARN_ONCE(worker && ((worker->current_pwq->wq->flags &
> +			      (WQ_MEM_RECLAIM | __WQ_LEGACY)) == WQ_MEM_RECLAIM),
>   		  "workqueue: WQ_MEM_RECLAIM %s:%pf is flushing !WQ_MEM_RECLAIM %s:%pf",
>   		  worker->current_pwq->wq->name, worker->current_func,
>   		  target_wq->name, target_func);
>

-- 
The Qualcomm Innovation Center, Inc. is a member of the Code Aurora 
Forum, hosted by The Linux Foundation

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

* Re: [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue
  2015-12-03 19:26           ` [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue Tejun Heo
                               ` (2 preceding siblings ...)
       [not found]             ` <20151203192616.GJ27463-qYNAdHglDFBN0TnZuCh8vA@public.gmane.org>
@ 2016-03-10 15:12             ` Adrian Hunter
  2016-03-11 17:52               ` Tejun Heo
  3 siblings, 1 reply; 51+ messages in thread
From: Adrian Hunter @ 2016-03-10 15:12 UTC (permalink / raw)
  To: Tejun Heo, Peter Zijlstra
  Cc: Ulrich Obergfell, Ingo Molnar, Andrew Morton, linux-kernel, kernel-team

On 03/12/15 21:26, Tejun Heo wrote:
> Task or work item involved in memory reclaim trying to flush a
> non-WQ_MEM_RECLAIM workqueue or one of its work items can lead to
> deadlock.  Trigger WARN_ONCE() if such conditions are detected.
> 
> Signed-off-by: Tejun Heo <tj@kernel.org>
> Cc: Peter Zijlstra <peterz@infradead.org>
> ---
> Hello,
> 
> So, something like this.  Seems to work fine here.  If there's no
> objection, I'm gonna push it through wq/for-4.5.
> 
> Thanks.
> 
>  kernel/workqueue.c |   35 +++++++++++++++++++++++++++++++++++
>  1 file changed, 35 insertions(+)
> 
> --- a/kernel/workqueue.c
> +++ b/kernel/workqueue.c
> @@ -2330,6 +2330,37 @@ repeat:
>  	goto repeat;
>  }
>  
> +/**
> + * check_flush_dependency - check for flush dependency sanity
> + * @target_wq: workqueue being flushed
> + * @target_work: work item being flushed (NULL for workqueue flushes)
> + *
> + * %current is trying to flush the whole @target_wq or @target_work on it.
> + * If @target_wq doesn't have %WQ_MEM_RECLAIM, verify that %current is not
> + * reclaiming memory or running on a workqueue which doesn't have
> + * %WQ_MEM_RECLAIM as that can break forward-progress guarantee leading to
> + * a deadlock.
> + */
> +static void check_flush_dependency(struct workqueue_struct *target_wq,
> +				   struct work_struct *target_work)
> +{
> +	work_func_t target_func = target_work ? target_work->func : NULL;
> +	struct worker *worker;
> +
> +	if (target_wq->flags & WQ_MEM_RECLAIM)
> +		return;
> +
> +	worker = current_wq_worker();
> +
> +	WARN_ONCE(current->flags & PF_MEMALLOC,
> +		  "workqueue: PF_MEMALLOC task %d(%s) is flushing !WQ_MEM_RECLAIM %s:%pf",
> +		  current->pid, current->comm, target_wq->name, target_func);
> +	WARN_ONCE(worker && (worker->current_pwq->wq->flags & WQ_MEM_RECLAIM),
> +		  "workqueue: WQ_MEM_RECLAIM %s:%pf is flushing !WQ_MEM_RECLAIM %s:%pf",
> +		  worker->current_pwq->wq->name, worker->current_func,
> +		  target_wq->name, target_func);
> +}
> +
>  struct wq_barrier {
>  	struct work_struct	work;
>  	struct completion	done;
> @@ -2539,6 +2570,8 @@ void flush_workqueue(struct workqueue_st
>  		list_add_tail(&this_flusher.list, &wq->flusher_overflow);
>  	}
>  
> +	check_flush_dependency(wq, NULL);
> +
>  	mutex_unlock(&wq->mutex);
>  
>  	wait_for_completion(&this_flusher.done);
> @@ -2711,6 +2744,8 @@ static bool start_flush_work(struct work
>  		pwq = worker->current_pwq;
>  	}
>  
> +	check_flush_dependency(pwq->wq, work);
> +
>  	insert_wq_barrier(pwq, barr, work, worker);
>  	spin_unlock_irq(&pool->lock);
>  
> 

I am hitting the warnings when using cancel_delayed_work_sync().  I would
have thought that forward progress would still be guaranteed in that case.
Is it true that it is not?

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

* Re: [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue
  2016-03-10 15:12             ` [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue Adrian Hunter
@ 2016-03-11 17:52               ` Tejun Heo
  0 siblings, 0 replies; 51+ messages in thread
From: Tejun Heo @ 2016-03-11 17:52 UTC (permalink / raw)
  To: Adrian Hunter
  Cc: Peter Zijlstra, Ulrich Obergfell, Ingo Molnar, Andrew Morton,
	linux-kernel, kernel-team

Hello,

On Thu, Mar 10, 2016 at 05:12:54PM +0200, Adrian Hunter wrote:
> > @@ -2711,6 +2744,8 @@ static bool start_flush_work(struct work
> >  		pwq = worker->current_pwq;
> >  	}
> >  
> > +	check_flush_dependency(pwq->wq, work);
> > +
> >  	insert_wq_barrier(pwq, barr, work, worker);
> >  	spin_unlock_irq(&pool->lock);
> 
> I am hitting the warnings when using cancel_delayed_work_sync().  I would
> have thought that forward progress would still be guaranteed in that case.
> Is it true that it is not?

I'm not sure I understand what you're trying to say.  If you're trying
to do cancel_delayed_work_sync() from a memreclaim wq on a work item
which is executing on !memreclaim wq, that'd be an incorrect thing to
do as that can deadlock the memreclaim wq under memory pressure.

Thanks.

-- 
tejun

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

end of thread, other threads:[~2016-03-11 17:52 UTC | newest]

Thread overview: 51+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2015-12-03  0:28 [PATCH 1/2] watchdog: introduce touch_softlockup_watchdog_sched() Tejun Heo
2015-12-03  0:28 ` [PATCH 2/2] workqueue: implement lockup detector Tejun Heo
2015-12-03 14:49   ` Tejun Heo
2015-12-03 17:50   ` Don Zickus
2015-12-03 19:43     ` Tejun Heo
2015-12-03 20:12       ` Ulrich Obergfell
2015-12-03 20:54         ` Tejun Heo
2015-12-04  8:02           ` Ingo Molnar
2015-12-04 16:52             ` Don Zickus
2015-12-04 13:19           ` Ulrich Obergfell
2015-12-07 19:06   ` [PATCH v2 " Tejun Heo
2015-12-07 21:38     ` Don Zickus
2015-12-07 21:39       ` Tejun Heo
2015-12-08 16:00         ` Don Zickus
2015-12-08 16:31           ` Tejun Heo
2015-12-03  9:33 ` [PATCH 1/2] watchdog: introduce touch_softlockup_watchdog_sched() Peter Zijlstra
2015-12-03 10:00   ` Peter Zijlstra
2015-12-03 14:48     ` Tejun Heo
2015-12-03 15:04       ` Peter Zijlstra
2015-12-03 15:06         ` Tejun Heo
2015-12-03 19:26           ` [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue Tejun Heo
2015-12-03 20:43             ` Peter Zijlstra
2015-12-03 20:56               ` Tejun Heo
2015-12-03 21:09                 ` Peter Zijlstra
2015-12-03 22:04                   ` Tejun Heo
2015-12-04 12:51                     ` Peter Zijlstra
2015-12-07 15:58             ` Tejun Heo
     [not found]             ` <20151203192616.GJ27463-qYNAdHglDFBN0TnZuCh8vA@public.gmane.org>
2016-01-26 17:38               ` Thierry Reding
2016-01-26 17:38                 ` Thierry Reding
     [not found]                 ` <20160126173843.GA11115-AwZRO8vwLAwmlAP/+Wk3EA@public.gmane.org>
2016-01-28 10:12                   ` Peter Zijlstra
2016-01-28 10:12                     ` Peter Zijlstra
     [not found]                     ` <20160128101210.GC6357-ndre7Fmf5hadTX5a5knrm8zTDFooKrT+cvkQGrU6aU0@public.gmane.org>
2016-01-28 12:47                       ` Thierry Reding
2016-01-28 12:47                         ` Thierry Reding
2016-01-28 12:48                         ` Thierry Reding
2016-01-28 12:48                           ` Thierry Reding
2016-01-29 11:09                       ` Tejun Heo
2016-01-29 11:09                         ` Tejun Heo
2016-01-29 11:09                         ` Tejun Heo
     [not found]                         ` <20160129110941.GK32380-piEFEHQLUPpN0TnZuCh8vA@public.gmane.org>
2016-01-29 15:17                           ` Peter Zijlstra
2016-01-29 15:17                             ` Peter Zijlstra
2016-01-29 15:17                             ` Peter Zijlstra
2016-01-29 18:28                             ` Tejun Heo
2016-01-29 18:28                               ` Tejun Heo
2016-01-29 10:59                   ` [PATCH wq/for-4.5-fixes] workqueue: skip flush dependency checks for legacy workqueues Tejun Heo
2016-01-29 10:59                     ` Tejun Heo
2016-01-29 15:07                     ` Thierry Reding
2016-01-29 18:32                     ` Tejun Heo
     [not found]                     ` <20160129105946.GJ32380-piEFEHQLUPpN0TnZuCh8vA@public.gmane.org>
2016-02-02  6:54                       ` Archit Taneja
2016-02-02  6:54                         ` Archit Taneja
2016-03-10 15:12             ` [PATCH] workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue Adrian Hunter
2016-03-11 17:52               ` 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.