linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH 0/4] printk: Softlockup avoidance
@ 2015-08-19 15:38 Jan Kara
  2015-08-19 15:38 ` [PATCH 1/4] printk: Hand over printing to console if printing too long Jan Kara
                   ` (5 more replies)
  0 siblings, 6 replies; 15+ messages in thread
From: Jan Kara @ 2015-08-19 15:38 UTC (permalink / raw)
  To: Andrew Morton; +Cc: LKML, pmladek, rostedt, Gavin Hu, KY Srinivasan, Jan Kara

From: Jan Kara <jack@suse.cz>

Hello,

since lately there were several attempts at dealing with softlockups due
to heavy printk traffic [1] [2] and I've been also privately pinged by
couple of people about the state of the patch set, I've decided to respin
the patch set.

To remind the original problem:

Currently, console_unlock() prints messages from kernel printk buffer to
console while the buffer is non-empty. When serial console is attached,
printing is slow and thus other CPUs in the system have plenty of time
to append new messages to the buffer while one CPU is printing. Thus the
CPU can spend unbounded amount of time doing printing in console_unlock().
This is especially serious when printk() gets called under some critical
spinlock or with interrupts disabled.
    
In practice users have observed a CPU can spend tens of seconds printing
in console_unlock() (usually during boot when hundreds of SCSI devices
are discovered) resulting in RCU stalls (CPU doing printing doesn't
reach quiescent state for a long time), softlockup reports (IPIs for the
printing CPU don't get served and thus other CPUs are spinning waiting
for the printing CPU to process IPIs), and eventually a machine death
(as messages from stalls and lockups append to printk buffer faster than
we are able to print). So these machines are unable to boot with serial
console attached. Also during artificial stress testing SATA disk
disappears from the system because its interrupts aren't served for too
long.

This series addresses the problem in the following way: If CPU has printed
more that printk_offload (defaults to 1000) characters, it wakes up one
of dedicated printk kthreads (we don't use workqueue because that has
deadlock potential if printk was called from workqueue code). Once we find
out kthread is spinning on a lock, we stop printing, drop console_sem, and
let kthread continue printing. Since there are two printing kthreads, they
will pass printing between them and thus no CPU gets hogged by printing.

Changes since the last posting [3]:
* I have replaced the state machine to pass printing and spinning on
  console_sem with a simple spinlock which makes the code
  somewhat easier to read and verify.
* Some of the patches were merged so I dropped them.

								Honza

[1] https://lkml.org/lkml/2015/7/8/215
[2] http://marc.info/?l=linux-kernel&m=143929238407816&w=2
[3] https://lkml.org/lkml/2014/3/17/68

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

* [PATCH 1/4] printk: Hand over printing to console if printing too long
  2015-08-19 15:38 [PATCH 0/4] printk: Softlockup avoidance Jan Kara
@ 2015-08-19 15:38 ` Jan Kara
  2015-09-18 22:14   ` Andrew Morton
  2015-08-19 15:38 ` [PATCH 2/4] printk: Start printing handover kthreads on demand Jan Kara
                   ` (4 subsequent siblings)
  5 siblings, 1 reply; 15+ messages in thread
From: Jan Kara @ 2015-08-19 15:38 UTC (permalink / raw)
  To: Andrew Morton; +Cc: LKML, pmladek, rostedt, Gavin Hu, KY Srinivasan, Jan Kara

From: Jan Kara <jack@suse.cz>

Currently, console_unlock() prints messages from kernel printk buffer to
console while the buffer is non-empty. When serial console is attached,
printing is slow and thus other CPUs in the system have plenty of time
to append new messages to the buffer while one CPU is printing. Thus the
CPU can spend unbounded amount of time doing printing in console_unlock().
This is especially serious problem if the printk() calling
console_unlock() was called with interrupts disabled.

In practice users have observed a CPU can spend tens of seconds printing
in console_unlock() (usually during boot when hundreds of SCSI devices
are discovered) resulting in RCU stalls (CPU doing printing doesn't
reach quiescent state for a long time), softlockup reports (IPIs for the
printing CPU don't get served and thus other CPUs are spinning waiting
for the printing CPU to process IPIs), and eventually a machine death
(as messages from stalls and lockups append to printk buffer faster than
we are able to print). So these machines are unable to boot with serial
console attached. Also during artificial stress testing SATA disk
disappears from the system because its interrupts aren't served for too
long.

This patch implements a mechanism where after printing specified number
of characters (tunable as a kernel parameter printk.offload_chars), CPU
doing printing asks for help by waking up one of dedicated kthreads.  As
soon as the printing CPU notices kthread got scheduled and is spinning
on print_lock dedicated for that purpose, it drops console_sem,
print_lock, and exits console_unlock(). Kthread then takes over printing
instead. This way no CPU should spend printing too long even if there
is heavy printk traffic.

Signed-off-by: Jan Kara <jack@suse.cz>
---
 Documentation/kernel-parameters.txt |  15 ++++
 kernel/printk/printk.c              | 163 ++++++++++++++++++++++++++++++++----
 2 files changed, 162 insertions(+), 16 deletions(-)

diff --git a/Documentation/kernel-parameters.txt b/Documentation/kernel-parameters.txt
index 1d6f0459cd7b..4e0d2d8c0f97 100644
--- a/Documentation/kernel-parameters.txt
+++ b/Documentation/kernel-parameters.txt
@@ -2943,6 +2943,21 @@ bytes respectively. Such letter suffixes can also be entirely omitted.
 			Format: <bool>  (1/Y/y=enable, 0/N/n=disable)
 			default: disabled
 
+	printk.offload_chars=
+			Printing to console can be relatively slow especially
+			in case of serial console. When there is intensive
+			printing happening from several cpus (as is the case
+			during boot), a cpu can be spending significant time
+			(seconds or more) doing printing. To avoid softlockups,
+			lost interrupts, and similar problems other cpus
+			will take over printing after the currently printing
+			cpu has printed 'printk.offload_chars' characters.
+			Higher value means possibly longer interrupt and other
+			latencies but lower overhead of printing due to handing
+			over of printing.
+			Format: <number> (0 = disabled)
+			default: 1000
+
 	printk.time=	Show timing data prefixed to each printk message line
 			Format: <bool>  (1/Y/y=enable, 0/N/n=disable)
 
diff --git a/kernel/printk/printk.c b/kernel/printk/printk.c
index cf8c24203368..f65f8ab50553 100644
--- a/kernel/printk/printk.c
+++ b/kernel/printk/printk.c
@@ -46,6 +46,7 @@
 #include <linux/utsname.h>
 #include <linux/ctype.h>
 #include <linux/uio.h>
+#include <linux/kthread.h>
 
 #include <asm/uaccess.h>
 
@@ -78,6 +79,29 @@ static DEFINE_SEMAPHORE(console_sem);
 struct console *console_drivers;
 EXPORT_SYMBOL_GPL(console_drivers);
 
+/*
+ * This spinlock is taken when printing to console. It is used only so that
+ * we can spin on it when some other thread wants to take over printing to
+ * console.
+ */
+static DEFINE_SPINLOCK(print_lock);
+
+/*
+ * Number of printing threads spinning on print_lock. Can go away once
+ * spin_is_contended() is reliable.
+ */
+static atomic_t printing_tasks_spinning = ATOMIC_INIT(0);
+
+/*
+ * Number of kernel threads for offloading printing. We need at least two so
+ * that they can hand over printing from one to another one and thus switch
+ * CPUs.
+ */
+#define PRINTING_TASKS 2
+
+/* Wait queue printing kthreads sleep on when idle */
+static DECLARE_WAIT_QUEUE_HEAD(print_queue);
+
 #ifdef CONFIG_LOCKDEP
 static struct lockdep_map console_lock_dep_map = {
 	.name = "console_lock"
@@ -279,6 +303,18 @@ static u32 clear_idx;
 static char __log_buf[__LOG_BUF_LEN] __aligned(LOG_ALIGN);
 static char *log_buf = __log_buf;
 static u32 log_buf_len = __LOG_BUF_LEN;
+/*
+ * How many characters can we print in one call of printk before asking
+ * other cpus to continue printing. 0 means infinity. Tunable via
+ * printk.offload_chars kernel parameter. Our default 1000 means about
+ * 0.1s maximum latency due to printing.
+ */
+static unsigned int __read_mostly printk_offload_chars = 1000;
+
+module_param_named(offload_chars, printk_offload_chars, uint,
+		   S_IRUGO | S_IWUSR);
+MODULE_PARM_DESC(offload_chars, "offload printing to console to a different"
+	" cpu after this number of characters");
 
 /* Return log buffer address */
 char *log_buf_addr_get(void)
@@ -2208,15 +2244,41 @@ out:
 	raw_spin_unlock_irqrestore(&logbuf_lock, flags);
 }
 
