linux-block.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [RFC PATCH 0/3] softirq/blk-mq: implement interrupt flood detection for avoiding cpu lockup
@ 2019-12-18  7:19 Ming Lei
  2019-12-18  7:19 ` [RFC PATCH 1/3] sched/core: add API for exporting runqueue clock Ming Lei
                   ` (2 more replies)
  0 siblings, 3 replies; 15+ messages in thread
From: Ming Lei @ 2019-12-18  7:19 UTC (permalink / raw)
  To: Thomas Gleixner, Jens Axboe
  Cc: linux-kernel, linux-block, Ming Lei, Long Li, Ingo Molnar,
	Peter Zijlstra, Keith Busch, Christoph Hellwig, Sagi Grimberg,
	John Garry, Hannes Reinecke

Hi Guys,

For minimizing IO latency, blk-mq often completes IO in the hardware
interrupt context. However, in case of multiple HBAs, or multiple
storage devices attached to same HBA, if one CPU core has to handle
interrupts of IOs submitted from multile CPU , there is risk to lock
up CPUs[1][2].

Follows the idea when there is mutliple IO submitter and single
CPU for completing these IOs:

1) in case of multiple storages attached to one single HBA, these
storages may handle IO more quickly than single CPU core

2) in case of multiple HBAs in one system, one single effective CPU
can be selected for handling interrupts from several queues, then
multiple storages still may handle IO more quickly than single CPU
core.

When handling IO completion in interrupt context, IO latency is good,
but there is risk to lock up CPU. When moving IO completion to process
context via NAPI or threaded interrupt handler, CPU lockup can be
avoided. So people try to move part of IO completion into process
context for avoiding CPU lockup, meantime trying to not hurt IO
performance, such as Keith's work[3].

However, it is hard to partition IO completion in the two contexts, if
less work is moved to process context, risk of locking up CPU can't be
eliminated; if more work is moved to process context, extra IO latency is
introduced, then IO performance is hurt.

Interrupt flood information can be one useful hint for partitioning IO
completion work into the two contexts effectively.

The 1st two patches implement interrpupt flood detection, and the 3rd
patch applies the hint to complet IO in process context when interrupt
flood happens. This way avoids CPU lockup, meantime IO performance isn't
hurt obviously.

[1] https://lore.kernel.org/lkml/1566281669-48212-1-git-send-email-longli@linuxonhyperv.com/
[2] https://lore.kernel.org/lkml/a7ef3810-31af-013a-6d18-ceb6154aa2ef@huawei.com/
[3] https://lore.kernel.org/linux-nvme/20191209175622.1964-1-kbusch@kernel.org/T/#t

Ming Lei (3):
  sched/core: add API for exporting runqueue clock
  softirq: implement interrupt flood detection
  blk-mq: complete request in rescuer process context in case of irq
    flood

 block/blk-mq.c          |  68 ++++++++++++++++-
 drivers/base/cpu.c      |  23 ++++++
 include/linux/hardirq.h |   2 +
 include/linux/sched.h   |   2 +
 kernel/sched/core.c     |   5 ++
 kernel/softirq.c        | 161 ++++++++++++++++++++++++++++++++++++++++
 6 files changed, 260 insertions(+), 1 deletion(-)

Cc: Long Li <longli@microsoft.com>
Cc: Ingo Molnar <mingo@redhat.com>,
Cc: Peter Zijlstra <peterz@infradead.org>
Cc: Keith Busch <keith.busch@intel.com>
Cc: Christoph Hellwig <hch@lst.de>
Cc: Sagi Grimberg <sagi@grimberg.me>
Cc: John Garry <john.garry@huawei.com>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: Hannes Reinecke <hare@suse.com>
-- 
2.20.1


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

* [RFC PATCH 1/3] sched/core: add API for exporting runqueue clock
  2019-12-18  7:19 [RFC PATCH 0/3] softirq/blk-mq: implement interrupt flood detection for avoiding cpu lockup Ming Lei
@ 2019-12-18  7:19 ` Ming Lei
  2019-12-18  9:51   ` Peter Zijlstra
  2019-12-18  7:19 ` [RFC PATCH 2/3] softirq: implement interrupt flood detection Ming Lei
  2019-12-18  7:19 ` [RFC PATCH 3/3] blk-mq: complete request in rescuer process context in case of irq flood Ming Lei
  2 siblings, 1 reply; 15+ messages in thread
From: Ming Lei @ 2019-12-18  7:19 UTC (permalink / raw)
  To: Thomas Gleixner, Jens Axboe
  Cc: linux-kernel, linux-block, Ming Lei, Long Li, Ingo Molnar,
	Peter Zijlstra, Keith Busch, Christoph Hellwig, Sagi Grimberg,
	John Garry, Hannes Reinecke

Scheduler runqueue maintains its own software clock that is periodically
synchronised with hardware. Export this clock so that it can be used
by interrupt flood detection for saving the cost of reading from hardware.

Cc: Long Li <longli@microsoft.com>
Cc: Ingo Molnar <mingo@redhat.com>,
Cc: Peter Zijlstra <peterz@infradead.org>
Cc: Keith Busch <keith.busch@intel.com>
Cc: Christoph Hellwig <hch@lst.de>
Cc: Sagi Grimberg <sagi@grimberg.me>
Cc: John Garry <john.garry@huawei.com>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: Hannes Reinecke <hare@suse.com>
Signed-off-by: Ming Lei <ming.lei@redhat.com>
---
 include/linux/sched.h | 2 ++
 kernel/sched/core.c   | 5 +++++
 2 files changed, 7 insertions(+)

diff --git a/include/linux/sched.h b/include/linux/sched.h
index 467d26046416..efe1a3ec0e9e 100644
--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -2011,4 +2011,6 @@ int sched_trace_rq_cpu(struct rq *rq);
 
 const struct cpumask *sched_trace_rd_span(struct root_domain *rd);
 
+u64 sched_local_rq_clock(void);
+
 #endif
diff --git a/kernel/sched/core.c b/kernel/sched/core.c
index 90e4b00ace89..03e2e3c36067 100644
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -219,6 +219,11 @@ void update_rq_clock(struct rq *rq)
 	update_rq_clock_task(rq, delta);
 }
 
