summaryrefslogtreecommitdiff
path: root/kernel/sched/rt.c
diff options
context:
space:
mode:
authorGreg Kroah-Hartman <gregkh@google.com>2017-11-30 15:43:08 +0000
committerGreg Kroah-Hartman <gregkh@google.com>2017-11-30 15:43:08 +0000
commit9fbf3d73741260aafb8d84b08f92a86daa6fc026 (patch)
treeed1f12cf5554e1547bc15be8ff774ce589e2627d /kernel/sched/rt.c
parent1d35c0438678c7ad4c367135082685d5754eed20 (diff)
parent08c15ad2e6278a5fe1b209e8fcdbd2d235c48f34 (diff)
Merge 4.4.103 into android-4.4
Changes in 4.4.103 s390: fix transactional execution control register handling s390/runtime instrumention: fix possible memory corruption s390/disassembler: add missing end marker for e7 table s390/disassembler: increase show_code buffer size ipv6: only call ip6_route_dev_notify() once for NETDEV_UNREGISTER AF_VSOCK: Shrink the area influenced by prepare_to_wait vsock: use new wait API for vsock_stream_sendmsg() sched: Make resched_cpu() unconditional lib/mpi: call cond_resched() from mpi_powm() loop x86/decoder: Add new TEST instruction pattern ARM: 8722/1: mm: make STRICT_KERNEL_RWX effective for LPAE ARM: 8721/1: mm: dump: check hardware RO bit for LPAE MIPS: ralink: Fix MT7628 pinmux MIPS: ralink: Fix typo in mt7628 pinmux function ALSA: hda: Add Raven PCI ID dm bufio: fix integer overflow when limiting maximum cache size dm: fix race between dm_get_from_kobject() and __dm_destroy() MIPS: Fix an n32 core file generation regset support regression MIPS: BCM47XX: Fix LED inversion for WRT54GSv1 autofs: don't fail mount for transient error nilfs2: fix race condition that causes file system corruption eCryptfs: use after free in ecryptfs_release_messaging() bcache: check ca->alloc_thread initialized before wake up it isofs: fix timestamps beyond 2027 NFS: Fix typo in nomigration mount option nfs: Fix ugly referral attributes nfsd: deal with revoked delegations appropriately rtlwifi: rtl8192ee: Fix memory leak when loading firmware rtlwifi: fix uninitialized rtlhal->last_suspend_sec time ata: fixes kernel crash while tracing ata_eh_link_autopsy event ext4: fix interaction between i_size, fallocate, and delalloc after a crash ALSA: pcm: update tstamp only if audio_tstamp changed ALSA: usb-audio: Add sanity checks to FE parser ALSA: usb-audio: Fix potential out-of-bound access at parsing SU ALSA: usb-audio: Add sanity checks in v2 clock parsers ALSA: timer: Remove kernel warning at compat ioctl error paths ALSA: hda/realtek - Fix ALC700 family no sound issue fix a page leak in vhost_scsi_iov_to_sgl() error recovery fs/9p: Compare qid.path in v9fs_test_inode iscsi-target: Fix non-immediate TMR reference leak target: Fix QUEUE_FULL + SCSI task attribute handling KVM: nVMX: set IDTR and GDTR limits when loading L1 host state KVM: SVM: obey guest PAT SUNRPC: Fix tracepoint storage issues with svc_recv and svc_rqst_status clk: ti: dra7-atl-clock: Fix of_node reference counting clk: ti: dra7-atl-clock: fix child-node lookups libnvdimm, namespace: fix label initialization to use valid seq numbers libnvdimm, namespace: make 'resource' attribute only readable by root IB/srpt: Do not accept invalid initiator port names IB/srp: Avoid that a cable pull can trigger a kernel crash NFC: fix device-allocation error return i40e: Use smp_rmb rather than read_barrier_depends igb: Use smp_rmb rather than read_barrier_depends igbvf: Use smp_rmb rather than read_barrier_depends ixgbevf: Use smp_rmb rather than read_barrier_depends i40evf: Use smp_rmb rather than read_barrier_depends fm10k: Use smp_rmb rather than read_barrier_depends ixgbe: Fix skb list corruption on Power systems parisc: Fix validity check of pointer size argument in new CAS implementation powerpc/signal: Properly handle return value from uprobe_deny_signal() media: Don't do DMA on stack for firmware upload in the AS102 driver media: rc: check for integer overflow cx231xx-cards: fix NULL-deref on missing association descriptor media: v4l2-ctrl: Fix flags field on Control events sched/rt: Simplify the IPI based RT balancing logic fscrypt: lock mutex before checking for bounce page pool net/9p: Switch to wait_event_killable() PM / OPP: Add missing of_node_put(np) e1000e: Fix error path in link detection e1000e: Fix return value test e1000e: Separate signaling for link check/link up RDS: RDMA: return appropriate error on rdma map failures PCI: Apply _HPX settings only to relevant devices dmaengine: zx: set DMA_CYCLIC cap_mask bit net: Allow IP_MULTICAST_IF to set index to L3 slave net: 3com: typhoon: typhoon_init_one: make return values more specific net: 3com: typhoon: typhoon_init_one: fix incorrect return values drm/armada: Fix compile fail ath10k: fix incorrect txpower set by P2P_DEVICE interface ath10k: ignore configuring the incorrect board_id ath10k: fix potential memory leak in ath10k_wmi_tlv_op_pull_fw_stats() ath10k: set CTS protection VDEV param only if VDEV is up ALSA: hda - Apply ALC269_FIXUP_NO_SHUTUP on HDA_FIXUP_ACT_PROBE drm: Apply range restriction after color adjustment when allocation mac80211: Remove invalid flag operations in mesh TSF synchronization mac80211: Suppress NEW_PEER_CANDIDATE event if no room iio: light: fix improper return value staging: iio: cdc: fix improper return value spi: SPI_FSL_DSPI should depend on HAS_DMA netfilter: nft_queue: use raw_smp_processor_id() netfilter: nf_tables: fix oob access ASoC: rsnd: don't double free kctrl btrfs: return the actual error value from from btrfs_uuid_tree_iterate ASoC: wm_adsp: Don't overrun firmware file buffer when reading region data s390/kbuild: enable modversions for symbols exported from asm xen: xenbus driver must not accept invalid transaction ids Revert "sctp: do not peel off an assoc from one netns to another one" Linux 4.4.103 Signed-off-by: Greg Kroah-Hartman <gregkh@google.com>
Diffstat (limited to 'kernel/sched/rt.c')
-rw-r--r--kernel/sched/rt.c235
1 files changed, 115 insertions, 120 deletions
diff --git a/kernel/sched/rt.c b/kernel/sched/rt.c
index c8322ab130eb..0af072f64e52 100644
--- a/kernel/sched/rt.c
+++ b/kernel/sched/rt.c
@@ -67,10 +67,6 @@ static void start_rt_bandwidth(struct rt_bandwidth *rt_b)
raw_spin_unlock(&rt_b->rt_runtime_lock);
}
-#if defined(CONFIG_SMP) && defined(HAVE_RT_PUSH_IPI)
-static void push_irq_work_func(struct irq_work *work);
-#endif
-
void init_rt_rq(struct rt_rq *rt_rq)
{
struct rt_prio_array *array;
@@ -90,13 +86,6 @@ void init_rt_rq(struct rt_rq *rt_rq)
rt_rq->rt_nr_migratory = 0;
rt_rq->overloaded = 0;
plist_head_init(&rt_rq->pushable_tasks);
-
-#ifdef HAVE_RT_PUSH_IPI
- rt_rq->push_flags = 0;
- rt_rq->push_cpu = nr_cpu_ids;
- raw_spin_lock_init(&rt_rq->push_lock);
- init_irq_work(&rt_rq->push_work, push_irq_work_func);
-#endif
#endif /* CONFIG_SMP */
/* We start is dequeued state, because no RT tasks are queued */
rt_rq->rt_queued = 0;
@@ -1867,160 +1856,166 @@ static void push_rt_tasks(struct rq *rq)
}
#ifdef HAVE_RT_PUSH_IPI
+
/*
- * The search for the next cpu always starts at rq->cpu and ends
- * when we reach rq->cpu again. It will never return rq->cpu.
- * This returns the next cpu to check, or nr_cpu_ids if the loop
- * is complete.
+ * When a high priority task schedules out from a CPU and a lower priority
+ * task is scheduled in, a check is made to see if there's any RT tasks
+ * on other CPUs that are waiting to run because a higher priority RT task
+ * is currently running on its CPU. In this case, the CPU with multiple RT
+ * tasks queued on it (overloaded) needs to be notified that a CPU has opened
+ * up that may be able to run one of its non-running queued RT tasks.
+ *
+ * All CPUs with overloaded RT tasks need to be notified as there is currently
+ * no way to know which of these CPUs have the highest priority task waiting
+ * to run. Instead of trying to take a spinlock on each of these CPUs,
+ * which has shown to cause large latency when done on machines with many
+ * CPUs, sending an IPI to the CPUs to have them push off the overloaded
+ * RT tasks waiting to run.
+ *
+ * Just sending an IPI to each of the CPUs is also an issue, as on large
+ * count CPU machines, this can cause an IPI storm on a CPU, especially
+ * if its the only CPU with multiple RT tasks queued, and a large number
+ * of CPUs scheduling a lower priority task at the same time.
+ *
+ * Each root domain has its own irq work function that can iterate over
+ * all CPUs with RT overloaded tasks. Since all CPUs with overloaded RT
+ * tassk must be checked if there's one or many CPUs that are lowering
+ * their priority, there's a single irq work iterator that will try to
+ * push off RT tasks that are waiting to run.
+ *
+ * When a CPU schedules a lower priority task, it will kick off the
+ * irq work iterator that will jump to each CPU with overloaded RT tasks.
+ * As it only takes the first CPU that schedules a lower priority task
+ * to start the process, the rto_start variable is incremented and if
+ * the atomic result is one, then that CPU will try to take the rto_lock.
+ * This prevents high contention on the lock as the process handles all
+ * CPUs scheduling lower priority tasks.
+ *
+ * All CPUs that are scheduling a lower priority task will increment the
+ * rt_loop_next variable. This will make sure that the irq work iterator
+ * checks all RT overloaded CPUs whenever a CPU schedules a new lower
+ * priority task, even if the iterator is in the middle of a scan. Incrementing
+ * the rt_loop_next will cause the iterator to perform another scan.
*
- * rq->rt.push_cpu holds the last cpu returned by this function,
- * or if this is the first instance, it must hold rq->cpu.
*/
static int rto_next_cpu(struct rq *rq)
{
- int prev_cpu = rq->rt.push_cpu;
+ struct root_domain *rd = rq->rd;
+ int next;
int cpu;
- cpu = cpumask_next(prev_cpu, rq->rd->rto_mask);
-
/*
- * If the previous cpu is less than the rq's CPU, then it already
- * passed the end of the mask, and has started from the beginning.
- * We end if the next CPU is greater or equal to rq's CPU.
+ * When starting the IPI RT pushing, the rto_cpu is set to -1,
+ * rt_next_cpu() will simply return the first CPU found in
+ * the rto_mask.
+ *
+ * If rto_next_cpu() is called with rto_cpu is a valid cpu, it
+ * will return the next CPU found in the rto_mask.
+ *
+ * If there are no more CPUs left in the rto_mask, then a check is made
+ * against rto_loop and rto_loop_next. rto_loop is only updated with
+ * the rto_lock held, but any CPU may increment the rto_loop_next
+ * without any locking.
*/
- if (prev_cpu < rq->cpu) {
- if (cpu >= rq->cpu)
- return nr_cpu_ids;
+ for (;;) {
- } else if (cpu >= nr_cpu_ids) {
- /*
- * We passed the end of the mask, start at the beginning.
- * If the result is greater or equal to the rq's CPU, then
- * the loop is finished.
- */
- cpu = cpumask_first(rq->rd->rto_mask);
- if (cpu >= rq->cpu)
- return nr_cpu_ids;
- }
- rq->rt.push_cpu = cpu;
+ /* When rto_cpu is -1 this acts like cpumask_first() */
+ cpu = cpumask_next(rd->rto_cpu, rd->rto_mask);
- /* Return cpu to let the caller know if the loop is finished or not */
- return cpu;
-}
+ rd->rto_cpu = cpu;
-static int find_next_push_cpu(struct rq *rq)
-{
- struct rq *next_rq;
- int cpu;
+ if (cpu < nr_cpu_ids)
+ return cpu;
- while (1) {
- cpu = rto_next_cpu(rq);
- if (cpu >= nr_cpu_ids)
- break;
- next_rq = cpu_rq(cpu);
+ rd->rto_cpu = -1;
- /* Make sure the next rq can push to this rq */
- if (next_rq->rt.highest_prio.next < rq->rt.highest_prio.curr)
+ /*
+ * ACQUIRE ensures we see the @rto_mask changes
+ * made prior to the @next value observed.
+ *
+ * Matches WMB in rt_set_overload().
+ */
+ next = atomic_read_acquire(&rd->rto_loop_next);
+
+ if (rd->rto_loop == next)
break;
+
+ rd->rto_loop = next;
}
- return cpu;
+ return -1;
+}
+
+static inline bool rto_start_trylock(atomic_t *v)
+{
+ return !atomic_cmpxchg_acquire(v, 0, 1);
}
-#define RT_PUSH_IPI_EXECUTING 1
-#define RT_PUSH_IPI_RESTART 2
+static inline void rto_start_unlock(atomic_t *v)
+{
+ atomic_set_release(v, 0);
+}
static void tell_cpu_to_push(struct rq *rq)
{
- int cpu;
+ int cpu = -1;
- if (rq->rt.push_flags & RT_PUSH_IPI_EXECUTING) {
- raw_spin_lock(&rq->rt.push_lock);
- /* Make sure it's still executing */
- if (rq->rt.push_flags & RT_PUSH_IPI_EXECUTING) {
- /*
- * Tell the IPI to restart the loop as things have
- * changed since it started.
- */
- rq->rt.push_flags |= RT_PUSH_IPI_RESTART;
- raw_spin_unlock(&rq->rt.push_lock);
- return;
- }
- raw_spin_unlock(&rq->rt.push_lock);
- }
+ /* Keep the loop going if the IPI is currently active */
+ atomic_inc(&rq->rd->rto_loop_next);
- /* When here, there's no IPI going around */
-
- rq->rt.push_cpu = rq->cpu;
- cpu = find_next_push_cpu(rq);
- if (cpu >= nr_cpu_ids)
+ /* Only one CPU can initiate a loop at a time */
+ if (!rto_start_trylock(&rq->rd->rto_loop_start))
return;
- rq->rt.push_flags = RT_PUSH_IPI_EXECUTING;
+ raw_spin_lock(&rq->rd->rto_lock);
+
+ /*
+ * The rto_cpu is updated under the lock, if it has a valid cpu
+ * then the IPI is still running and will continue due to the
+ * update to loop_next, and nothing needs to be done here.
+ * Otherwise it is finishing up and an ipi needs to be sent.
+ */
+ if (rq->rd->rto_cpu < 0)
+ cpu = rto_next_cpu(rq);
+
+ raw_spin_unlock(&rq->rd->rto_lock);
- irq_work_queue_on(&rq->rt.push_work, cpu);
+ rto_start_unlock(&rq->rd->rto_loop_start);
+
+ if (cpu >= 0)
+ irq_work_queue_on(&rq->rd->rto_push_work, cpu);
}
/* Called from hardirq context */
-static void try_to_push_tasks(void *arg)
+void rto_push_irq_work_func(struct irq_work *work)
{
- struct rt_rq *rt_rq = arg;
- struct rq *rq, *src_rq;
- int this_cpu;
+ struct rq *rq;
int cpu;
- this_cpu = rt_rq->push_cpu;
+ rq = this_rq();
- /* Paranoid check */
- BUG_ON(this_cpu != smp_processor_id());
-
- rq = cpu_rq(this_cpu);
- src_rq = rq_of_rt_rq(rt_rq);
-
-again:
+ /*
+ * We do not need to grab the lock to check for has_pushable_tasks.
+ * When it gets updated, a check is made if a push is possible.
+ */
if (has_pushable_tasks(rq)) {
raw_spin_lock(&rq->lock);
- push_rt_task(rq);
+ push_rt_tasks(rq);
raw_spin_unlock(&rq->lock);
}
- /* Pass the IPI to the next rt overloaded queue */
- raw_spin_lock(&rt_rq->push_lock);
- /*
- * If the source queue changed since the IPI went out,
- * we need to restart the search from that CPU again.
- */
- if (rt_rq->push_flags & RT_PUSH_IPI_RESTART) {
- rt_rq->push_flags &= ~RT_PUSH_IPI_RESTART;
- rt_rq->push_cpu = src_rq->cpu;
- }
+ raw_spin_lock(&rq->rd->rto_lock);
- cpu = find_next_push_cpu(src_rq);
+ /* Pass the IPI to the next rt overloaded queue */
+ cpu = rto_next_cpu(rq);
- if (cpu >= nr_cpu_ids)
- rt_rq->push_flags &= ~RT_PUSH_IPI_EXECUTING;
- raw_spin_unlock(&rt_rq->push_lock);
+ raw_spin_unlock(&rq->rd->rto_lock);
- if (cpu >= nr_cpu_ids)
+ if (cpu < 0)
return;
- /*
- * It is possible that a restart caused this CPU to be
- * chosen again. Don't bother with an IPI, just see if we
- * have more to push.
- */
- if (unlikely(cpu == rq->cpu))
- goto again;
-
/* Try the next RT overloaded CPU */
- irq_work_queue_on(&rt_rq->push_work, cpu);
-}
-
-static void push_irq_work_func(struct irq_work *work)
-{
- struct rt_rq *rt_rq = container_of(work, struct rt_rq, push_work);
-
- try_to_push_tasks(rt_rq);
+ irq_work_queue_on(&rq->rd->rto_push_work, cpu);
}
#endif /* HAVE_RT_PUSH_IPI */