+/*
+ * Returns true iff there is other cpu waiting to take over printing. This
+ * function also takes are of setting PRINTK_HANDOVER_B if we want to hand over
+ * printing to some other cpu.
+ */
+static bool cpu_stop_printing(int printed_chars)
+{
+	/* Oops? Print everything now to maximize chances user will see it */
+	if (oops_in_progress)
+		return false;
+	if (!printk_offload_chars || printed_chars < printk_offload_chars)
+		return false;
+	/*
+	 * Make sure we load fresh value of printing_tasks_spinning. Matches
+	 * the barrier in printing_task()
+	 */
+	smp_rmb();
+	if (atomic_read(&printing_tasks_spinning))
+		return true;
+	wake_up(&print_queue);
+
+	return false;
+}
+
 /**
  * console_unlock - unlock the console system
  *
  * Releases the console_lock which the caller holds on the console system
  * and the console driver list.
  *
- * While the console_lock was held, console output may have been buffered
- * by printk().  If this is the case, console_unlock(); emits
- * the output prior to releasing the lock.
+ * While the console_lock was held, console output may have been buffered by
+ * printk(). If this is the case, console_unlock() emits the output prior to
+ * releasing the lock. However we need not write all the data in the buffer if
+ * we would hog the CPU for too long. In such case we try to hand over printing
+ * to a different cpu.
  *
  * If there is output waiting, we wake /dev/kmsg and syslog() users.
  *
@@ -2230,6 +2292,8 @@ void console_unlock(void)
 	unsigned long flags;
 	bool wake_klogd = false;
 	bool retry;
+	bool hand_over = false;
+	int printed_chars = 0;
 
 	if (console_suspended) {
 		up_console_sem();
@@ -2241,12 +2305,18 @@ void console_unlock(void)
 	/* flush buffered message fragment immediately to console */
 	console_cont_flush(text, sizeof(text));
 again:
+	spin_lock(&print_lock);
 	for (;;) {
 		struct printk_log *msg;
 		size_t ext_len = 0;
 		size_t len;
 		int level;
 
+		if (cpu_stop_printing(printed_chars)) {
+			hand_over = true;
+			break;
+		}
+
 		raw_spin_lock_irqsave(&logbuf_lock, flags);
 		if (seen_seq != log_next_seq) {
 			wake_klogd = true;
@@ -2265,8 +2335,10 @@ again:
 			len = 0;
 		}
 skip:
-		if (console_seq == log_next_seq)
+		if (console_seq == log_next_seq) {
+			raw_spin_unlock(&logbuf_lock);
 			break;
+		}
 
 		msg = log_from_idx(console_idx);
 		if (msg->flags & LOG_NOCONS) {
@@ -2306,30 +2378,41 @@ skip:
 		stop_critical_timings();	/* don't trace print latency */
 		call_console_drivers(level, ext_text, ext_len, text, len);
 		start_critical_timings();
+		printed_chars += len;
 		local_irq_restore(flags);
 	}
-	console_locked = 0;
 
 	/* Release the exclusive_console once it is used */
 	if (unlikely(exclusive_console))
 		exclusive_console = NULL;
 
-	raw_spin_unlock(&logbuf_lock);
-
+	console_locked = 0;
 	up_console_sem();
+	/*
+	 * Release print_lock after console_sem so that printing_task()
+	 * succeeds in getting console_sem (unless someone else takes it and
+	 * then he'll be responsible for printing).
+         */
+	spin_unlock(&print_lock);
 
 	/*
-	 * Someone could have filled up the buffer again, so re-check if there's
-	 * something to flush. In case we cannot trylock the console_sem again,
-	 * there's a new owner and the console_unlock() from them will do the
-	 * flush, no worries.
+	 * Subtlety: We have interrupts disabled iff hand_over == false (to
+	 * save one cli/sti pair in the fast path.
 	 */
-	raw_spin_lock(&logbuf_lock);
-	retry = console_seq != log_next_seq;
-	raw_spin_unlock_irqrestore(&logbuf_lock, flags);
+	if (!hand_over) {
+		/*
+		 * Someone could have filled up the buffer again, so re-check
+		 * if there's something to flush. In case we cannot trylock the
+		 * console_sem again, there's a new owner and the
+		 * console_unlock() from them will do the flush, no worries.
+		 */
+		raw_spin_lock(&logbuf_lock);
+		retry = console_seq != log_next_seq;
+		raw_spin_unlock_irqrestore(&logbuf_lock, flags);
 
-	if (retry && console_trylock())
-		goto again;
+		if (retry && console_trylock())
+			goto again;
+	}
 
 	if (wake_klogd)
 		wake_up_klogd();
@@ -2654,9 +2737,44 @@ int unregister_console(struct console *console)
 }
 EXPORT_SYMBOL(unregister_console);
 
+/* Kthread which takes over printing from a CPU which asks for help */
+static int printing_task(void *arg)
+{
+	DEFINE_WAIT(wait);
+
+	while (1) {
+		prepare_to_wait_exclusive(&print_queue, &wait,
+					  TASK_INTERRUPTIBLE);
+		schedule();
+		finish_wait(&print_queue, &wait);
+		preempt_disable();
+		atomic_inc(&printing_tasks_spinning);
+		/*
+		 * Store printing_tasks_spinning value before we spin. Matches
+		 * the barrier in cpu_stop_printing().
+		 */
+		smp_mb__after_atomic();
+		/*
+		 * Wait for currently printing thread to complete. We spin on
+		 * print_lock instead of waiting on console_sem since we don't
+		 * want to sleep once we got scheduled to make sure we take
+		 * over printing without depending on the scheduler.
+		 */
+		spin_lock(&print_lock);
+		atomic_dec(&printing_tasks_spinning);
+		spin_unlock(&print_lock);
+		if (console_trylock())
+			console_unlock();
+		preempt_enable();
+	}
+	return 0;
+}
+
 static int __init printk_late_init(void)
 {
 	struct console *con;
+	int i;
+	struct task_struct *task;
 
 	for_each_console(con) {
 		if (!keep_bootcon && con->flags & CON_BOOT) {
@@ -2664,6 +2782,19 @@ static int __init printk_late_init(void)
 		}
 	}
 	hotcpu_notifier(console_cpu_notify, 0);
+
+	/* Does any handover of printing have any sence? */
+	if (num_possible_cpus() <= 1)
+		return 0;
+
+	for (i = 0; i < PRINTING_TASKS; i++) {
+		task = kthread_run(printing_task, NULL, "print/%d", i);
+		if (IS_ERR(task)) {
+			pr_err("printk: Cannot create printing thread: %ld\n",
+			       PTR_ERR(task));
+		}
+	}
+
 	return 0;
 }
 late_initcall(printk_late_init);
-- 
2.1.4


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

* [PATCH 2/4] printk: Start printing handover kthreads on demand
  2015-08-19 15:38 [PATCH 0/4] printk: Softlockup avoidance Jan Kara
  2015-08-19 15:38 ` [PATCH 1/4] printk: Hand over printing to console if printing too long Jan Kara
@ 2015-08-19 15:38 ` Jan Kara
  2015-08-19 15:38 ` [PATCH 3/4] kernel: Avoid softlockups in stop_machine() during heavy printing Jan Kara
                   ` (3 subsequent siblings)
  5 siblings, 0 replies; 15+ messages in thread
From: Jan Kara @ 2015-08-19 15:38 UTC (permalink / raw)
  To: Andrew Morton; +Cc: LKML, pmladek, rostedt, Gavin Hu, KY Srinivasan, Jan Kara

From: Jan Kara <jack@suse.cz>

Start kthreads for handing over printing only when printk.offload_chars
is set to value > 0 (i.e., when print offloading gets enabled).

Signed-off-by: Jan Kara <jack@suse.cz>
---
 kernel/printk/printk.c | 65 +++++++++++++++++++++++++++++++++++++++-----------
 1 file changed, 51 insertions(+), 14 deletions(-)

diff --git a/kernel/printk/printk.c b/kernel/printk/printk.c
index f65f8ab50553..a10845f0e5b0 100644
--- a/kernel/printk/printk.c
+++ b/kernel/printk/printk.c
@@ -98,6 +98,10 @@ static atomic_t printing_tasks_spinning = ATOMIC_INIT(0);
  * CPUs.
  */
 #define PRINTING_TASKS 2
+/* Pointers to printing kthreads */
+static struct task_struct *printing_kthread[PRINTING_TASKS];
+/* Serialization of changes to printk_offload_chars and kthread creation */
+static DEFINE_MUTEX(printk_kthread_mutex);
 
 /* Wait queue printing kthreads sleep on when idle */
 static DECLARE_WAIT_QUEUE_HEAD(print_queue);
@@ -303,6 +307,13 @@ static u32 clear_idx;
 static char __log_buf[__LOG_BUF_LEN] __aligned(LOG_ALIGN);
 static char *log_buf = __log_buf;
 static u32 log_buf_len = __LOG_BUF_LEN;
+
+static int offload_chars_set(const char *val, const struct kernel_param *kp);
+static struct kernel_param_ops offload_chars_ops = {
+	.set = offload_chars_set,
+	.get = param_get_uint,
+};
+
 /*
  * How many characters can we print in one call of printk before asking
  * other cpus to continue printing. 0 means infinity. Tunable via
@@ -311,7 +322,7 @@ static u32 log_buf_len = __LOG_BUF_LEN;
  */
 static unsigned int __read_mostly printk_offload_chars = 1000;
 
-module_param_named(offload_chars, printk_offload_chars, uint,
+module_param_cb(offload_chars, &offload_chars_ops, &printk_offload_chars,
 		   S_IRUGO | S_IWUSR);
 MODULE_PARM_DESC(offload_chars, "offload printing to console to a different"
 	" cpu after this number of characters");
@@ -2770,30 +2781,56 @@ static int printing_task(void *arg)
 	return 0;
 }
 