+u64 sched_local_rq_clock(void)
+{
+	return this_rq()->clock;
+}
+EXPORT_SYMBOL_GPL(sched_local_rq_clock);
 
 #ifdef CONFIG_SCHED_HRTICK
 /*
-- 
2.20.1


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

* [RFC PATCH 2/3] softirq: implement interrupt flood detection
  2019-12-18  7:19 [RFC PATCH 0/3] softirq/blk-mq: implement interrupt flood detection for avoiding cpu lockup Ming Lei
  2019-12-18  7:19 ` [RFC PATCH 1/3] sched/core: add API for exporting runqueue clock Ming Lei
@ 2019-12-18  7:19 ` Ming Lei
  2019-12-18 10:49   ` Peter Zijlstra
  2019-12-18  7:19 ` [RFC PATCH 3/3] blk-mq: complete request in rescuer process context in case of irq flood Ming Lei
  2 siblings, 1 reply; 15+ messages in thread
From: Ming Lei @ 2019-12-18  7:19 UTC (permalink / raw)
  To: Thomas Gleixner, Jens Axboe
  Cc: linux-kernel, linux-block, Ming Lei, Long Li, Ingo Molnar,
	Peter Zijlstra, Christoph Hellwig, Keith Busch, Sagi Grimberg,
	John Garry, Hannes Reinecke

For some high performance IO devices, interrupt may come very frequently,
meantime IO request is often completed in irq context for minimizing IO
latency. Especially on some devices(SCSI or NVMe), IO requests can be
submitted concurrently from multiple CPU cores, meantime these IO's
completion is only done on single CPU core. Either fast IO storage is
comming, or multiple storage devices attached to same HBA, or multiple
HBAs in same system, IOs from multiple storage devices may saturate one
single CPU core for hanlding these IO's interrups.

Interrupt flood can be triggered, and scheduler is stuck, then CPU lockup
is followed.

The interrupt flood infomation is very useful for the following use
cases:

1) avoid CPU soft lockup

After interrupt flood is detected, block layer may switch to complete IO
in process context, then CPU lockup can be avoided. And switch back to
interrupt context after interrupt flood is gone.

2) partition IO completion into interrupt and process contexts effectively

When handling IO completion in interrupt context, IO latency is good,
but there is risk to lock up CPU. When moving IO completion to process
context via NAPI or threaded interrupt handler, CPU lockup can be
avoided. So people try to move part of IO completion into process
context for avoiding CPU lockup, such as Keith's work:

https://lore.kernel.org/linux-nvme/20191209175622.1964-1-kbusch@kernel.org/T/#t

However, it is hard to partition IO completion in the two contexts, if
less work is moved to process context, risk of locking up CPU can't be
eliminated; if more work is moved to process context, extra IO latency is
introduced, then IO performance is hurt.

Interrupt flood information can be one useful hint for partitioning IO
completion work into the two contexts effectively.

Implement one simple and efficient CPU interrupt flood detection mechanism.
This mechanism uses average interrupt interval on each CPU to evaluate if
interrupt flood is triggered. For saving cost of reading clock from hardware,
applies two-stage detection:

- in the 1st stage, read clock from scheduler runqueue clock which is
software clock, and the cost is quite low

- after small enough interrupt interval is observed in the 1st stage, switch
to the 2nd stage detection given it is reasonable to assume that scheduler
has been stuck

- the 2nd stage is for confirming if interrupt flood really happens by using
hardware clock after scheduler is stuck

Cc: Long Li <longli@microsoft.com>
Cc: Ingo Molnar <mingo@redhat.com>,
Cc: Peter Zijlstra <peterz@infradead.org>
Cc: Christoph Hellwig <hch@lst.de>
Cc: Keith Busch <keith.busch@intel.com>
Cc: Sagi Grimberg <sagi@grimberg.me>
Cc: John Garry <john.garry@huawei.com>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: Hannes Reinecke <hare@suse.com>
Signed-off-by: Ming Lei <ming.lei@redhat.com>
---
 drivers/base/cpu.c      |  23 ++++++
 include/linux/hardirq.h |   2 +
 kernel/softirq.c        | 161 ++++++++++++++++++++++++++++++++++++++++
 3 files changed, 186 insertions(+)

diff --git a/drivers/base/cpu.c b/drivers/base/cpu.c
index 6265871a4af2..34c253cde5f3 100644
--- a/drivers/base/cpu.c
+++ b/drivers/base/cpu.c
@@ -20,6 +20,7 @@
 #include <linux/tick.h>
 #include <linux/pm_qos.h>
 #include <linux/sched/isolation.h>
+#include <linux/hardirq.h>
 
 #include "base.h"
 
@@ -183,10 +184,31 @@ static struct attribute_group crash_note_cpu_attr_group = {
 };
 #endif
 
+static ssize_t show_irq_interval(struct device *dev,
+				 struct device_attribute *attr, char *buf)
+{
+	struct cpu *cpu = container_of(dev, struct cpu, dev);
+	ssize_t rc;
+	u64 avg = irq_get_avg_interval(cpu->dev.id);
+
+	rc = sprintf(buf, "%d: %d\n",  (int)(avg & 1), (int)(avg >> 1));
+	return rc;
+}
+
+static DEVICE_ATTR(irq_interval, 0400, show_irq_interval, NULL);
+static struct attribute *irq_interval_cpu_attrs[] = {
+	&dev_attr_irq_interval.attr,
+	NULL
+};
+static struct attribute_group irq_interval_cpu_attr_group = {
+	.attrs = irq_interval_cpu_attrs,
+};
+
 static const struct attribute_group *common_cpu_attr_groups[] = {
 #ifdef CONFIG_KEXEC
 	&crash_note_cpu_attr_group,
 #endif
+	&irq_interval_cpu_attr_group,
 	NULL
 };
 
@@ -194,6 +216,7 @@ static const struct attribute_group *hotplugable_cpu_attr_groups[] = {
 #ifdef CONFIG_KEXEC
 	&crash_note_cpu_attr_group,
 #endif
+	&irq_interval_cpu_attr_group,
 	NULL
 };
 
diff --git a/include/linux/hardirq.h b/include/linux/hardirq.h
index da0af631ded5..a0db287bb574 100644
--- a/include/linux/hardirq.h
+++ b/include/linux/hardirq.h
@@ -8,6 +8,8 @@
 #include <linux/vtime.h>
 #include <asm/hardirq.h>
 
+extern u64 irq_get_avg_interval(int);
+extern bool irq_is_flood(void);
 
 extern void synchronize_irq(unsigned int irq);
 extern bool synchronize_hardirq(unsigned int irq);
diff --git a/kernel/softirq.c b/kernel/softirq.c
index 0427a86743a4..f6e434ac4183 100644
--- a/kernel/softirq.c
+++ b/kernel/softirq.c
@@ -25,6 +25,8 @@
 #include <linux/smpboot.h>
 #include <linux/tick.h>
 #include <linux/irq.h>
+#include <linux/sched.h>
+#include <linux/sched/clock.h>
 
 #define CREATE_TRACE_POINTS
 #include <trace/events/irq.h>
@@ -52,6 +54,26 @@ DEFINE_PER_CPU_ALIGNED(irq_cpustat_t, irq_stat);
 EXPORT_PER_CPU_SYMBOL(irq_stat);
 #endif
 
+#define IRQ_INTERVAL_STAGE1_WEIGHT_BITS		ilog2(512)
+#define IRQ_INTERVAL_STAGE2_WEIGHT_BITS		ilog2(128)
+
+#define IRQ_INTERVAL_THRESHOLD_UNIT_NS	1000
+
+#define IRQ_INTERVAL_MIN_THRESHOLD_NS	IRQ_INTERVAL_THRESHOLD_UNIT_NS
+#define IRQ_INTERVAL_MAX_MIN_THRESHOLD_TIME_NS  4000000000
+
+struct irq_interval {
+	u64                     clock;
+	int			avg;
+	int			std_threshold:31;
+	int			stage:1;
+
+	u64			stage_start_clock;
+	unsigned		stage1_time;
+	unsigned		stage2_time;
+};
+DEFINE_PER_CPU(struct irq_interval, avg_irq_interval);
+
 static struct softirq_action softirq_vec[NR_SOFTIRQS] __cacheline_aligned_in_smp;
 
 DEFINE_PER_CPU(struct task_struct *, ksoftirqd);
@@ -339,6 +361,140 @@ asmlinkage __visible void do_softirq(void)
 	local_irq_restore(flags);
 }
 
+static inline void irq_interval_update_avg(struct irq_interval *inter,
+		u64 now, int weight_bits)
+{
+	inter->avg = inter->avg - ((inter->avg) >> weight_bits) +
+		((now - inter->clock) >> weight_bits);
+	if (unlikely(inter->avg < 0))
+		inter->avg = 0;
+}
+
+/*
+ * Keep the ratio of stage2 time to stage1 time between 1/2 and 1/8. If
+ * it is out of the range, adjust .std_threshold for maintaining the ratio.
+ */
+static inline void irq_interval_update_threshold(struct irq_interval *inter)
+{
+	if (inter->stage2_time * 2 > inter->stage1_time)
+		inter->std_threshold -= IRQ_INTERVAL_THRESHOLD_UNIT_NS;
+	if (inter->stage2_time * 8 < inter->stage1_time)
+		inter->std_threshold += IRQ_INTERVAL_THRESHOLD_UNIT_NS;
+
+	if (inter->std_threshold <= 0)
+		inter->std_threshold = IRQ_INTERVAL_THRESHOLD_UNIT_NS;
+
+	if (inter->std_threshold >= 32 * IRQ_INTERVAL_THRESHOLD_UNIT_NS)
+		inter->std_threshold = 32 * IRQ_INTERVAL_THRESHOLD_UNIT_NS;
+}
+
+/*
+ * If we stay at stage1 for too long with minimized threshold and low enough
+ * interrupt average interval, there may have risk to lock up CPU.
+ */
+static bool irq_interval_cpu_lockup_risk(struct irq_interval *inter, u64 now)
+{
+	if (inter->avg > inter->std_threshold)
+		return false;
+
+	if (inter->std_threshold != IRQ_INTERVAL_MIN_THRESHOLD_NS)
+		return false;
+
+	if (now - inter->stage_start_clock <=
+			IRQ_INTERVAL_MAX_MIN_THRESHOLD_TIME_NS)
+		return false;
+	return true;
+}
+
+/*
+ * Update average interrupt interval with the Exponential Weighted Moving
+ * Average(EWMA), and implement two-stage interrupt flood detection.
+ *
+ * Use scheduler's runqueue software clock at default for figuring
+ * interrupt interval for saving cost. When the interval becomes zero,
+ * it is reasonable to conclude scheduler's activity on this CPU has been
+ * stopped because of interrupt flood. Then switch to the 2nd stage
+ * detection in which clock is read from hardware, and the detection
+ * result can be more reliable.
+ */
+static void irq_interval_update(void)
+{
+	struct irq_interval *inter = raw_cpu_ptr(&avg_irq_interval);
+	u64 now;
+
+	if (likely(!inter->stage)) {
+		now = sched_local_rq_clock();
+		irq_interval_update_avg(inter, now,
+				IRQ_INTERVAL_STAGE1_WEIGHT_BITS);
+
+		if (unlikely(inter->avg < inter->std_threshold / 2 ||
+				irq_interval_cpu_lockup_risk(inter, now))) {
+			inter->stage = 1;
+			now = sched_clock_cpu(smp_processor_id());
+			inter->stage1_time = now - inter->stage_start_clock;
+			inter->stage_start_clock = now;
+
+			/*
+			 * reset as the mean of the min and the max value of
+			 * stage2's threshold
+			 */
+			inter->avg = inter->std_threshold +
+				(inter->std_threshold >> 2);
+		}
+	} else {
+		now = sched_clock_cpu(smp_processor_id());
+
+		irq_interval_update_avg(inter, now,
+				IRQ_INTERVAL_STAGE2_WEIGHT_BITS);
+
+		if (inter->avg > inter->std_threshold * 2) {
+			inter->stage = 0;
+			inter->stage2_time = now - inter->stage_start_clock;
+			inter->stage_start_clock = now;
+
+			irq_interval_update_threshold(inter);
+		}
+	}
+}
+
+static void irq_interval_update_clock(void)
+{
+	struct irq_interval *inter = raw_cpu_ptr(&avg_irq_interval);
+
+	if (likely(!inter->stage))
+		inter->clock = sched_local_rq_clock();
+	else
+		inter->clock = sched_clock_cpu(smp_processor_id());
+}
+
+u64 irq_get_avg_interval(int cpu)
+{
+	struct irq_interval *inter = per_cpu_ptr(&avg_irq_interval, cpu);
+
+	WARN_ON_ONCE(inter->stage > 1);
+	return (inter->avg << 1) | inter->stage;
+}
+EXPORT_SYMBOL_GPL(irq_get_avg_interval);
+
+bool irq_is_flood(void)
+{
+	struct irq_interval *inter = raw_cpu_ptr(&avg_irq_interval);
+
+	return inter->stage && inter->avg < inter->std_threshold;
+}
+EXPORT_SYMBOL_GPL(irq_is_flood);
+
+static void irq_interval_init(void)
+{
+	int i;
+
+	for_each_possible_cpu(i) {
+		struct irq_interval *inter = &per_cpu(avg_irq_interval, i);
+
+		inter->std_threshold = IRQ_INTERVAL_THRESHOLD_UNIT_NS * 2;
+	}
+}
+
 /*
  * Enter an interrupt context.
  */