-static int __init printk_late_init(void)
+static void printk_start_offload_kthreads(void)
 {
-	struct console *con;
 	int i;
 	struct task_struct *task;
 
-	for_each_console(con) {
-		if (!keep_bootcon && con->flags & CON_BOOT) {
-			unregister_console(con);
-		}
-	}
-	hotcpu_notifier(console_cpu_notify, 0);
-
-	/* Does any handover of printing have any sence? */
-	if (num_possible_cpus() <= 1)
-		return 0;
-
+	/* Does handover of printing make any sense? */
+	if (printk_offload_chars == 0 || num_possible_cpus() <= 1)
+		return;
 	for (i = 0; i < PRINTING_TASKS; i++) {
+		if (printing_kthread[i])
+			continue;
 		task = kthread_run(printing_task, NULL, "print/%d", i);
 		if (IS_ERR(task)) {
 			pr_err("printk: Cannot create printing thread: %ld\n",
 			       PTR_ERR(task));
 		}
+		printing_kthread[i] = task;
+	}
+}
+
+static int offload_chars_set(const char *val, const struct kernel_param *kp)
+{
+	int ret;
+
+	/* Protect against parallel change of printk_offload_chars */
+	mutex_lock(&printk_kthread_mutex);
+	ret = param_set_uint(val, kp);
+	if (ret) {
+		mutex_unlock(&printk_kthread_mutex);
+		return ret;
+	}
+	printk_start_offload_kthreads();
+	mutex_unlock(&printk_kthread_mutex);
+	return 0;
+}
+
+static int __init printk_late_init(void)
+{
+	struct console *con;
+
+	for_each_console(con) {
+		if (!keep_bootcon && con->flags & CON_BOOT) {
+			unregister_console(con);
+		}
 	}
+	hotcpu_notifier(console_cpu_notify, 0);
+
+	mutex_lock(&printk_kthread_mutex);
+	printk_start_offload_kthreads();
+	mutex_unlock(&printk_kthread_mutex);
 
 	return 0;
 }
-- 
2.1.4


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

* [PATCH 3/4] kernel: Avoid softlockups in stop_machine() during heavy printing
  2015-08-19 15:38 [PATCH 0/4] printk: Softlockup avoidance Jan Kara
  2015-08-19 15:38 ` [PATCH 1/4] printk: Hand over printing to console if printing too long Jan Kara
  2015-08-19 15:38 ` [PATCH 2/4] printk: Start printing handover kthreads on demand Jan Kara
@ 2015-08-19 15:38 ` Jan Kara
  2015-09-18 22:15   ` Andrew Morton
  2015-08-19 15:38 ` [PATCH 4/4] printk: Add config option for disabling printk offloading Jan Kara
                   ` (2 subsequent siblings)
  5 siblings, 1 reply; 15+ messages in thread
From: Jan Kara @ 2015-08-19 15:38 UTC (permalink / raw)
  To: Andrew Morton; +Cc: LKML, pmladek, rostedt, Gavin Hu, KY Srinivasan, Jan Kara

From: Jan Kara <jack@suse.cz>

When there are lots of messages accumulated in printk buffer, printing
them (especially over serial console) can take a long time (tens of
seconds). stop_machine() will effectively make all cpus spin in
multi_cpu_stop() waiting for the CPU doing printing to print all the
messages which triggers NMI softlockup watchdog and RCU stall detector
which add even more to the messages to print. Since machine doesn't do
anything (except serving interrupts) during this time, also network
connections are dropped and other disturbances may happen.

Paper over the problem by waiting for printk buffer to be empty before
starting to stop CPUs. In theory a burst of new messages can be appended
to the printk buffer before CPUs enter multi_cpu_stop() so this isn't a 100%
solution but it works OK in practice and I'm not aware of a reasonably
simple better solution.

Signed-off-by: Jan Kara <jack@suse.cz>
---
 include/linux/console.h |  1 +
 kernel/printk/printk.c  | 22 ++++++++++++++++++++++
 kernel/stop_machine.c   |  9 +++++++++
 3 files changed, 32 insertions(+)

diff --git a/include/linux/console.h b/include/linux/console.h
index bd194343c346..465b51093cad 100644
--- a/include/linux/console.h
+++ b/include/linux/console.h
@@ -150,6 +150,7 @@ extern int console_trylock(void);
 extern void console_unlock(void);
 extern void console_conditional_schedule(void);
 extern void console_unblank(void);
+extern void console_flush(void);
 extern struct tty_driver *console_device(int *);
 extern void console_stop(struct console *);
 extern void console_start(struct console *);
diff --git a/kernel/printk/printk.c b/kernel/printk/printk.c
index a10845f0e5b0..c443f1aa7e84 100644
--- a/kernel/printk/printk.c
+++ b/kernel/printk/printk.c
@@ -2489,6 +2489,28 @@ struct tty_driver *console_device(int *index)
 }
 
 /*
+ * Wait until all messages accumulated in the printk buffer are printed to
+ * console. Note that as soon as this function returns, new messages may be
+ * added to the printk buffer by other CPUs.
+ */
+void console_flush(void)
+{
+	bool retry;
+	unsigned long flags;
+
+	while (1) {
+		raw_spin_lock_irqsave(&logbuf_lock, flags);
+		retry = console_seq != log_next_seq;
+		raw_spin_unlock_irqrestore(&logbuf_lock, flags);
+		if (!retry || console_suspended)
+			break;
+		/* Cycle console_sem to wait for outstanding printing */
+		console_lock();
+		console_unlock();
+	}
+}
+
+/*
  * Prevent further output on the passed console device so that (for example)
  * serial drivers can disable console output before suspending a port, and can
  * re-enable output afterwards.
diff --git a/kernel/stop_machine.c b/kernel/stop_machine.c
index fd643d8c4b42..016d34621d2e 100644
--- a/kernel/stop_machine.c
+++ b/kernel/stop_machine.c
@@ -21,6 +21,7 @@
 #include <linux/smpboot.h>
 #include <linux/atomic.h>
 #include <linux/lglock.h>
+#include <linux/console.h>
 
 /*
  * Structure to determine completion condition and record errors.  May
@@ -543,6 +544,14 @@ int __stop_machine(int (*fn)(void *), void *data, const struct cpumask *cpus)
 		return ret;
 	}
 
+	/*
+	 * If there are lots of outstanding messages, printing them can take a
+	 * long time and all cpus would be spinning waiting for the printing to
+	 * finish thus triggering NMI watchdog, RCU lockups etc. Wait for the
+	 * printing here to avoid these.
+	 */
+	console_flush();
+
 	/* Set the initial state and stop all online cpus. */
 	set_state(&msdata, MULTI_STOP_PREPARE);
 	return stop_cpus(cpu_online_mask, multi_cpu_stop, &msdata);
-- 
2.1.4


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

* [PATCH 4/4] printk: Add config option for disabling printk offloading
  2015-08-19 15:38 [PATCH 0/4] printk: Softlockup avoidance Jan Kara
                   ` (2 preceding siblings ...)
  2015-08-19 15:38 ` [PATCH 3/4] kernel: Avoid softlockups in stop_machine() during heavy printing Jan Kara
@ 2015-08-19 15:38 ` Jan Kara
  2015-09-18 22:15   ` Andrew Morton
  2015-08-20  2:37 ` [PATCH 0/4] printk: Softlockup avoidance KY Srinivasan
  2015-09-18 22:14 ` Andrew Morton
  5 siblings, 1 reply; 15+ messages in thread
From: Jan Kara @ 2015-08-19 15:38 UTC (permalink / raw)
  To: Andrew Morton; +Cc: LKML, pmladek, rostedt, Gavin Hu, KY Srinivasan, Jan Kara

From: Jan Kara <jack@suse.cz>

Necessity for offloading of printing was observed only for large
systems. So add a config option (disabled by default) which removes most
of the overhead added by this functionality.

Signed-off-by: Jan Kara <jack@suse.cz>
---
 Documentation/kernel-parameters.txt | 13 +++++++------
 init/Kconfig                        | 14 ++++++++++++++
 kernel/printk/printk.c              | 21 +++++++++++++++++++++
 3 files changed, 42 insertions(+), 6 deletions(-)

diff --git a/Documentation/kernel-parameters.txt b/Documentation/kernel-parameters.txt
index 4e0d2d8c0f97..0ed7f0f1d87a 100644
--- a/Documentation/kernel-parameters.txt
+++ b/Documentation/kernel-parameters.txt
@@ -2943,18 +2943,19 @@ bytes respectively. Such letter suffixes can also be entirely omitted.
 			Format: <bool>  (1/Y/y=enable, 0/N/n=disable)
 			default: disabled
 
-	printk.offload_chars=
+	printk.offload_chars=	[KNL]
 			Printing to console can be relatively slow especially
 			in case of serial console. When there is intensive
 			printing happening from several cpus (as is the case
 			during boot), a cpu can be spending significant time
 			(seconds or more) doing printing. To avoid softlockups,
 			lost interrupts, and similar problems other cpus
-			will take over printing after the currently printing
-			cpu has printed 'printk.offload_chars' characters.
-			Higher value means possibly longer interrupt and other
-			latencies but lower overhead of printing due to handing
-			over of printing.
+			will take over printing (if CONFIG_PRINTK_OFFLOAD=y)
+			after the currently printing cpu has printed
+			'printk.offload_chars' characters. Higher value means
+			possibly longer interrupt and other latencies but
+			lower overhead of printing due to handing over of
+			printing.
 			Format: <number> (0 = disabled)
 			default: 1000
 
diff --git a/init/Kconfig b/init/Kconfig
index af09b4fb43d2..71b296483d0d 100644
--- a/init/Kconfig
+++ b/init/Kconfig
@@ -1439,6 +1439,20 @@ config PRINTK
 	  very difficult to diagnose system problems, saying N here is
 	  strongly discouraged.
 
+config PRINTK_OFFLOAD
+	default y
+	bool "Enable support for offloading printing to different CPU"
+	depends on PRINTK
+	help
+	  Printing to console can be relatively slow especially in case of
+	  serial console. On large machines when there is intensive printing
+	  happening from several cpus (as is the case during boot), a cpu can
+	  be spending significant time (seconds or more) doing printing. To
+	  avoid softlockups, lost interrupts, and similar problems other cpus
+	  will take over printing after the currently printing cpu has printed
+	  certain number of characters (tunable via 'printk.offload_chars'
+	  kernel parameter).
+
 config BUG
 	bool "BUG() support" if EXPERT
 	default y
diff --git a/kernel/printk/printk.c b/kernel/printk/printk.c
index c443f1aa7e84..63d78ab19278 100644
--- a/kernel/printk/printk.c
+++ b/kernel/printk/printk.c
@@ -79,6 +79,7 @@ static DEFINE_SEMAPHORE(console_sem);
 struct console *console_drivers;
 EXPORT_SYMBOL_GPL(console_drivers);
 
+#ifdef CONFIG_PRINTK_OFFLOAD
 /*
  * This spinlock is taken when printing to console. It is used only so that
  * we can spin on it when some other thread wants to take over printing to
@@ -105,6 +106,7 @@ static DEFINE_MUTEX(printk_kthread_mutex);
 
 /* Wait queue printing kthreads sleep on when idle */
 static DECLARE_WAIT_QUEUE_HEAD(print_queue);
+#endif	/* CONFIG_PRINTK_OFFLOAD */
 
 #ifdef CONFIG_LOCKDEP
 static struct lockdep_map console_lock_dep_map = {
@@ -308,6 +310,7 @@ static char __log_buf[__LOG_BUF_LEN] __aligned(LOG_ALIGN);
 static char *log_buf = __log_buf;
 static u32 log_buf_len = __LOG_BUF_LEN;
 
+#ifdef CONFIG_PRINTK_OFFLOAD
 static int offload_chars_set(const char *val, const struct kernel_param *kp);
 static struct kernel_param_ops offload_chars_ops = {
 	.set = offload_chars_set,
@@ -326,6 +329,7 @@ module_param_cb(offload_chars, &offload_chars_ops, &printk_offload_chars,
 		   S_IRUGO | S_IWUSR);
 MODULE_PARM_DESC(offload_chars, "offload printing to console to a different"
 	" cpu after this number of characters");
+#endif
 
 /* Return log buffer address */
 char *log_buf_addr_get(void)
@@ -2255,6 +2259,7 @@ out:
 	raw_spin_unlock_irqrestore(&logbuf_lock, flags);
 }
 
+#ifdef CONFIG_PRINTK_OFFLOAD
 /*
  * Returns true iff there is other cpu waiting to take over printing. This
  * function also takes are of setting PRINTK_HANDOVER_B if we want to hand over
@@ -2278,6 +2283,14 @@ static bool cpu_stop_printing(int printed_chars)
 
 	return false;
 }
+#else
+
+static bool cpu_stop_printing(int printed_chars, bool *woken)
+{
+	return false;
+}
+
+#endif
 
 /**
  * console_unlock - unlock the console system
@@ -2316,7 +2329,9 @@ void console_unlock(void)
 	/* flush buffered message fragment immediately to console */
 	console_cont_flush(text, sizeof(text));
 again:
+#ifdef CONFIG_PRINTK_OFFLOAD
 	spin_lock(&print_lock);
+#endif
 	for (;;) {
 		struct printk_log *msg;
 		size_t ext_len = 0;
@@ -2399,12 +2414,14 @@ skip:
 
 	console_locked = 0;
 	up_console_sem();
+#ifdef CONFIG_PRINTK_OFFLOAD
 	/*
 	 * Release print_lock after console_sem so that printing_task()
 	 * succeeds in getting console_sem (unless someone else takes it and
 	 * then he'll be responsible for printing).
          */
 	spin_unlock(&print_lock);
+#endif
 
 	/*
 	 * Subtlety: We have interrupts disabled iff hand_over == false (to
@@ -2770,6 +2787,7 @@ int unregister_console(struct console *console)
 }
 EXPORT_SYMBOL(unregister_console);
 
+#ifdef CONFIG_PRINTK_OFFLOAD
 /* Kthread which takes over printing from a CPU which asks for help */
 static int printing_task(void *arg)
 {
@@ -2838,6 +2856,7 @@ static int offload_chars_set(const char *val, const struct kernel_param *kp)
 	mutex_unlock(&printk_kthread_mutex);
 	return 0;
 }
+#endif	/* CONFIG_PRINTK_OFFLOAD */
 
 static int __init printk_late_init(void)
 {
@@ -2850,9 +2869,11 @@ static int __init printk_late_init(void)
 	}
 	hotcpu_notifier(console_cpu_notify, 0);
 
+#ifdef CONFIG_PRINTK_OFFLOAD
 	mutex_lock(&printk_kthread_mutex);
 	printk_start_offload_kthreads();
 	mutex_unlock(&printk_kthread_mutex);
+#endif
 
 	return 0;
 }
-- 
2.1.4


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

* RE: [PATCH 0/4] printk: Softlockup avoidance
  2015-08-19 15:38 [PATCH 0/4] printk: Softlockup avoidance Jan Kara
                   ` (3 preceding siblings ...)
  2015-08-19 15:38 ` [PATCH 4/4] printk: Add config option for disabling printk offloading Jan Kara
@ 2015-08-20  2:37 ` KY Srinivasan
  2015-09-18 22:14 ` Andrew Morton
  5 siblings, 0 replies; 15+ messages in thread
From: KY Srinivasan @ 2015-08-20  2:37 UTC (permalink / raw)
  To: Jan Kara, Andrew Morton
  Cc: LKML, pmladek, rostedt, Gavin Hu, Jan Kara, Vitaly Kuznetsov



> -----Original Message-----
> From: Jan Kara [mailto:jack@suse.com]
> Sent: Wednesday, August 19, 2015 8:38 AM
> To: Andrew Morton <akpm@linux-foundation.org>
> Cc: LKML <linux-kernel@vger.kernel.org>; pmladek@suse.com;
> rostedt@goodmis.org; Gavin Hu <gavin.hu.2010@gmail.com>; KY Srinivasan
> <kys@microsoft.com>; Jan Kara <jack@suse.cz>
> Subject: [PATCH 0/4] printk: Softlockup avoidance
> 
> From: Jan Kara <jack@suse.cz>
> 
> Hello,
> 
> since lately there were several attempts at dealing with softlockups due
> to heavy printk traffic [1] [2] and I've been also privately pinged by
> couple of people about the state of the patch set, I've decided to respin
> the patch set.
> 
> To remind the original problem:
> 
> Currently, console_unlock() prints messages from kernel printk buffer to
> console while the buffer is non-empty. When serial console is attached,
> printing is slow and thus other CPUs in the system have plenty of time
> to append new messages to the buffer while one CPU is printing. Thus the
> CPU can spend unbounded amount of time doing printing in
> console_unlock().
> This is especially serious when printk() gets called under some critical
> spinlock or with interrupts disabled.
> 
> In practice users have observed a CPU can spend tens of seconds printing
> in console_unlock() (usually during boot when hundreds of SCSI devices
> are discovered) resulting in RCU stalls (CPU doing printing doesn't
> reach quiescent state for a long time), softlockup reports (IPIs for the
> printing CPU don't get served and thus other CPUs are spinning waiting
> for the printing CPU to process IPIs), and eventually a machine death
> (as messages from stalls and lockups append to printk buffer faster than
> we are able to print). So these machines are unable to boot with serial
> console attached. Also during artificial stress testing SATA disk
> disappears from the system because its interrupts aren't served for too
> long.
> 
> This series addresses the problem in the following way: If CPU has printed
> more that printk_offload (defaults to 1000) characters, it wakes up one
> of dedicated printk kthreads (we don't use workqueue because that has
> deadlock potential if printk was called from workqueue code). Once we find
> out kthread is spinning on a lock, we stop printing, drop console_sem, and
> let kthread continue printing. Since there are two printing kthreads, they
> will pass printing between them and thus no CPU gets hogged by printing.
> 
> Changes since the last posting [3]:
> * I have replaced the state machine to pass printing and spinning on
>   console_sem with a simple spinlock which makes the code
>   somewhat easier to read and verify.
> * Some of the patches were merged so I dropped them.
> 
> 								Honza

Thanks Jan. I would like to add that the problem described here is further aggravated
in virtual machines and the solution proposed here effectively solves the problem.

Regards,

K. Y
> 
> [1]
> https://na01.safelinks.protection.outlook.com/?url=https%3a%2f%2flkml.or
> g%2flkml%2f2015%2f7%2f8%2f215&data=01%7c01%7ckys%40microsoft.com
> %7c0be64449b7734417b58e08d2a8ac4215%7c72f988bf86f141af91ab2d7cd011
> db47%7c1&sdata=tIGC5%2bms890etIzVbaj3x3B3XUrgC54C79vaniZzRIY%3d
> [2]
> https://na01.safelinks.protection.outlook.com/?url=http%3a%2f%2fmarc.inf
> o%2f%3fl%3dlinux-
> kernel%26m%3d143929238407816%26w%3d2&data=01%7c01%7ckys%40micr
> osoft.com%7c0be64449b7734417b58e08d2a8ac4215%7c72f988bf86f141af91a
> b2d7cd011db47%7c1&sdata=DFEq8NILXnLGTo%2fscI5zjzWrX9%2buJlj9lmo8r
> ahuIt0%3d
> [3]
> https://na01.safelinks.protection.outlook.com/?url=https%3a%2f%2flkml.or
> g%2flkml%2f2014%2f3%2f17%2f68&data=01%7c01%7ckys%40microsoft.com
> %7c0be64449b7734417b58e08d2a8ac4215%7c72f988bf86f141af91ab2d7cd011
> db47%7c1&sdata=j9uJalk7Cup0q78gl8rgIIjySU0l7HIwk1AhYJ5cAd4%3d

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

* Re: [PATCH 0/4] printk: Softlockup avoidance
  2015-08-19 15:38 [PATCH 0/4] printk: Softlockup avoidance Jan Kara
                   ` (4 preceding siblings ...)
  2015-08-20  2:37 ` [PATCH 0/4] printk: Softlockup avoidance KY Srinivasan
@ 2015-09-18 22:14 ` Andrew Morton
  2015-09-22 10:10   ` Jan Kara
  5 siblings, 1 reply; 15+ messages in thread
From: Andrew Morton @ 2015-09-18 22:14 UTC (permalink / raw)
  To: Jan Kara; +Cc: LKML, pmladek, rostedt, Gavin Hu, KY Srinivasan, Jan Kara

On Wed, 19 Aug 2015 17:38:27 +0200 Jan Kara <jack@suse.com> wrote:

> From: Jan Kara <jack@suse.cz>
> 
> Hello,
> 
> since lately there were several attempts at dealing with softlockups due
> to heavy printk traffic [1] [2] and I've been also privately pinged by
> couple of people about the state of the patch set, I've decided to respin
> the patch set.
> 
> To remind the original problem:
> 
> Currently, console_unlock() prints messages from kernel printk buffer to
> console while the buffer is non-empty. When serial console is attached,
> printing is slow and thus other CPUs in the system have plenty of time
> to append new messages to the buffer while one CPU is printing. Thus the
> CPU can spend unbounded amount of time doing printing in console_unlock().
> This is especially serious when printk() gets called under some critical
> spinlock or with interrupts disabled.
>     
> In practice users have observed a CPU can spend tens of seconds printing
> in console_unlock() (usually during boot when hundreds of SCSI devices
> are discovered) resulting in RCU stalls (CPU doing printing doesn't
> reach quiescent state for a long time), softlockup reports (IPIs for the
> printing CPU don't get served and thus other CPUs are spinning waiting
> for the printing CPU to process IPIs), and eventually a machine death
> (as messages from stalls and lockups append to printk buffer faster than
> we are able to print). So these machines are unable to boot with serial
> console attached. Also during artificial stress testing SATA disk
> disappears from the system because its interrupts aren't served for too
> long.
> 
> This series addresses the problem in the following way: If CPU has printed
> more that printk_offload (defaults to 1000) characters, it wakes up one
> of dedicated printk kthreads (we don't use workqueue because that has
> deadlock potential if printk was called from workqueue code). Once we find
> out kthread is spinning on a lock, we stop printing, drop console_sem, and
> let kthread continue printing. Since there are two printing kthreads, they
> will pass printing between them and thus no CPU gets hogged by printing.

I still hate your patchset ;)