@@ -356,6 +512,7 @@ void irq_enter(void)
 	}
 
 	__irq_enter();
+	irq_interval_update();
 }
 
 static inline void invoke_softirq(void)
@@ -408,6 +565,8 @@ void irq_exit(void)
 	lockdep_assert_irqs_disabled();
 #endif
 	account_irq_exit_time(current);
+	irq_interval_update_clock();
+
 	preempt_count_sub(HARDIRQ_OFFSET);
 	if (!in_interrupt() && local_softirq_pending())
 		invoke_softirq();
@@ -683,6 +842,8 @@ static __init int spawn_ksoftirqd(void)
 				  takeover_tasklets);
 	BUG_ON(smpboot_register_percpu_thread(&softirq_threads));
 
+	irq_interval_init();
+
 	return 0;
 }
 early_initcall(spawn_ksoftirqd);
-- 
2.20.1


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

* [RFC PATCH 3/3] blk-mq: complete request in rescuer process context in case of irq flood
  2019-12-18  7:19 [RFC PATCH 0/3] softirq/blk-mq: implement interrupt flood detection for avoiding cpu lockup Ming Lei
  2019-12-18  7:19 ` [RFC PATCH 1/3] sched/core: add API for exporting runqueue clock Ming Lei
  2019-12-18  7:19 ` [RFC PATCH 2/3] softirq: implement interrupt flood detection Ming Lei
@ 2019-12-18  7:19 ` Ming Lei
  2 siblings, 0 replies; 15+ messages in thread
From: Ming Lei @ 2019-12-18  7:19 UTC (permalink / raw)
  To: Thomas Gleixner, Jens Axboe
  Cc: linux-kernel, linux-block, Ming Lei, Long Li, Ingo Molnar,
	Peter Zijlstra, Keith Busch, Christoph Hellwig, Sagi Grimberg,
	John Garry, Hannes Reinecke

When irq flood is detected, complete requests in the percpu rescuer
context for avoiding lockup cpu.

IO interrupt flood might be triggered in the following situations:

1) the storage device is quicker to handle IO than single CPU core

2) N:1 queue mapping, single CPU core is saturated by handling IO interrupts
from multiple storage disks or multiple HBAs

Cc: Long Li <longli@microsoft.com>
Cc: Ingo Molnar <mingo@redhat.com>,
Cc: Peter Zijlstra <peterz@infradead.org>
Cc: Keith Busch <keith.busch@intel.com>
Cc: Christoph Hellwig <hch@lst.de>
Cc: Sagi Grimberg <sagi@grimberg.me>
Cc: John Garry <john.garry@huawei.com>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: Hannes Reinecke <hare@suse.com>
Signed-off-by: Ming Lei <ming.lei@redhat.com>
---
 block/blk-mq.c | 68 +++++++++++++++++++++++++++++++++++++++++++++++++-
 1 file changed, 67 insertions(+), 1 deletion(-)

diff --git a/block/blk-mq.c b/block/blk-mq.c
index 323c9cb28066..a7fe00f1a313 100644
--- a/block/blk-mq.c
+++ b/block/blk-mq.c
@@ -40,6 +40,14 @@
 #include "blk-mq-sched.h"
 #include "blk-rq-qos.h"
 
+struct blk_mq_comp_rescuer {
+	struct list_head head;
+	bool running;
+	struct work_struct work;
+};
+
+static DEFINE_PER_CPU(struct blk_mq_comp_rescuer, blk_mq_comp_rescuer);
+
 static void blk_mq_poll_stats_start(struct request_queue *q);
 static void blk_mq_poll_stats_fn(struct blk_stat_callback *cb);
 
@@ -624,6 +632,50 @@ static void hctx_lock(struct blk_mq_hw_ctx *hctx, int *srcu_idx)
 		*srcu_idx = srcu_read_lock(hctx->srcu);
 }
 
+static void blk_mq_complete_rq_in_rescuer(struct request *rq)
+{
+	struct blk_mq_comp_rescuer *rescuer;
+	unsigned long flags;
+
+	WARN_ON(!in_interrupt());
+
+	local_irq_save(flags);
+	rescuer = this_cpu_ptr(&blk_mq_comp_rescuer);
+	list_add_tail(&rq->queuelist, &rescuer->head);
+	if (!rescuer->running) {
+		rescuer->running = true;
+		kblockd_schedule_work(&rescuer->work);
+	}
+	local_irq_restore(flags);
+
+}
+
+static void blk_mq_complete_rescue_work(struct work_struct *work)
+{
+	struct blk_mq_comp_rescuer *rescuer =
+		container_of(work, struct blk_mq_comp_rescuer, work);
+	struct list_head local_list;
+
+	local_irq_disable();
+ run_again:
+	list_replace_init(&rescuer->head, &local_list);
+	local_irq_enable();
+
+	while (!list_empty(&local_list)) {
+		struct request *rq = list_entry(local_list.next,
+				struct request, queuelist);
+		list_del_init(&rq->queuelist);
+		__blk_mq_complete_request(rq);
+		cond_resched();
+	}
+
+	local_irq_disable();
+	if (!list_empty(&rescuer->head))
+		goto run_again;
+	rescuer->running = false;
+	local_irq_enable();
+}
+
 /**
  * blk_mq_complete_request - end I/O on a request
  * @rq:		the request being processed
@@ -636,7 +688,11 @@ bool blk_mq_complete_request(struct request *rq)
 {
 	if (unlikely(blk_should_fake_timeout(rq->q)))
 		return false;
-	__blk_mq_complete_request(rq);
+
+	if (likely(!irq_is_flood() || !in_interrupt()))
+		__blk_mq_complete_request(rq);
+	else
+		blk_mq_complete_rq_in_rescuer(rq);
 	return true;
 }
 EXPORT_SYMBOL(blk_mq_complete_request);
@@ -3525,6 +3581,16 @@ EXPORT_SYMBOL(blk_mq_rq_cpu);
 
 static int __init blk_mq_init(void)
 {
+	int i;
+
+	for_each_possible_cpu(i) {
+		struct blk_mq_comp_rescuer *rescuer =
+			&per_cpu(blk_mq_comp_rescuer, i);
+
+		INIT_LIST_HEAD(&rescuer->head);
+		INIT_WORK(&rescuer->work, blk_mq_complete_rescue_work);
+	}
+
 	cpuhp_setup_state_multi(CPUHP_BLK_MQ_DEAD, "block/mq:dead", NULL,
 				blk_mq_hctx_notify_dead);
 	return 0;
-- 
2.20.1


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

* Re: [RFC PATCH 1/3] sched/core: add API for exporting runqueue clock
  2019-12-18  7:19 ` [RFC PATCH 1/3] sched/core: add API for exporting runqueue clock Ming Lei