But nothing better suggests itself.  I have a few review comments -
please let's work through that stuff, get a fresh version out and we'll
see how it goes.

Is this patchset being used in the field?  Perhaps in the suse kernel? 
If so, a mention of that in the changelog would help things along.


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

* Re: [PATCH 1/4] printk: Hand over printing to console if printing too long
  2015-08-19 15:38 ` [PATCH 1/4] printk: Hand over printing to console if printing too long Jan Kara
@ 2015-09-18 22:14   ` Andrew Morton
  2015-09-22 10:27     ` Jan Kara
  0 siblings, 1 reply; 15+ messages in thread
From: Andrew Morton @ 2015-09-18 22:14 UTC (permalink / raw)
  To: Jan Kara; +Cc: LKML, pmladek, rostedt, Gavin Hu, KY Srinivasan, Jan Kara

On Wed, 19 Aug 2015 17:38:28 +0200 Jan Kara <jack@suse.com> wrote:

> From: Jan Kara <jack@suse.cz>
> 
> Currently, console_unlock() prints messages from kernel printk buffer to
> console while the buffer is non-empty. When serial console is attached,
> printing is slow and thus other CPUs in the system have plenty of time
> to append new messages to the buffer while one CPU is printing. Thus the
> CPU can spend unbounded amount of time doing printing in console_unlock().
> This is especially serious problem if the printk() calling
> console_unlock() was called with interrupts disabled.
> 
> In practice users have observed a CPU can spend tens of seconds printing
> in console_unlock() (usually during boot when hundreds of SCSI devices
> are discovered) resulting in RCU stalls (CPU doing printing doesn't
> reach quiescent state for a long time), softlockup reports (IPIs for the
> printing CPU don't get served and thus other CPUs are spinning waiting
> for the printing CPU to process IPIs), and eventually a machine death
> (as messages from stalls and lockups append to printk buffer faster than
> we are able to print). So these machines are unable to boot with serial
> console attached. Also during artificial stress testing SATA disk
> disappears from the system because its interrupts aren't served for too
> long.
> 
> This patch implements a mechanism where after printing specified number
> of characters (tunable as a kernel parameter printk.offload_chars), CPU
> doing printing asks for help by waking up one of dedicated kthreads.  As
> soon as the printing CPU notices kthread got scheduled and is spinning
> on print_lock dedicated for that purpose, it drops console_sem,
> print_lock, and exits console_unlock(). Kthread then takes over printing
> instead. This way no CPU should spend printing too long even if there
> is heavy printk traffic.
> 
> ...
>
> @@ -2230,6 +2292,8 @@ void console_unlock(void)
>  	unsigned long flags;
>  	bool wake_klogd = false;
>  	bool retry;
> +	bool hand_over = false;
> +	int printed_chars = 0;
>  
>  	if (console_suspended) {
>  		up_console_sem();
> @@ -2241,12 +2305,18 @@ void console_unlock(void)
>  	/* flush buffered message fragment immediately to console */
>  	console_cont_flush(text, sizeof(text));
>  again:
> +	spin_lock(&print_lock);

I'm surprised this isn't spin_lock_irqsave().  How come this isn't
deadlockable?

>  	for (;;) {
>  		struct printk_log *msg;
>  		size_t ext_len = 0;
>  		size_t len;
>  		int level;
>  
> +		if (cpu_stop_printing(printed_chars)) {
> +			hand_over = true;
> +			break;
> +		}
> +
>  		raw_spin_lock_irqsave(&logbuf_lock, flags);
>  		if (seen_seq != log_next_seq) {
>  			wake_klogd = true;
>
> ...
>
> +/* Kthread which takes over printing from a CPU which asks for help */
> +static int printing_task(void *arg)
> +{
> +	DEFINE_WAIT(wait);
> +
> +	while (1) {
> +		prepare_to_wait_exclusive(&print_queue, &wait,
> +					  TASK_INTERRUPTIBLE);
> +		schedule();
> +		finish_wait(&print_queue, &wait);
> +		preempt_disable();

I don't understand the preempt_disable().  Code comment, please?

> +		atomic_inc(&printing_tasks_spinning);
> +		/*
> +		 * Store printing_tasks_spinning value before we spin. Matches
> +		 * the barrier in cpu_stop_printing().
> +		 */
> +		smp_mb__after_atomic();
> +		/*
> +		 * Wait for currently printing thread to complete. We spin on
> +		 * print_lock instead of waiting on console_sem since we don't
> +		 * want to sleep once we got scheduled to make sure we take
> +		 * over printing without depending on the scheduler.
> +		 */
> +		spin_lock(&print_lock);
> +		atomic_dec(&printing_tasks_spinning);
> +		spin_unlock(&print_lock);
> +		if (console_trylock())
> +			console_unlock();
> +		preempt_enable();
> +	}
> +	return 0;
> +}
>
> ...
>

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

* Re: [PATCH 3/4] kernel: Avoid softlockups in stop_machine() during heavy printing
  2015-08-19 15:38 ` [PATCH 3/4] kernel: Avoid softlockups in stop_machine() during heavy printing Jan Kara
@ 2015-09-18 22:15   ` Andrew Morton
  2015-09-22 10:55     ` Jan Kara
  0 siblings, 1 reply; 15+ messages in thread
From: Andrew Morton @ 2015-09-18 22:15 UTC (permalink / raw)
  To: Jan Kara; +Cc: LKML, pmladek, rostedt, Gavin Hu, KY Srinivasan, Jan Kara

On Wed, 19 Aug 2015 17:38:30 +0200 Jan Kara <jack@suse.com> wrote:

> From: Jan Kara <jack@suse.cz>
> 
> When there are lots of messages accumulated in printk buffer, printing
> them (especially over serial console) can take a long time (tens of
> seconds). stop_machine() will effectively make all cpus spin in
> multi_cpu_stop() waiting for the CPU doing printing to print all the
> messages which triggers NMI softlockup watchdog and RCU stall detector
> which add even more to the messages to print. Since machine doesn't do
> anything (except serving interrupts) during this time, also network
> connections are dropped and other disturbances may happen.
> 
> Paper over the problem by waiting for printk buffer to be empty before
> starting to stop CPUs. In theory a burst of new messages can be appended
> to the printk buffer before CPUs enter multi_cpu_stop() so this isn't a 100%
> solution but it works OK in practice and I'm not aware of a reasonably
> simple better solution.

Confused.  Why don't patches 1 and 2 already fix this problem?

>
> ...
>
> @@ -2489,6 +2489,28 @@ struct tty_driver *console_device(int *index)
>  }
>  
>  /*
> + * Wait until all messages accumulated in the printk buffer are printed to
> + * console. Note that as soon as this function returns, new messages may be
> + * added to the printk buffer by other CPUs.
> + */
> +void console_flush(void)

This doesn't seem a very good name.  We already have
console_cont_flush(), cont_flush(), etc.  Can we think of something
more specific?  printk_log_buf_drain() perhaps.

> +{
> +	bool retry;
> +	unsigned long flags;
> +
> +	while (1) {
> +		raw_spin_lock_irqsave(&logbuf_lock, flags);
> +		retry = console_seq != log_next_seq;
> +		raw_spin_unlock_irqrestore(&logbuf_lock, flags);

Does this lock/unlock do anything useful?

> +		if (!retry || console_suspended)
> +			break;
> +		/* Cycle console_sem to wait for outstanding printing */
> +		console_lock();
> +		console_unlock();
> +	}
> +}
> +
> +/*
>   * Prevent further output on the passed console device so that (for example)
>   * serial drivers can disable console output before suspending a port, and can
>   * re-enable output afterwards.
> diff --git a/kernel/stop_machine.c b/kernel/stop_machine.c
> index fd643d8c4b42..016d34621d2e 100644
> --- a/kernel/stop_machine.c
> +++ b/kernel/stop_machine.c
> @@ -21,6 +21,7 @@
>  #include <linux/smpboot.h>
>  #include <linux/atomic.h>
>  #include <linux/lglock.h>
> +#include <linux/console.h>
>  
>  /*
>   * Structure to determine completion condition and record errors.  May
> @@ -543,6 +544,14 @@ int __stop_machine(int (*fn)(void *), void *data, const struct cpumask *cpus)
>  		return ret;
>  	}
>  
> +	/*
> +	 * If there are lots of outstanding messages, printing them can take a
> +	 * long time and all cpus would be spinning waiting for the printing to
> +	 * finish thus triggering NMI watchdog, RCU lockups etc. Wait for the
> +	 * printing here to avoid these.
> +	 */
> +	console_flush();

This is pretty pointless if num_possible_cpus==1.  I'd suggest setting
printk_offload_chars=0 in this case, add some early bale-out into
console_flush().  Or something along those lines.

And make console_flush() go away altogether if CONFIG_SMP=n - it's
pointless bloat.

>  	/* Set the initial state and stop all online cpus. */
>  	set_state(&msdata, MULTI_STOP_PREPARE);
>  	return stop_cpus(cpu_online_mask, multi_cpu_stop, &msdata);


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

* Re: [PATCH 4/4] printk: Add config option for disabling printk offloading
  2015-08-19 15:38 ` [PATCH 4/4] printk: Add config option for disabling printk offloading Jan Kara
@ 2015-09-18 22:15   ` Andrew Morton
  2015-09-22 11:51     ` Jan Kara
  0 siblings, 1 reply; 15+ messages in thread
From: Andrew Morton @ 2015-09-18 22:15 UTC (permalink / raw)
  To: Jan Kara; +Cc: LKML, pmladek, rostedt, Gavin Hu, KY Srinivasan, Jan Kara

On Wed, 19 Aug 2015 17:38:31 +0200 Jan Kara <jack@suse.com> wrote:

> From: Jan Kara <jack@suse.cz>
> 
> Necessity for offloading of printing was observed only for large
> systems. So add a config option (disabled by default)

The Kconfig has "default y"?

> which removes most
> of the overhead added by this functionality.
> 
> ...
>
> +#ifdef CONFIG_PRINTK_OFFLOAD
>  /*
>   * Returns true iff there is other cpu waiting to take over printing. This
>   * function also takes are of setting PRINTK_HANDOVER_B if we want to hand over
> @@ -2278,6 +2283,14 @@ static bool cpu_stop_printing(int printed_chars)
>  
>  	return false;
>  }
> +#else
> +
> +static bool cpu_stop_printing(int printed_chars, bool *woken)
> +{
> +	return false;
> +}
> +
> +#endif
>  
>  /**
>   * console_unlock - unlock the console system
> @@ -2316,7 +2329,9 @@ void console_unlock(void)
>  	/* flush buffered message fragment immediately to console */
>  	console_cont_flush(text, sizeof(text));
>  again:
> +#ifdef CONFIG_PRINTK_OFFLOAD
>  	spin_lock(&print_lock);
> +#endif

You could nuke a couple of ugly ifdefs by adding
spin_[un]lock_printk_lock() wrappers into that ifdef/else/endif block
which holds cpu_stop_printing().

>  	for (;;) {
>  		struct printk_log *msg;
>  		size_t ext_len = 0;
> @@ -2399,12 +2414,14 @@ skip:
>  
>  	console_locked = 0;
>  	up_console_sem();
> +#ifdef CONFIG_PRINTK_OFFLOAD
>  	/*
>  	 * Release print_lock after console_sem so that printing_task()
>  	 * succeeds in getting console_sem (unless someone else takes it and
>  	 * then he'll be responsible for printing).
>           */
>  	spin_unlock(&print_lock);
> +#endif
>  
>  	/*
>  	 * Subtlety: We have interrupts disabled iff hand_over == false (to
> @@ -2770,6 +2787,7 @@ int unregister_console(struct console *console)
>  }
>  EXPORT_SYMBOL(unregister_console);
>  
> +#ifdef CONFIG_PRINTK_OFFLOAD
>  /* Kthread which takes over printing from a CPU which asks for help */
>  static int printing_task(void *arg)
>  {
> @@ -2838,6 +2856,7 @@ static int offload_chars_set(const char *val, const struct kernel_param *kp)
>  	mutex_unlock(&printk_kthread_mutex);
>  	return 0;
>  }
> +#endif	/* CONFIG_PRINTK_OFFLOAD */
>  
>  static int __init printk_late_init(void)
>  {
> @@ -2850,9 +2869,11 @@ static int __init printk_late_init(void)
>  	}
>  	hotcpu_notifier(console_cpu_notify, 0);
>  
> +#ifdef CONFIG_PRINTK_OFFLOAD
>  	mutex_lock(&printk_kthread_mutex);
>  	printk_start_offload_kthreads();
>  	mutex_unlock(&printk_kthread_mutex);
> +#endif

Possibly ditto here.  Maybe move the mutex_lock() into
printk_start_offload_kthreads() too?




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

* Re: [PATCH 0/4] printk: Softlockup avoidance
  2015-09-18 22:14 ` Andrew Morton
@ 2015-09-22 10:10   ` Jan Kara
  0 siblings, 0 replies; 15+ messages in thread
From: Jan Kara @ 2015-09-22 10:10 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Jan Kara, LKML, pmladek, rostedt, Gavin Hu, KY Srinivasan, Jan Kara

On Fri 18-09-15 15:14:11, Andrew Morton wrote:
> On Wed, 19 Aug 2015 17:38:27 +0200 Jan Kara <jack@suse.com> wrote:
> 
> > From: Jan Kara <jack@suse.cz>
> > 
> > Hello,
> > 
> > since lately there were several attempts at dealing with softlockups due
> > to heavy printk traffic [1] [2] and I've been also privately pinged by
> > couple of people about the state of the patch set, I've decided to respin
> > the patch set.
> > 
> > To remind the original problem:
> > 
> > Currently, console_unlock() prints messages from kernel printk buffer to
> > console while the buffer is non-empty. When serial console is attached,
> > printing is slow and thus other CPUs in the system have plenty of time
> > to append new messages to the buffer while one CPU is printing. Thus the
> > CPU can spend unbounded amount of time doing printing in console_unlock().
> > This is especially serious when printk() gets called under some critical
> > spinlock or with interrupts disabled.
> >     
> > In practice users have observed a CPU can spend tens of seconds printing
> > in console_unlock() (usually during boot when hundreds of SCSI devices
> > are discovered) resulting in RCU stalls (CPU doing printing doesn't
> > reach quiescent state for a long time), softlockup reports (IPIs for the
> > printing CPU don't get served and thus other CPUs are spinning waiting
> > for the printing CPU to process IPIs), and eventually a machine death
> > (as messages from stalls and lockups append to printk buffer faster than
> > we are able to print). So these machines are unable to boot with serial
> > console attached. Also during artificial stress testing SATA disk
> > disappears from the system because its interrupts aren't served for too
> > long.
> > 
> > This series addresses the problem in the following way: If CPU has printed
> > more that printk_offload (defaults to 1000) characters, it wakes up one
> > of dedicated printk kthreads (we don't use workqueue because that has
> > deadlock potential if printk was called from workqueue code). Once we find
> > out kthread is spinning on a lock, we stop printing, drop console_sem, and
> > let kthread continue printing. Since there are two printing kthreads, they
> > will pass printing between them and thus no CPU gets hogged by printing.
> 
> I still hate your patchset ;)
> 
> But nothing better suggests itself.  I have a few review comments -
> please let's work through that stuff, get a fresh version out and we'll
> see how it goes.
> 
> Is this patchset being used in the field?  Perhaps in the suse kernel? 
> If so, a mention of that in the changelog would help things along.

Yes, SUSE kernels contain these patches (well, previous versions of the
patch set...). So far they fix the issues reported by customers we haven't
observed any problems with those patches.

								Honza
-- 
Jan Kara <jack@suse.com>
SUSE Labs, CR

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

* Re: [PATCH 1/4] printk: Hand over printing to console if printing too long
  2015-09-18 22:14   ` Andrew Morton