@ 2019-12-18  9:51   ` Peter Zijlstra
  2019-12-19  1:29     ` Ming Lei
  0 siblings, 1 reply; 15+ messages in thread
From: Peter Zijlstra @ 2019-12-18  9:51 UTC (permalink / raw)
  To: Ming Lei
  Cc: Thomas Gleixner, Jens Axboe, linux-kernel, linux-block, Long Li,
	Ingo Molnar, Keith Busch, Christoph Hellwig, Sagi Grimberg,
	John Garry, Hannes Reinecke

On Wed, Dec 18, 2019 at 03:19:40PM +0800, Ming Lei wrote:
> Scheduler runqueue maintains its own software clock that is periodically
> synchronised with hardware. Export this clock so that it can be used
> by interrupt flood detection for saving the cost of reading from hardware.

But you don't have much, if any, guarantees the thing gets updated.

> diff --git a/kernel/sched/core.c b/kernel/sched/core.c
> index 90e4b00ace89..03e2e3c36067 100644
> --- a/kernel/sched/core.c
> +++ b/kernel/sched/core.c
> @@ -219,6 +219,11 @@ void update_rq_clock(struct rq *rq)
>  	update_rq_clock_task(rq, delta);
>  }
>  
> +u64 sched_local_rq_clock(void)
> +{
> +	return this_rq()->clock;
> +}
> +EXPORT_SYMBOL_GPL(sched_local_rq_clock);

Also, more NAK, you're exporting a variant of __rq_clock_broken().

(which, now that I git-grep for it, has become unused, good!)

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

* Re: [RFC PATCH 2/3] softirq: implement interrupt flood detection
  2019-12-18  7:19 ` [RFC PATCH 2/3] softirq: implement interrupt flood detection Ming Lei
@ 2019-12-18 10:49   ` Peter Zijlstra
  2019-12-18 12:29     ` Peter Zijlstra
  2019-12-19  1:59     ` Ming Lei
  0 siblings, 2 replies; 15+ messages in thread
From: Peter Zijlstra @ 2019-12-18 10:49 UTC (permalink / raw)
  To: Ming Lei
  Cc: Thomas Gleixner, Jens Axboe, linux-kernel, linux-block, Long Li,
	Ingo Molnar, Christoph Hellwig, Keith Busch, Sagi Grimberg,
	John Garry, Hannes Reinecke

On Wed, Dec 18, 2019 at 03:19:41PM +0800, Ming Lei wrote:

> diff --git a/kernel/softirq.c b/kernel/softirq.c
> index 0427a86743a4..f6e434ac4183 100644
> --- a/kernel/softirq.c
> +++ b/kernel/softirq.c
> @@ -25,6 +25,8 @@
>  #include <linux/smpboot.h>
>  #include <linux/tick.h>
>  #include <linux/irq.h>
> +#include <linux/sched.h>
> +#include <linux/sched/clock.h>
>  
>  #define CREATE_TRACE_POINTS
>  #include <trace/events/irq.h>
> @@ -52,6 +54,26 @@ DEFINE_PER_CPU_ALIGNED(irq_cpustat_t, irq_stat);
>  EXPORT_PER_CPU_SYMBOL(irq_stat);
>  #endif
>  
> +#define IRQ_INTERVAL_STAGE1_WEIGHT_BITS		ilog2(512)
> +#define IRQ_INTERVAL_STAGE2_WEIGHT_BITS		ilog2(128)

That must be the most difficult way of writing 9 and 7 resp.

> +#define IRQ_INTERVAL_THRESHOLD_UNIT_NS	1000
> +
> +#define IRQ_INTERVAL_MIN_THRESHOLD_NS	IRQ_INTERVAL_THRESHOLD_UNIT_NS
> +#define IRQ_INTERVAL_MAX_MIN_THRESHOLD_TIME_NS  4000000000

(seriously a name with MAX_MIN in it ?!?)

That's unreadable, we have (4*NSEC_PER_SEC) for that (if I counted the
0s correctly)

These are all a bunch of magic value, any justification for them? Will
they always work?

> +
> +struct irq_interval {
> +	u64                     clock;
> +	int			avg;
> +	int			std_threshold:31;

I know of a standard deviation, but what is a standard threshold?

> +	int			stage:1;

signed single bit.. there's people that object to that. They figure just
a sign bit isn't much useful.

> +
> +	u64			stage_start_clock;
> +	unsigned		stage1_time;
> +	unsigned		stage2_time;
> +};
> +DEFINE_PER_CPU(struct irq_interval, avg_irq_interval);
> +
>  static struct softirq_action softirq_vec[NR_SOFTIRQS] __cacheline_aligned_in_smp;
>  
>  DEFINE_PER_CPU(struct task_struct *, ksoftirqd);
> @@ -339,6 +361,140 @@ asmlinkage __visible void do_softirq(void)
>  	local_irq_restore(flags);
>  }
>  
> +static inline void irq_interval_update_avg(struct irq_interval *inter,
> +		u64 now, int weight_bits)
> +{
> +	inter->avg = inter->avg - ((inter->avg) >> weight_bits) +
> +		((now - inter->clock) >> weight_bits);

Did you perhaps want to write something like:

	s64 sample = now - inter->clock;

	inter->avg += (sample - inter->avg) >> weight_bits;

Which is a recognisable form.

It also shows the obvious overflow when sample is large (interrupts
didn't happen for a while). You'll want to clamp @sample to some max.

> +	if (unlikely(inter->avg < 0))
> +		inter->avg = 0;

And since inter->avg must never be <0, wth are you using a signed
bitfield? This generates shit code. Use an on-stack temporary if
anything:

	int avg = inter->avg;

	avg += (sample - avg) >> bits;
	if (avg < 0)
		avg = 0;

	inter->avg = avg;

and presto! no signed bitfields required.

> +}
> +
> +/*
> + * Keep the ratio of stage2 time to stage1 time between 1/2 and 1/8. If
> + * it is out of the range, adjust .std_threshold for maintaining the ratio.

it is either @std_threshold or @irq_interval::std_threshold

> + */
> +static inline void irq_interval_update_threshold(struct irq_interval *inter)
> +{
> +	if (inter->stage2_time * 2 > inter->stage1_time)
> +		inter->std_threshold -= IRQ_INTERVAL_THRESHOLD_UNIT_NS;
> +	if (inter->stage2_time * 8 < inter->stage1_time)
> +		inter->std_threshold += IRQ_INTERVAL_THRESHOLD_UNIT_NS;

I suppose that will eventually converge.

> +	if (inter->std_threshold <= 0)
> +		inter->std_threshold = IRQ_INTERVAL_THRESHOLD_UNIT_NS;

I think you'll find you actually meant to write:

	if (inter->std_threshold < IRQ_INTERVAL_THRESHOLD_UNIT_NS)


> +	if (inter->std_threshold >= 32 * IRQ_INTERVAL_THRESHOLD_UNIT_NS)
> +		inter->std_threshold = 32 * IRQ_INTERVAL_THRESHOLD_UNIT_NS;

We actually have a macro for this:

	inter->std_threshold = clamp(inter->std_threshold,
				     IRQ_INTERVAL_THRESHOLD_UNIT_NS,
				     32 * IRQ_INTERVAL_THRESHOLD_UNIT_NS);

> +}
> +
> +/*
> + * If we stay at stage1 for too long with minimized threshold and low enough
> + * interrupt average interval, there may have risk to lock up CPU.

It's not locked up...

> + */
> +static bool irq_interval_cpu_lockup_risk(struct irq_interval *inter, u64 now)
> +{
> +	if (inter->avg > inter->std_threshold)
> +		return false;
> +
> +	if (inter->std_threshold != IRQ_INTERVAL_MIN_THRESHOLD_NS)
> +		return false;
> +
> +	if (now - inter->stage_start_clock <=
> +			IRQ_INTERVAL_MAX_MIN_THRESHOLD_TIME_NS)
> +		return false;
> +	return true;
> +}
> +
> +/*
> + * Update average interrupt interval with the Exponential Weighted Moving
> + * Average(EWMA), and implement two-stage interrupt flood detection.
> + *
> + * Use scheduler's runqueue software clock at default for figuring
> + * interrupt interval for saving cost. When the interval becomes zero,
> + * it is reasonable to conclude scheduler's activity on this CPU has been
> + * stopped because of interrupt flood. Then switch to the 2nd stage
> + * detection in which clock is read from hardware, and the detection
> + * result can be more reliable.
> + */
> +static void irq_interval_update(void)
> +{
> +	struct irq_interval *inter = raw_cpu_ptr(&avg_irq_interval);

raw_cpu_ptr is wrong, this wants to be this_cpu_ptr()

> +	u64 now;
> +
> +	if (likely(!inter->stage)) {
> +		now = sched_local_rq_clock();
> +		irq_interval_update_avg(inter, now,
> +				IRQ_INTERVAL_STAGE1_WEIGHT_BITS);
> +
> +		if (unlikely(inter->avg < inter->std_threshold / 2 ||
> +				irq_interval_cpu_lockup_risk(inter, now))) {
> +			inter->stage = 1;
> +			now = sched_clock_cpu(smp_processor_id());
> +			inter->stage1_time = now - inter->stage_start_clock;
> +			inter->stage_start_clock = now;
> +
> +			/*
> +			 * reset as the mean of the min and the max value of
> +			 * stage2's threshold
> +			 */
> +			inter->avg = inter->std_threshold +
> +				(inter->std_threshold >> 2);
> +		}
> +	} else {
> +		now = sched_clock_cpu(smp_processor_id());
> +
> +		irq_interval_update_avg(inter, now,
> +				IRQ_INTERVAL_STAGE2_WEIGHT_BITS);
> +
> +		if (inter->avg > inter->std_threshold * 2) {
> +			inter->stage = 0;
> +			inter->stage2_time = now - inter->stage_start_clock;
> +			inter->stage_start_clock = now;
> +
> +			irq_interval_update_threshold(inter);
> +		}
> +	}
> +}

AFAICT the only reason for much of this complexity is so that you can
use this sched_local_rq_clock() thing, right? Once that reaches a
threshold, you go use the more accurate sched_clock_cpu() and once that
tickles the threshold you call it golden and raise hell.

So pray tell, why did you not integrate this with IRQ_TIME_ACCOUNTING ?
That already takes a timestamp and does most of what you need.

> @@ -356,6 +512,7 @@ void irq_enter(void)
>  	}
>  
>  	__irq_enter();
> +	irq_interval_update();
>  }