@ 2015-09-22 10:27     ` Jan Kara
  0 siblings, 0 replies; 15+ messages in thread
From: Jan Kara @ 2015-09-22 10:27 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Jan Kara, LKML, pmladek, rostedt, Gavin Hu, KY Srinivasan, Jan Kara

On Fri 18-09-15 15:14:59, Andrew Morton wrote:
> On Wed, 19 Aug 2015 17:38:28 +0200 Jan Kara <jack@suse.com> wrote:
> 
> > From: Jan Kara <jack@suse.cz>
> > 
> > Currently, console_unlock() prints messages from kernel printk buffer to
> > console while the buffer is non-empty. When serial console is attached,
> > printing is slow and thus other CPUs in the system have plenty of time
> > to append new messages to the buffer while one CPU is printing. Thus the
> > CPU can spend unbounded amount of time doing printing in console_unlock().
> > This is especially serious problem if the printk() calling
> > console_unlock() was called with interrupts disabled.
> > 
> > In practice users have observed a CPU can spend tens of seconds printing
> > in console_unlock() (usually during boot when hundreds of SCSI devices
> > are discovered) resulting in RCU stalls (CPU doing printing doesn't
> > reach quiescent state for a long time), softlockup reports (IPIs for the
> > printing CPU don't get served and thus other CPUs are spinning waiting
> > for the printing CPU to process IPIs), and eventually a machine death
> > (as messages from stalls and lockups append to printk buffer faster than
> > we are able to print). So these machines are unable to boot with serial
> > console attached. Also during artificial stress testing SATA disk
> > disappears from the system because its interrupts aren't served for too
> > long.
> > 
> > This patch implements a mechanism where after printing specified number
> > of characters (tunable as a kernel parameter printk.offload_chars), CPU
> > doing printing asks for help by waking up one of dedicated kthreads.  As
> > soon as the printing CPU notices kthread got scheduled and is spinning
> > on print_lock dedicated for that purpose, it drops console_sem,
> > print_lock, and exits console_unlock(). Kthread then takes over printing
> > instead. This way no CPU should spend printing too long even if there
> > is heavy printk traffic.
> > 
> > ...
> >
> > @@ -2230,6 +2292,8 @@ void console_unlock(void)
> >  	unsigned long flags;
> >  	bool wake_klogd = false;
> >  	bool retry;
> > +	bool hand_over = false;
> > +	int printed_chars = 0;
> >  
> >  	if (console_suspended) {
> >  		up_console_sem();
> > @@ -2241,12 +2305,18 @@ void console_unlock(void)
> >  	/* flush buffered message fragment immediately to console */
> >  	console_cont_flush(text, sizeof(text));
> >  again:
> > +	spin_lock(&print_lock);
> 
> I'm surprised this isn't spin_lock_irqsave().  How come this isn't
> deadlockable?

Yes, it should be spin_lock_irqsave(). My original plan was to nest
print_lock inside logbuf_lock which would provide the protection but later
I've ordered them the other way around and forgot to update the irq
protection. Will fix.

> >  	for (;;) {
> >  		struct printk_log *msg;
> >  		size_t ext_len = 0;
> >  		size_t len;
> >  		int level;
> >  
> > +		if (cpu_stop_printing(printed_chars)) {
> > +			hand_over = true;
> > +			break;
> > +		}
> > +
> >  		raw_spin_lock_irqsave(&logbuf_lock, flags);
> >  		if (seen_seq != log_next_seq) {
> >  			wake_klogd = true;
> >
> > ...
> >
> > +/* Kthread which takes over printing from a CPU which asks for help */
> > +static int printing_task(void *arg)
> > +{
> > +	DEFINE_WAIT(wait);
> > +
> > +	while (1) {
> > +		prepare_to_wait_exclusive(&print_queue, &wait,
> > +					  TASK_INTERRUPTIBLE);
> > +		schedule();
> > +		finish_wait(&print_queue, &wait);
> > +		preempt_disable();
> 
> I don't understand the preempt_disable().  Code comment, please?

We don't want to be scheduled away in preemptible kernels when spinning on
print_lock or after we acquired print_lock and before we got console_sem.
I'll add a comment.

Thanks for review!
								Honza

> 
> > +		atomic_inc(&printing_tasks_spinning);
> > +		/*
> > +		 * Store printing_tasks_spinning value before we spin. Matches
> > +		 * the barrier in cpu_stop_printing().
> > +		 */
> > +		smp_mb__after_atomic();
> > +		/*
> > +		 * Wait for currently printing thread to complete. We spin on
> > +		 * print_lock instead of waiting on console_sem since we don't
> > +		 * want to sleep once we got scheduled to make sure we take
> > +		 * over printing without depending on the scheduler.
> > +		 */
> > +		spin_lock(&print_lock);
> > +		atomic_dec(&printing_tasks_spinning);
> > +		spin_unlock(&print_lock);
> > +		if (console_trylock())
> > +			console_unlock();
> > +		preempt_enable();
> > +	}
> > +	return 0;
> > +}
> >
> > ...
> >
-- 
Jan Kara <jack@suse.com>
SUSE Labs, CR

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

* Re: [PATCH 3/4] kernel: Avoid softlockups in stop_machine() during heavy printing
  2015-09-18 22:15   ` Andrew Morton
@ 2015-09-22 10:55     ` Jan Kara
  2015-09-23  8:37       ` Jan Kara
  0 siblings, 1 reply; 15+ messages in thread
From: Jan Kara @ 2015-09-22 10:55 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Jan Kara, LKML, pmladek, rostedt, Gavin Hu, KY Srinivasan, Jan Kara

On Fri 18-09-15 15:15:21, Andrew Morton wrote:
> On Wed, 19 Aug 2015 17:38:30 +0200 Jan Kara <jack@suse.com> wrote:
> 
> > From: Jan Kara <jack@suse.cz>
> > 
> > When there are lots of messages accumulated in printk buffer, printing
> > them (especially over serial console) can take a long time (tens of
> > seconds). stop_machine() will effectively make all cpus spin in
> > multi_cpu_stop() waiting for the CPU doing printing to print all the
> > messages which triggers NMI softlockup watchdog and RCU stall detector
> > which add even more to the messages to print. Since machine doesn't do
> > anything (except serving interrupts) during this time, also network
> > connections are dropped and other disturbances may happen.
> > 
> > Paper over the problem by waiting for printk buffer to be empty before
> > starting to stop CPUs. In theory a burst of new messages can be appended
> > to the printk buffer before CPUs enter multi_cpu_stop() so this isn't a 100%
> > solution but it works OK in practice and I'm not aware of a reasonably
> > simple better solution.
> 
> Confused.  Why don't patches 1 and 2 already fix this problem?

Because stop_machine() will not allow printing threads to run on any CPU
(all but one CPUs are spinning in multi_cpu_stop() without possibility of
preemption) and thus any printk offloading cannot happen.

> > ...
> >
> > @@ -2489,6 +2489,28 @@ struct tty_driver *console_device(int *index)
> >  }
> >  
> >  /*
> > + * Wait until all messages accumulated in the printk buffer are printed to
> > + * console. Note that as soon as this function returns, new messages may be
> > + * added to the printk buffer by other CPUs.
> > + */
> > +void console_flush(void)
> 
> This doesn't seem a very good name.  We already have
> console_cont_flush(), cont_flush(), etc.  Can we think of something
> more specific?  printk_log_buf_drain() perhaps.

Thanks for suggestion. I'll change the name.

> > +{
> > +	bool retry;
> > +	unsigned long flags;
> > +
> > +	while (1) {
> > +		raw_spin_lock_irqsave(&logbuf_lock, flags);
> > +		retry = console_seq != log_next_seq;
> > +		raw_spin_unlock_irqrestore(&logbuf_lock, flags);
> 
> Does this lock/unlock do anything useful?
> 
> > +		if (!retry || console_suspended)
> > +			break;
> > +		/* Cycle console_sem to wait for outstanding printing */
> > +		console_lock();
> > +		console_unlock();
> > +	}
> > +}
> > +
> > +/*
> >   * Prevent further output on the passed console device so that (for example)
> >   * serial drivers can disable console output before suspending a port, and can
> >   * re-enable output afterwards.
> > diff --git a/kernel/stop_machine.c b/kernel/stop_machine.c
> > index fd643d8c4b42..016d34621d2e 100644
> > --- a/kernel/stop_machine.c
> > +++ b/kernel/stop_machine.c
> > @@ -21,6 +21,7 @@
> >  #include <linux/smpboot.h>
> >  #include <linux/atomic.h>
> >  #include <linux/lglock.h>
> > +#include <linux/console.h>
> >  
> >  /*
> >   * Structure to determine completion condition and record errors.  May
> > @@ -543,6 +544,14 @@ int __stop_machine(int (*fn)(void *), void *data, const struct cpumask *cpus)
> >  		return ret;
> >  	}
> >  
> > +	/*
> > +	 * If there are lots of outstanding messages, printing them can take a
> > +	 * long time and all cpus would be spinning waiting for the printing to
> > +	 * finish thus triggering NMI watchdog, RCU lockups etc. Wait for the
> > +	 * printing here to avoid these.
> > +	 */
> > +	console_flush();
> 
> This is pretty pointless if num_possible_cpus==1.  I'd suggest setting
> printk_offload_chars=0 in this case, add some early bale-out into
> console_flush().  Or something along those lines.
> 
> And make console_flush() go away altogether if CONFIG_SMP=n - it's
> pointless bloat.