Arggh.. you're going to make every single interrupt take at least 2
extra cache misses for this gunk?!?

And it lumps all interrupts on a single heap, and doesn't do any of the
otherwise useful things we've been wanting to have IRQ timings for :/


_If_ you want to do something like this, do it like the below. That only
adds a few instruction to irq_exit() and only touches a cacheline that's
already touched.

It computes both the avg duration and the avg inter-arrival-time of
hardirqs. Things get critical when:

	inter-arrival-avg < 2*duration-avg

or something like that.

---
diff --git a/kernel/sched/cputime.c b/kernel/sched/cputime.c
index d43318a489f2..6f5ef70b5a1d 100644
--- a/kernel/sched/cputime.c
+++ b/kernel/sched/cputime.c
@@ -50,7 +50,7 @@ static void irqtime_account_delta(struct irqtime *irqtime, u64 delta,
 void irqtime_account_irq(struct task_struct *curr)
 {
 	struct irqtime *irqtime = this_cpu_ptr(&cpu_irqtime);
-	s64 delta;
+	s64 delta, iat;
 	int cpu;
 
 	if (!sched_clock_irqtime)
@@ -58,7 +58,6 @@ void irqtime_account_irq(struct task_struct *curr)
 
 	cpu = smp_processor_id();
 	delta = sched_clock_cpu(cpu) - irqtime->irq_start_time;
-	irqtime->irq_start_time += delta;
 
 	/*
 	 * We do not account for softirq time from ksoftirqd here.
@@ -66,10 +65,21 @@ void irqtime_account_irq(struct task_struct *curr)
 	 * in that case, so as not to confuse scheduler with a special task
 	 * that do not consume any time, but still wants to run.
 	 */
-	if (hardirq_count())
+	if (hardirq_count()) {
 		irqtime_account_delta(irqtime, delta, CPUTIME_IRQ);
-	else if (in_serving_softirq() && curr != this_cpu_ksoftirqd())
+
+		/* this is irq_exit(), delta is the duration of the hardirq */
+		irqtime->duration_avg += (delta - irqtime->duration_avg) >> 7;
+
+		/* compute the inter arrival time using the previous arrival_time */
+		iat = irqtime->irq_start_time - irqtime->irq_arrival_time;
+		irqtime->irq_arrival_time += iat;
+		irqtime->irq_inter_arrival_avg += (iat - irqtime->inter_arrival_avg) >> 7;
+
+	} else if (in_serving_softirq() && curr != this_cpu_ksoftirqd())
 		irqtime_account_delta(irqtime, delta, CPUTIME_SOFTIRQ);
+
+	irqtime->irq_start_time += delta;
 }
 EXPORT_SYMBOL_GPL(irqtime_account_irq);
 
diff --git a/kernel/sched/sched.h b/kernel/sched/sched.h
index 280a3c735935..cab07e5a6c11 100644
--- a/kernel/sched/sched.h
+++ b/kernel/sched/sched.h
@@ -2236,6 +2236,9 @@ struct irqtime {
 	u64			total;
 	u64			tick_delta;
 	u64			irq_start_time;
+	u64			irq_duration_avg;
+	u64			irq_arrival_time;
+	u64			irq_inter_arrival_avg;
 	struct u64_stats_sync	sync;
 };
 

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

* Re: [RFC PATCH 2/3] softirq: implement interrupt flood detection
  2019-12-18 10:49   ` Peter Zijlstra
@ 2019-12-18 12:29     ` Peter Zijlstra
  2019-12-19  1:59     ` Ming Lei
  1 sibling, 0 replies; 15+ messages in thread
From: Peter Zijlstra @ 2019-12-18 12:29 UTC (permalink / raw)
  To: Ming Lei
  Cc: Thomas Gleixner, Jens Axboe, linux-kernel, linux-block, Long Li,
	Ingo Molnar, Christoph Hellwig, Keith Busch, Sagi Grimberg,
	John Garry, Hannes Reinecke

On Wed, Dec 18, 2019 at 11:49:41AM +0100, Peter Zijlstra wrote:
> 
> _If_ you want to do something like this, do it like the below. That only
> adds a few instruction to irq_exit() and only touches a cacheline that's
> already touched.
> 
> It computes both the avg duration and the avg inter-arrival-time of
> hardirqs. Things get critical when:
> 
> 	inter-arrival-avg < 2*duration-avg
> 
> or something like that.

Better yet, try something like:

bool cpu_irq_heavy(int cpu)
{
	return cpu_util_irq(cpu_rq(cpu)) >= arch_scale_cpu_capacity(cpu);
}



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

* Re: [RFC PATCH 1/3] sched/core: add API for exporting runqueue clock
  2019-12-18  9:51   ` Peter Zijlstra
@ 2019-12-19  1:29     ` Ming Lei
  2019-12-19  9:20       ` Peter Zijlstra
  0 siblings, 1 reply; 15+ messages in thread
From: Ming Lei @ 2019-12-19  1:29 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Thomas Gleixner, Jens Axboe, linux-kernel, linux-block, Long Li,
	Ingo Molnar, Keith Busch, Christoph Hellwig, Sagi Grimberg,
	John Garry, Hannes Reinecke

On Wed, Dec 18, 2019 at 10:51:01AM +0100, Peter Zijlstra wrote:
> On Wed, Dec 18, 2019 at 03:19:40PM +0800, Ming Lei wrote:
> > Scheduler runqueue maintains its own software clock that is periodically
> > synchronised with hardware. Export this clock so that it can be used
> > by interrupt flood detection for saving the cost of reading from hardware.
> 
> But you don't have much, if any, guarantees the thing gets updated.

Any software clock won't be guaranteed to be updated in time, however,
they are still useful.

Thanks,
Ming


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

* Re: [RFC PATCH 2/3] softirq: implement interrupt flood detection
  2019-12-18 10:49   ` Peter Zijlstra
  2019-12-18 12:29     ` Peter Zijlstra
@ 2019-12-19  1:59     ` Ming Lei
  2019-12-19  9:23       ` Peter Zijlstra
  1 sibling, 1 reply; 15+ messages in thread
From: Ming Lei @ 2019-12-19  1:59 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Thomas Gleixner, Jens Axboe, linux-kernel, linux-block, Long Li,
	Ingo Molnar, Christoph Hellwig, Keith Busch, Sagi Grimberg,
	John Garry, Hannes Reinecke

Hi Peter,

Thanks for your review!

On Wed, Dec 18, 2019 at 11:49:41AM +0100, Peter Zijlstra wrote:
> On Wed, Dec 18, 2019 at 03:19:41PM +0800, Ming Lei wrote:
> 
> > diff --git a/kernel/softirq.c b/kernel/softirq.c
> > index 0427a86743a4..f6e434ac4183 100644
> > --- a/kernel/softirq.c
> > +++ b/kernel/softirq.c
> > @@ -25,6 +25,8 @@
> >  #include <linux/smpboot.h>
> >  #include <linux/tick.h>
> >  #include <linux/irq.h>
> > +#include <linux/sched.h>
> > +#include <linux/sched/clock.h>
> >  
> >  #define CREATE_TRACE_POINTS
> >  #include <trace/events/irq.h>
> > @@ -52,6 +54,26 @@ DEFINE_PER_CPU_ALIGNED(irq_cpustat_t, irq_stat);
> >  EXPORT_PER_CPU_SYMBOL(irq_stat);
> >  #endif
> >  
> > +#define IRQ_INTERVAL_STAGE1_WEIGHT_BITS		ilog2(512)
> > +#define IRQ_INTERVAL_STAGE2_WEIGHT_BITS		ilog2(128)
> 
> That must be the most difficult way of writing 9 and 7 resp.
> 
> > +#define IRQ_INTERVAL_THRESHOLD_UNIT_NS	1000
> > +
> > +#define IRQ_INTERVAL_MIN_THRESHOLD_NS	IRQ_INTERVAL_THRESHOLD_UNIT_NS
> > +#define IRQ_INTERVAL_MAX_MIN_THRESHOLD_TIME_NS  4000000000
> 
> (seriously a name with MAX_MIN in it ?!?)
> 
> That's unreadable, we have (4*NSEC_PER_SEC) for that (if I counted the
> 0s correctly)
> 
> These are all a bunch of magic value, any justification for them? Will
> they always work?

The two weight constant just decides rate of convergence.

IRQ_INTERVAL_THRESHOLD_UNIT_NS is the unit of updating the threshold.

IRQ_INTERVAL_MIN_THRESHOLD_NS is the minimized allowed threshold.

IRQ_INTERVAL_MAX_MIN_THRESHOLD_TIME_NS could be the only one magic
value, which is for avoiding to use the smallest threshold for too
long.

> 
> > +
> > +struct irq_interval {
> > +	u64                     clock;
> > +	int			avg;
> > +	int			std_threshold:31;
> 
> I know of a standard deviation, but what is a standard threshold?

It is just one threshold, will rename it.

> 
> > +	int			stage:1;
> 
> signed single bit.. there's people that object to that. They figure just
> a sign bit isn't much useful.

OK, the stage has just two value, zero or non-zero. Maybe it can be
changed to use two bits.

> 
> > +
> > +	u64			stage_start_clock;
> > +	unsigned		stage1_time;
> > +	unsigned		stage2_time;
> > +};
> > +DEFINE_PER_CPU(struct irq_interval, avg_irq_interval);
> > +
> >  static struct softirq_action softirq_vec[NR_SOFTIRQS] __cacheline_aligned_in_smp;
> >  
> >  DEFINE_PER_CPU(struct task_struct *, ksoftirqd);
> > @@ -339,6 +361,140 @@ asmlinkage __visible void do_softirq(void)
> >  	local_irq_restore(flags);
> >  }
> >  
> > +static inline void irq_interval_update_avg(struct irq_interval *inter,
> > +		u64 now, int weight_bits)
> > +{
> > +	inter->avg = inter->avg - ((inter->avg) >> weight_bits) +
> > +		((now - inter->clock) >> weight_bits);
> 
> Did you perhaps want to write something like:
> 
> 	s64 sample = now - inter->clock;
> 
> 	inter->avg += (sample - inter->avg) >> weight_bits;
> 
> Which is a recognisable form.

Yeah, that is it.

> 
> It also shows the obvious overflow when sample is large (interrupts
> didn't happen for a while). You'll want to clamp @sample to some max.

Looks not necessary, even it is overflowed, it is exponential decay,
so the average will become to normal level very quick. Given it is run
in fast path, I'd suggest to not introduce the unnecessary clamp.

> 
> > +	if (unlikely(inter->avg < 0))
> > +		inter->avg = 0;
> 
> And since inter->avg must never be <0, wth are you using a signed
> bitfield? This generates shit code. Use an on-stack temporary if
> anything:
> 
> 	int avg = inter->avg;
> 
> 	avg += (sample - avg) >> bits;
> 	if (avg < 0)
> 		avg = 0;
> 
> 	inter->avg = avg;
> 
> and presto! no signed bitfields required.

Fine.

> 
> > +}
> > +
> > +/*
> > + * Keep the ratio of stage2 time to stage1 time between 1/2 and 1/8. If
> > + * it is out of the range, adjust .std_threshold for maintaining the ratio.
> 
> it is either @std_threshold or @irq_interval::std_threshold
> 
> > + */
> > +static inline void irq_interval_update_threshold(struct irq_interval *inter)
> > +{
> > +	if (inter->stage2_time * 2 > inter->stage1_time)
> > +		inter->std_threshold -= IRQ_INTERVAL_THRESHOLD_UNIT_NS;
> > +	if (inter->stage2_time * 8 < inter->stage1_time)
> > +		inter->std_threshold += IRQ_INTERVAL_THRESHOLD_UNIT_NS;
> 
> I suppose that will eventually converge.

Right.

> 
> > +	if (inter->std_threshold <= 0)
> > +		inter->std_threshold = IRQ_INTERVAL_THRESHOLD_UNIT_NS;
> 
> I think you'll find you actually meant to write:
> 
> 	if (inter->std_threshold < IRQ_INTERVAL_THRESHOLD_UNIT_NS)
> 
> 
> > +	if (inter->std_threshold >= 32 * IRQ_INTERVAL_THRESHOLD_UNIT_NS)
> > +		inter->std_threshold = 32 * IRQ_INTERVAL_THRESHOLD_UNIT_NS;
> 
> We actually have a macro for this:
> 
> 	inter->std_threshold = clamp(inter->std_threshold,
> 				     IRQ_INTERVAL_THRESHOLD_UNIT_NS,
> 				     32 * IRQ_INTERVAL_THRESHOLD_UNIT_NS);

OK.

> 
> > +}
> > +
> > +/*
> > + * If we stay at stage1 for too long with minimized threshold and low enough
> > + * interrupt average interval, there may have risk to lock up CPU.
> 
> It's not locked up...

If the interrupt flood isn't detected, the lock up may happen. Given the
stage1 uses runqueue's software clock which may not be accurate enough, we
need this way to avoid the risk.

> 
> > + */
> > +static bool irq_interval_cpu_lockup_risk(struct irq_interval *inter, u64 now)
> > +{
> > +	if (inter->avg > inter->std_threshold)
> > +		return false;
> > +
> > +	if (inter->std_threshold != IRQ_INTERVAL_MIN_THRESHOLD_NS)
> > +		return false;
> > +
> > +	if (now - inter->stage_start_clock <=
> > +			IRQ_INTERVAL_MAX_MIN_THRESHOLD_TIME_NS)
> > +		return false;
> > +	return true;
> > +}
> > +
> > +/*
> > + * Update average interrupt interval with the Exponential Weighted Moving
> > + * Average(EWMA), and implement two-stage interrupt flood detection.
> > + *
> > + * Use scheduler's runqueue software clock at default for figuring
> > + * interrupt interval for saving cost. When the interval becomes zero,
> > + * it is reasonable to conclude scheduler's activity on this CPU has been
> > + * stopped because of interrupt flood. Then switch to the 2nd stage
> > + * detection in which clock is read from hardware, and the detection
> > + * result can be more reliable.
> > + */
> > +static void irq_interval_update(void)
> > +{
> > +	struct irq_interval *inter = raw_cpu_ptr(&avg_irq_interval);
> 
> raw_cpu_ptr is wrong, this wants to be this_cpu_ptr()

OK.

> 
> > +	u64 now;
> > +
> > +	if (likely(!inter->stage)) {
> > +		now = sched_local_rq_clock();
> > +		irq_interval_update_avg(inter, now,
> > +				IRQ_INTERVAL_STAGE1_WEIGHT_BITS);
> > +
> > +		if (unlikely(inter->avg < inter->std_threshold / 2 ||
> > +				irq_interval_cpu_lockup_risk(inter, now))) {
> > +			inter->stage = 1;
> > +			now = sched_clock_cpu(smp_processor_id());
> > +			inter->stage1_time = now - inter->stage_start_clock;
> > +			inter->stage_start_clock = now;
> > +
> > +			/*
> > +			 * reset as the mean of the min and the max value of
> > +			 * stage2's threshold
> > +			 */
> > +			inter->avg = inter->std_threshold +
> > +				(inter->std_threshold >> 2);
> > +		}
> > +	} else {
> > +		now = sched_clock_cpu(smp_processor_id());
> > +
> > +		irq_interval_update_avg(inter, now,
> > +				IRQ_INTERVAL_STAGE2_WEIGHT_BITS);
> > +
> > +		if (inter->avg > inter->std_threshold * 2) {
> > +			inter->stage = 0;
> > +			inter->stage2_time = now - inter->stage_start_clock;
> > +			inter->stage_start_clock = now;
> > +
> > +			irq_interval_update_threshold(inter);
> > +		}
> > +	}
> > +}
> 
> AFAICT the only reason for much of this complexity is so that you can
> use this sched_local_rq_clock() thing, right? Once that reaches a
> threshold, you go use the more accurate sched_clock_cpu() and once that
> tickles the threshold you call it golden and raise hell.

Right.

> 
> So pray tell, why did you not integrate this with IRQ_TIME_ACCOUNTING ?
> That already takes a timestamp and does most of what you need.

Yeah, that was the 1st approach I thought of, but IRQ_TIME_ACCOUNTING
may be disabled, and enabling it may cause observable effect on IO
performance.

> 
> > @@ -356,6 +512,7 @@ void irq_enter(void)
> >  	}
> >  
> >  	__irq_enter();
> > +	irq_interval_update();
> >  }
> 
> Arggh.. you're going to make every single interrupt take at least 2
> extra cache misses for this gunk?!?