Sure, I'll do that.

								Honza
-- 
Jan Kara <jack@suse.com>
SUSE Labs, CR

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

* Re: [PATCH 4/4] printk: Add config option for disabling printk offloading
  2015-09-18 22:15   ` Andrew Morton
@ 2015-09-22 11:51     ` Jan Kara
  0 siblings, 0 replies; 15+ messages in thread
From: Jan Kara @ 2015-09-22 11:51 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Jan Kara, LKML, pmladek, rostedt, Gavin Hu, KY Srinivasan, Jan Kara

On Fri 18-09-15 15:15:36, Andrew Morton wrote:
> On Wed, 19 Aug 2015 17:38:31 +0200 Jan Kara <jack@suse.com> wrote:
> 
> > From: Jan Kara <jack@suse.cz>
> > 
> > Necessity for offloading of printing was observed only for large
> > systems. So add a config option (disabled by default)
> 
> The Kconfig has "default y"?

Changed to 'n'. Thanks for spotting this.

> > which removes most
> > of the overhead added by this functionality.
> > 
> > ...
> >
> > +#ifdef CONFIG_PRINTK_OFFLOAD
> >  /*
> >   * Returns true iff there is other cpu waiting to take over printing. This
> >   * function also takes are of setting PRINTK_HANDOVER_B if we want to hand over
> > @@ -2278,6 +2283,14 @@ static bool cpu_stop_printing(int printed_chars)
> >  
> >  	return false;
> >  }
> > +#else
> > +
> > +static bool cpu_stop_printing(int printed_chars, bool *woken)
> > +{
> > +	return false;
> > +}
> > +
> > +#endif
> >  
> >  /**
> >   * console_unlock - unlock the console system
> > @@ -2316,7 +2329,9 @@ void console_unlock(void)
> >  	/* flush buffered message fragment immediately to console */
> >  	console_cont_flush(text, sizeof(text));
> >  again:
> > +#ifdef CONFIG_PRINTK_OFFLOAD
> >  	spin_lock(&print_lock);
> > +#endif
> 
> You could nuke a couple of ugly ifdefs by adding
> spin_[un]lock_printk_lock() wrappers into that ifdef/else/endif block
> which holds cpu_stop_printing().

Yup, done.

> >  	for (;;) {
> >  		struct printk_log *msg;
> >  		size_t ext_len = 0;
> > @@ -2399,12 +2414,14 @@ skip:
> >  
> >  	console_locked = 0;
> >  	up_console_sem();
> > +#ifdef CONFIG_PRINTK_OFFLOAD
> >  	/*
> >  	 * Release print_lock after console_sem so that printing_task()
> >  	 * succeeds in getting console_sem (unless someone else takes it and
> >  	 * then he'll be responsible for printing).
> >           */
> >  	spin_unlock(&print_lock);
> > +#endif
> >  
> >  	/*
> >  	 * Subtlety: We have interrupts disabled iff hand_over == false (to
> > @@ -2770,6 +2787,7 @@ int unregister_console(struct console *console)
> >  }
> >  EXPORT_SYMBOL(unregister_console);
> >  
> > +#ifdef CONFIG_PRINTK_OFFLOAD
> >  /* Kthread which takes over printing from a CPU which asks for help */
> >  static int printing_task(void *arg)
> >  {
> > @@ -2838,6 +2856,7 @@ static int offload_chars_set(const char *val, const struct kernel_param *kp)
> >  	mutex_unlock(&printk_kthread_mutex);
> >  	return 0;
> >  }
> > +#endif	/* CONFIG_PRINTK_OFFLOAD */
> >  
> >  static int __init printk_late_init(void)
> >  {
> > @@ -2850,9 +2869,11 @@ static int __init printk_late_init(void)
> >  	}
> >  	hotcpu_notifier(console_cpu_notify, 0);
> >  
> > +#ifdef CONFIG_PRINTK_OFFLOAD
> >  	mutex_lock(&printk_kthread_mutex);
> >  	printk_start_offload_kthreads();
> >  	mutex_unlock(&printk_kthread_mutex);
> > +#endif
> 
> Possibly ditto here.  Maybe move the mutex_lock() into
> printk_start_offload_kthreads() too?

OK, done.

								Honza
-- 
Jan Kara <jack@suse.com>
SUSE Labs, CR

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

* Re: [PATCH 3/4] kernel: Avoid softlockups in stop_machine() during heavy printing
  2015-09-22 10:55     ` Jan Kara
@ 2015-09-23  8:37       ` Jan Kara
  0 siblings, 0 replies; 15+ messages in thread
From: Jan Kara @ 2015-09-23  8:37 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Jan Kara, LKML, pmladek, rostedt, Gavin Hu, KY Srinivasan, Jan Kara

On Tue 22-09-15 12:55:02, Jan Kara wrote:
> > > +{
> > > +	bool retry;
> > > +	unsigned long flags;
> > > +
> > > +	while (1) {
> > > +		raw_spin_lock_irqsave(&logbuf_lock, flags);
> > > +		retry = console_seq != log_next_seq;
> > > +		raw_spin_unlock_irqrestore(&logbuf_lock, flags);
> > 
> > Does this lock/unlock do anything useful?

Forgot to comment on this: console_seq and log_next_seq are updated under
logbuf_lock. Also they are 64-bit so on 32-bit archs their updates are
non-atomic. So although in practice the check will likely work fine without
logbuf_lock, I prefer taking the lock to save reader some pondering and the
code isn't performance sensitive in any way.

> > > +		if (!retry || console_suspended)
> > > +			break;
> > > +		/* Cycle console_sem to wait for outstanding printing */
> > > +		console_lock();
> > > +		console_unlock();
> > > +	}
> > > +}
> > > +
> > > +/*
> > >   * Prevent further output on the passed console device so that (for example)
> > >   * serial drivers can disable console output before suspending a port, and can
> > >   * re-enable output afterwards.
> > > diff --git a/kernel/stop_machine.c b/kernel/stop_machine.c
> > > index fd643d8c4b42..016d34621d2e 100644
> > > --- a/kernel/stop_machine.c
> > > +++ b/kernel/stop_machine.c
> > > @@ -21,6 +21,7 @@
> > >  #include <linux/smpboot.h>
> > >  #include <linux/atomic.h>
> > >  #include <linux/lglock.h>
> > > +#include <linux/console.h>
> > >  
> > >  /*
> > >   * Structure to determine completion condition and record errors.  May
> > > @@ -543,6 +544,14 @@ int __stop_machine(int (*fn)(void *), void *data, const struct cpumask *cpus)
> > >  		return ret;
> > >  	}
> > >  
> > > +	/*
> > > +	 * If there are lots of outstanding messages, printing them can take a
> > > +	 * long time and all cpus would be spinning waiting for the printing to
> > > +	 * finish thus triggering NMI watchdog, RCU lockups etc. Wait for the
> > > +	 * printing here to avoid these.
> > > +	 */
> > > +	console_flush();
> > 
> > This is pretty pointless if num_possible_cpus==1.  I'd suggest setting
> > printk_offload_chars=0 in this case, add some early bale-out into
> > console_flush().  Or something along those lines.
> > 
> > And make console_flush() go away altogether if CONFIG_SMP=n - it's
> > pointless bloat.
> 
> Sure, I'll do that.

I've implemented all this except for early bail out from console_flush()
when num_possible_cpus==1 - that doesn't seem very useful since we'll just
check that the buffer is empty and bail out anyway...

								Honza
-- 
Jan Kara <jack@suse.com>
SUSE Labs, CR

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

end of thread, other threads:[~2015-09-23  8:37 UTC | newest]

Thread overview: 15+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2015-08-19 15:38 [PATCH 0/4] printk: Softlockup avoidance Jan Kara
2015-08-19 15:38 ` [PATCH 1/4] printk: Hand over printing to console if printing too long Jan Kara
2015-09-18 22:14   ` Andrew Morton
2015-09-22 10:27     ` Jan Kara
2015-08-19 15:38 ` [PATCH 2/4] printk: Start printing handover kthreads on demand Jan Kara
2015-08-19 15:38 ` [PATCH 3/4] kernel: Avoid softlockups in stop_machine() during heavy printing Jan Kara
2015-09-18 22:15   ` Andrew Morton
2015-09-22 10:55     ` Jan Kara
2015-09-23  8:37       ` Jan Kara
2015-08-19 15:38 ` [PATCH 4/4] printk: Add config option for disabling printk offloading Jan Kara
2015-09-18 22:15   ` Andrew Morton
2015-09-22 11:51     ` Jan Kara
2015-08-20  2:37 ` [PATCH 0/4] printk: Softlockup avoidance KY Srinivasan
2015-09-18 22:14 ` Andrew Morton
2015-09-22 10:10   ` Jan Kara

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