Could you explain it a bit why two cache misses are involved?

I understand at most one miss is caused, which should only happen in
irq_interval_update(), and what is the other one?

> 
> And it lumps all interrupts on a single heap, and doesn't do any of the
> otherwise useful things we've been wanting to have IRQ timings for :/
> 
> 
> _If_ you want to do something like this, do it like the below. That only
> adds a few instruction to irq_exit() and only touches a cacheline that's
> already touched.
> 
> It computes both the avg duration and the avg inter-arrival-time of
> hardirqs. Things get critical when:
> 
> 	inter-arrival-avg < 2*duration-avg
> 
> or something like that.
> 
> ---
> diff --git a/kernel/sched/cputime.c b/kernel/sched/cputime.c
> index d43318a489f2..6f5ef70b5a1d 100644
> --- a/kernel/sched/cputime.c
> +++ b/kernel/sched/cputime.c
> @@ -50,7 +50,7 @@ static void irqtime_account_delta(struct irqtime *irqtime, u64 delta,
>  void irqtime_account_irq(struct task_struct *curr)
>  {
>  	struct irqtime *irqtime = this_cpu_ptr(&cpu_irqtime);
> -	s64 delta;
> +	s64 delta, iat;
>  	int cpu;
>  
>  	if (!sched_clock_irqtime)
> @@ -58,7 +58,6 @@ void irqtime_account_irq(struct task_struct *curr)
>  
>  	cpu = smp_processor_id();
>  	delta = sched_clock_cpu(cpu) - irqtime->irq_start_time;
> -	irqtime->irq_start_time += delta;
>  
>  	/*
>  	 * We do not account for softirq time from ksoftirqd here.
> @@ -66,10 +65,21 @@ void irqtime_account_irq(struct task_struct *curr)
>  	 * in that case, so as not to confuse scheduler with a special task
>  	 * that do not consume any time, but still wants to run.
>  	 */
> -	if (hardirq_count())
> +	if (hardirq_count()) {
>  		irqtime_account_delta(irqtime, delta, CPUTIME_IRQ);
> -	else if (in_serving_softirq() && curr != this_cpu_ksoftirqd())
> +
> +		/* this is irq_exit(), delta is the duration of the hardirq */
> +		irqtime->duration_avg += (delta - irqtime->duration_avg) >> 7;
> +
> +		/* compute the inter arrival time using the previous arrival_time */
> +		iat = irqtime->irq_start_time - irqtime->irq_arrival_time;
> +		irqtime->irq_arrival_time += iat;
> +		irqtime->irq_inter_arrival_avg += (iat - irqtime->inter_arrival_avg) >> 7;
> +
> +	} else if (in_serving_softirq() && curr != this_cpu_ksoftirqd())
>  		irqtime_account_delta(irqtime, delta, CPUTIME_SOFTIRQ);
> +
> +	irqtime->irq_start_time += delta;
>  }
>  EXPORT_SYMBOL_GPL(irqtime_account_irq);
>  
> diff --git a/kernel/sched/sched.h b/kernel/sched/sched.h
> index 280a3c735935..cab07e5a6c11 100644
> --- a/kernel/sched/sched.h
> +++ b/kernel/sched/sched.h
> @@ -2236,6 +2236,9 @@ struct irqtime {
>  	u64			total;
>  	u64			tick_delta;
>  	u64			irq_start_time;
> +	u64			irq_duration_avg;
> +	u64			irq_arrival_time;
> +	u64			irq_inter_arrival_avg;
>  	struct u64_stats_sync	sync;
>  };

I think we can do that in case of IRQ_TIME_ACCOUNTING, but the
option may not be enabled.

How about only using the rq software clock in case that
IRQ_TIME_ACCOUNTING is disabled?  Meantime replies irqtime
for computing the interval average when the option is enabled.


Thanks,
Ming


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

* Re: [RFC PATCH 1/3] sched/core: add API for exporting runqueue clock
  2019-12-19  1:29     ` Ming Lei
@ 2019-12-19  9:20       ` Peter Zijlstra
  0 siblings, 0 replies; 15+ messages in thread
From: Peter Zijlstra @ 2019-12-19  9:20 UTC (permalink / raw)
  To: Ming Lei
  Cc: Thomas Gleixner, Jens Axboe, linux-kernel, linux-block, Long Li,
	Ingo Molnar, Keith Busch, Christoph Hellwig, Sagi Grimberg,
	John Garry, Hannes Reinecke

On Thu, Dec 19, 2019 at 09:29:14AM +0800, Ming Lei wrote:
> On Wed, Dec 18, 2019 at 10:51:01AM +0100, Peter Zijlstra wrote:
> > On Wed, Dec 18, 2019 at 03:19:40PM +0800, Ming Lei wrote:
> > > Scheduler runqueue maintains its own software clock that is periodically
> > > synchronised with hardware. Export this clock so that it can be used
> > > by interrupt flood detection for saving the cost of reading from hardware.
> > 
> > But you don't have much, if any, guarantees the thing gets updated.
> 
> Any software clock won't be guaranteed to be updated in time, however,
> they are still useful.

It still is broken, and I really don't want to expose this for whatever
reason.

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

* Re: [RFC PATCH 2/3] softirq: implement interrupt flood detection
  2019-12-19  1:59     ` Ming Lei
@ 2019-12-19  9:23       ` Peter Zijlstra
  2019-12-19  9:52         ` Ming Lei
  2019-12-19 10:43         ` Daniel Wagner
  0 siblings, 2 replies; 15+ messages in thread
From: Peter Zijlstra @ 2019-12-19  9:23 UTC (permalink / raw)
  To: Ming Lei
  Cc: Thomas Gleixner, Jens Axboe, linux-kernel, linux-block, Long Li,
	Ingo Molnar, Christoph Hellwig, Keith Busch, Sagi Grimberg,
	John Garry, Hannes Reinecke

On Thu, Dec 19, 2019 at 09:59:48AM +0800, Ming Lei wrote:
> > So pray tell, why did you not integrate this with IRQ_TIME_ACCOUNTING ?
> > That already takes a timestamp and does most of what you need.
> 
> Yeah, that was the 1st approach I thought of, but IRQ_TIME_ACCOUNTING
> may be disabled, and enabling it may cause observable effect on IO
> performance.

Is that an actual concern, are people disabling it?

> > > @@ -356,6 +512,7 @@ void irq_enter(void)
> > >  	}
> > >  
> > >  	__irq_enter();
> > > +	irq_interval_update();
> > >  }
> > 
> > Arggh.. you're going to make every single interrupt take at least 2
> > extra cache misses for this gunk?!?
> 
> Could you explain it a bit why two cache misses are involved?
> 
> I understand at most one miss is caused, which should only happen in
> irq_interval_update(), and what is the other one?

The rq clock thing IIRC.


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

* Re: [RFC PATCH 2/3] softirq: implement interrupt flood detection
  2019-12-19  9:23       ` Peter Zijlstra
@ 2019-12-19  9:52         ` Ming Lei
  2019-12-19 10:43         ` Daniel Wagner
  1 sibling, 0 replies; 15+ messages in thread
From: Ming Lei @ 2019-12-19  9:52 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Thomas Gleixner, Jens Axboe, linux-kernel, linux-block, Long Li,
	Ingo Molnar, Christoph Hellwig, Keith Busch, Sagi Grimberg,
	John Garry, Hannes Reinecke

On Thu, Dec 19, 2019 at 10:23:19AM +0100, Peter Zijlstra wrote:
> On Thu, Dec 19, 2019 at 09:59:48AM +0800, Ming Lei wrote:
> > > So pray tell, why did you not integrate this with IRQ_TIME_ACCOUNTING ?
> > > That already takes a timestamp and does most of what you need.
> > 
> > Yeah, that was the 1st approach I thought of, but IRQ_TIME_ACCOUNTING
> > may be disabled, and enabling it may cause observable effect on IO
> > performance.
> 
> Is that an actual concern, are people disabling it?

For example, it is only enabled for x86 on RHEL8.

And the interrupt flood issue is easier to trigger on other ARCH,
for example, John reported the issue on arm64:

https://lore.kernel.org/lkml/a7ef3810-31af-013a-6d18-ceb6154aa2ef@huawei.com/

> 
> > > > @@ -356,6 +512,7 @@ void irq_enter(void)
> > > >  	}
> > > >  
> > > >  	__irq_enter();
> > > > +	irq_interval_update();
> > > >  }
> > > 
> > > Arggh.. you're going to make every single interrupt take at least 2
> > > extra cache misses for this gunk?!?
> > 
> > Could you explain it a bit why two cache misses are involved?
> > 
> > I understand at most one miss is caused, which should only happen in
> > irq_interval_update(), and what is the other one?
> 
> The rq clock thing IIRC.

OK.

But task is often waken up by interrupt event, I guess the rq clock
thing should be fine.


Thanks, 
Ming


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

* Re: [RFC PATCH 2/3] softirq: implement interrupt flood detection
  2019-12-19  9:23       ` Peter Zijlstra
  2019-12-19  9:52         ` Ming Lei
@ 2019-12-19 10:43         ` Daniel Wagner
  2019-12-31  3:48           ` Ming Lei
  1 sibling, 1 reply; 15+ messages in thread
From: Daniel Wagner @ 2019-12-19 10:43 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Ming Lei, Thomas Gleixner, Jens Axboe, linux-kernel, linux-block,
	Long Li, Ingo Molnar, Christoph Hellwig, Keith Busch,
	Sagi Grimberg, John Garry, Hannes Reinecke

Hi,

On Thu, Dec 19, 2019 at 10:23:19AM +0100, Peter Zijlstra wrote:
> On Thu, Dec 19, 2019 at 09:59:48AM +0800, Ming Lei wrote:
> > > So pray tell, why did you not integrate this with IRQ_TIME_ACCOUNTING ?
> > > That already takes a timestamp and does most of what you need.
> > 
> > Yeah, that was the 1st approach I thought of, but IRQ_TIME_ACCOUNTING
> > may be disabled, and enabling it may cause observable effect on IO
> > performance.
> 
> Is that an actual concern, are people disabling it?

In SLE and openSUSE kernels it is disabled for x86_64 at this
point. And if I am not completely misstaken only x86_64 supports it at
this point. I was looking at enable_sched_clock_irqtime() which is
only called from x86_64.

Another thing I noticed get_util_irq() is defined in
kernel/sched/sched.h. I don't think the block/blq-mq.c driver should
include it direclty.

Thanks,
Daniel

ps: A customer observes the same problem as Ming is reporting.

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

* Re: [RFC PATCH 2/3] softirq: implement interrupt flood detection
  2019-12-19 10:43         ` Daniel Wagner
@ 2019-12-31  3:48           ` Ming Lei
  2020-01-02 10:28             ` Daniel Wagner
  0 siblings, 1 reply; 15+ messages in thread
From: Ming Lei @ 2019-12-31  3:48 UTC (permalink / raw)
  To: Daniel Wagner
  Cc: Peter Zijlstra, Thomas Gleixner, Jens Axboe, linux-kernel,
	linux-block, Long Li, Ingo Molnar, Christoph Hellwig,
	Keith Busch, Sagi Grimberg, John Garry, Hannes Reinecke

On Thu, Dec 19, 2019 at 11:43:47AM +0100, Daniel Wagner wrote:
> Hi,
> 
> On Thu, Dec 19, 2019 at 10:23:19AM +0100, Peter Zijlstra wrote:
> > On Thu, Dec 19, 2019 at 09:59:48AM +0800, Ming Lei wrote:
> > > > So pray tell, why did you not integrate this with IRQ_TIME_ACCOUNTING ?
> > > > That already takes a timestamp and does most of what you need.
> > > 
> > > Yeah, that was the 1st approach I thought of, but IRQ_TIME_ACCOUNTING
> > > may be disabled, and enabling it may cause observable effect on IO
> > > performance.
> > 
> > Is that an actual concern, are people disabling it?
> 
> In SLE and openSUSE kernels it is disabled for x86_64 at this
> point. And if I am not completely misstaken only x86_64 supports it at
> this point. I was looking at enable_sched_clock_irqtime() which is
> only called from x86_64.
> 
> Another thing I noticed get_util_irq() is defined in
> kernel/sched/sched.h. I don't think the block/blq-mq.c driver should
> include it direclty.

get_util_irq() only works in case of HAVE_SCHED_AVG_IRQ which depends
on IRQ_TIME_ACCOUNTING or PARAVIRT_TIME_ACCOUNTING.

Also rq->avg_irq.util_avg is only updated when there is scheduler
activities. However, when interrupt flood happens, scheduler can't
have chance to be called. Looks get_util_irq() can't be relied on
for this task.

> 
> Thanks,
> Daniel
> 
> ps: A customer observes the same problem as Ming is reporting.

Actually this issue should be more serious on ARM64 system, in which
there are more CPU cores, and each CPU core is often slower than
x86's, and each interrupt is only delivered to single CPU target.

Meantime the storage device performance is same for the two kinds of
systems.


Thanks,
Ming


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

* Re: [RFC PATCH 2/3] softirq: implement interrupt flood detection
  2019-12-31  3:48           ` Ming Lei
@ 2020-01-02 10:28             ` Daniel Wagner
  0 siblings, 0 replies; 15+ messages in thread
From: Daniel Wagner @ 2020-01-02 10:28 UTC (permalink / raw)
  To: Ming Lei
  Cc: Peter Zijlstra, Thomas Gleixner, Jens Axboe, linux-kernel,
	linux-block, Long Li, Ingo Molnar, Christoph Hellwig,
	Keith Busch, Sagi Grimberg, John Garry, Hannes Reinecke

Hi,

On Tue, Dec 31, 2019 at 11:48:06AM +0800, Ming Lei wrote:
> On Thu, Dec 19, 2019 at 11:43:47AM +0100, Daniel Wagner wrote:
> get_util_irq() only works in case of HAVE_SCHED_AVG_IRQ which depends
> on IRQ_TIME_ACCOUNTING or PARAVIRT_TIME_ACCOUNTING.
> 
> Also rq->avg_irq.util_avg is only updated when there is scheduler
> activities. However, when interrupt flood happens, scheduler can't
> have chance to be called. Looks get_util_irq() can't be relied on
> for this task.

I am not totally sold on the idea to do so as much work as possible in
the IRQ context. I started to play with the patches from Keith [1] which
move the work to proper kernel thread.

> > ps: A customer observes the same problem as Ming is reporting.
> 
> Actually this issue should be more serious on ARM64 system, in which
> there are more CPU cores, and each CPU core is often slower than
> x86's, and each interrupt is only delivered to single CPU target.
> 
> Meantime the storage device performance is same for the two kinds of
> systems.

As it turnes out, we missed one fix 2887e41b910b ("blk-wbt: Avoid lock
contention and thundering herd issue in wbt_wait") in our enterprise
kernel which helps but doesn't solve the real cause. But as I said
moving the work out of the IRQ context will address all those
problems. Obvious there is no free lunch, let's see if we find a way
to address all the performance issues.

Thanks,
Daniel

[1] https://lore.kernel.org/linux-nvme/20191209175622.1964-1-kbusch@kernel.org/

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

end of thread, other threads:[~2020-01-02 10:28 UTC | newest]

Thread overview: 15+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2019-12-18  7:19 [RFC PATCH 0/3] softirq/blk-mq: implement interrupt flood detection for avoiding cpu lockup Ming Lei
2019-12-18  7:19 ` [RFC PATCH 1/3] sched/core: add API for exporting runqueue clock Ming Lei
2019-12-18  9:51   ` Peter Zijlstra
2019-12-19  1:29     ` Ming Lei
2019-12-19  9:20       ` Peter Zijlstra
2019-12-18  7:19 ` [RFC PATCH 2/3] softirq: implement interrupt flood detection Ming Lei
2019-12-18 10:49   ` Peter Zijlstra
2019-12-18 12:29     ` Peter Zijlstra
2019-12-19  1:59     ` Ming Lei
2019-12-19  9:23       ` Peter Zijlstra
2019-12-19  9:52         ` Ming Lei
2019-12-19 10:43         ` Daniel Wagner
2019-12-31  3:48           ` Ming Lei
2020-01-02 10:28             ` Daniel Wagner
2019-12-18  7:19 ` [RFC PATCH 3/3] blk-mq: complete request in rescuer process context in case of irq flood Ming Lei

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