linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [patch V3 00/20] sched: Rewrite MM CID management
@ 2025-10-29 13:08 Thomas Gleixner
  2025-10-29 13:08 ` [patch V3 01/20] sched/mmcid: Revert the complex " Thomas Gleixner
                   ` (21 more replies)
  0 siblings, 22 replies; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-29 13:08 UTC (permalink / raw)
  To: LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov, Shrikanth Hegde

This is a follow up on V2 series which can be found here:

    https://lore.kernel.org/20251022104005.907410538@linutronix.de

The V1 cover letter contains a detailed analyisis of the issues:

    https://lore.kernel.org/20251015164952.694882104@linutronix.de

TLDR: The CID management is way to complex and adds significant overhead
into scheduler hotpaths.

The series rewrites MM CID management in a more simplistic way which
focusses on low overhead in the scheduler while maintaining per task CIDs
as long as the number of threads is not exceeding the number of possible
CPUs.

The series is based on the V6 series of the rseq rewrite:

    https://lore.kernel.org/20251027084220.785525188@linutronix.de

which is also available from git:

    git://git.kernel.org/pub/scm/linux/kernel/git/tip/tip.git core/rseq

The series on top of the tip core/rseq branch is available from git as
well:

    git://git.kernel.org/pub/scm/linux/kernel/git/tglx/devel.git rseq/cid

Changes vs. V2:

   - Rename to cpumask/bitmap_weighted_or() - Yury

   - Zero the bitmap with length of bitmap_size(nr_possible_cpus()) -
     Shrikanth
   
   - Move cpu_relax() out of for() as that fails to build when cpu_relax()
     is a macro. - Shrikanth

   - Picked up Reviewed/Acked-by tags where appropriate

Thanks,

	tglx
---
Thomas Gleixner (20):
      sched/mmcid: Revert the complex CID management
      sched/mmcid: Use proper data structures
      sched/mmcid: Cacheline align MM CID storage
      sched: Fixup whitespace damage
      sched/mmcid: Move scheduler code out of global header
      sched/mmcid: Prevent pointless work in mm_update_cpus_allowed()
      cpumask: Introduce cpumask_weighted_or()
      sched/mmcid: Use cpumask_weighted_or()
      cpumask: Cache num_possible_cpus()
      sched/mmcid: Convert mm CID mask to a bitmap
      signal: Move MMCID exit out of sighand lock
      sched/mmcid: Move initialization out of line
      sched/mmcid: Provide precomputed maximal value
      sched/mmcid: Serialize sched_mm_cid_fork()/exit() with a mutex
      sched/mmcid: Introduce per task/CPU ownership infrastrcuture
      sched/mmcid: Provide new scheduler CID mechanism
      sched/mmcid: Provide CID ownership mode fixup functions
      irqwork: Move data struct to a types header
      sched/mmcid: Implement deferred mode change
      sched/mmcid: Switch over to the new mechanism

 include/linux/bitmap.h         |   15 
 include/linux/cpumask.h        |   26 +
 include/linux/irq_work.h       |    9 
 include/linux/irq_work_types.h |   14 
 include/linux/mm_types.h       |  125 ------
 include/linux/rseq.h           |   27 -
 include/linux/rseq_types.h     |   71 +++
 include/linux/sched.h          |   19 
 init/init_task.c               |    3 
 kernel/cpu.c                   |   15 
 kernel/exit.c                  |    1 
 kernel/fork.c                  |    7 
 kernel/sched/core.c            |  815 +++++++++++++++++++----------------------
 kernel/sched/sched.h           |  395 ++++++++-----------
 kernel/signal.c                |    2 
 lib/bitmap.c                   |    6 
 16 files changed, 727 insertions(+), 823 deletions(-)



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

* [patch V3 01/20] sched/mmcid: Revert the complex CID management
  2025-10-29 13:08 [patch V3 00/20] sched: Rewrite MM CID management Thomas Gleixner
@ 2025-10-29 13:08 ` Thomas Gleixner
  2025-10-29 13:08 ` [patch V3 02/20] sched/mmcid: Use proper data structures Thomas Gleixner
                   ` (20 subsequent siblings)
  21 siblings, 0 replies; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-29 13:08 UTC (permalink / raw)
  To: LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov, Shrikanth Hegde

The CID management is a complex beast, which affects both scheduling and
task migration. The compaction mechanism forces random tasks of a process
into task work on exit to user space causing latency spikes.

Revert back to the initial simple bitmap allocating mechanics, which are
known to have scalability issues as that allows to gradually build up a
replacement functionality in a reviewable way.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 include/linux/mm_types.h |   53 ----
 kernel/fork.c            |    5 
 kernel/sched/core.c      |  514 +----------------------------------------------
 kernel/sched/sched.h     |  289 +++-----------------------
 4 files changed, 64 insertions(+), 797 deletions(-)

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Link: https://lore.kernel.org/87tt0k3oks.ffs@tglx


--- a/include/linux/mm_types.h
+++ b/include/linux/mm_types.h
@@ -922,13 +922,9 @@ struct vm_area_struct {
 #define vma_policy(vma) NULL
 #endif
 
-#ifdef CONFIG_SCHED_MM_CID
 struct mm_cid {
-	u64 time;
-	int cid;
-	int recent_cid;
+	unsigned int cid;
 };
-#endif
 
 /*
  * Opaque type representing current mm_struct flag state. Must be accessed via
@@ -1000,12 +996,6 @@ struct mm_struct {
 		 * runqueue locks.
 		 */
 		struct mm_cid __percpu *pcpu_cid;
-		/*
-		 * @mm_cid_next_scan: Next mm_cid scan (in jiffies).
-		 *
-		 * When the next mm_cid scan is due (in jiffies).
-		 */
-		unsigned long mm_cid_next_scan;
 		/**
 		 * @nr_cpus_allowed: Number of CPUs allowed for mm.
 		 *
@@ -1014,14 +1004,6 @@ struct mm_struct {
 		 */
 		unsigned int nr_cpus_allowed;
 		/**
-		 * @max_nr_cid: Maximum number of allowed concurrency
-		 *              IDs allocated.
-		 *
-		 * Track the highest number of allowed concurrency IDs
-		 * allocated for the mm.
-		 */
-		atomic_t max_nr_cid;
-		/**
 		 * @cpus_allowed_lock: Lock protecting mm cpus_allowed.
 		 *
 		 * Provide mutual exclusion for mm cpus_allowed and
@@ -1371,35 +1353,7 @@ static inline void vma_iter_init(struct
 
 #ifdef CONFIG_SCHED_MM_CID
 
-enum mm_cid_state {
-	MM_CID_UNSET = -1U,		/* Unset state has lazy_put flag set. */
-	MM_CID_LAZY_PUT = (1U << 31),
-};
-
-static inline bool mm_cid_is_unset(int cid)
-{
-	return cid == MM_CID_UNSET;
-}
-
-static inline bool mm_cid_is_lazy_put(int cid)
-{
-	return !mm_cid_is_unset(cid) && (cid & MM_CID_LAZY_PUT);
-}
-
-static inline bool mm_cid_is_valid(int cid)
-{
-	return !(cid & MM_CID_LAZY_PUT);
-}
-
-static inline int mm_cid_set_lazy_put(int cid)
-{
-	return cid | MM_CID_LAZY_PUT;
-}
-
-static inline int mm_cid_clear_lazy_put(int cid)
-{
-	return cid & ~MM_CID_LAZY_PUT;
-}
+#define	MM_CID_UNSET	(~0U)
 
 /*
  * mm_cpus_allowed: Union of all mm's threads allowed CPUs.
@@ -1432,11 +1386,8 @@ static inline void mm_init_cid(struct mm
 		struct mm_cid *pcpu_cid = per_cpu_ptr(mm->pcpu_cid, i);
 
 		pcpu_cid->cid = MM_CID_UNSET;
-		pcpu_cid->recent_cid = MM_CID_UNSET;
-		pcpu_cid->time = 0;
 	}
 	mm->nr_cpus_allowed = p->nr_cpus_allowed;
-	atomic_set(&mm->max_nr_cid, 0);
 	raw_spin_lock_init(&mm->cpus_allowed_lock);
 	cpumask_copy(mm_cpus_allowed(mm), &p->cpus_mask);
 	cpumask_clear(mm_cidmask(mm));
--- a/kernel/fork.c
+++ b/kernel/fork.c
@@ -955,10 +955,9 @@ static struct task_struct *dup_task_stru
 #endif
 
 #ifdef CONFIG_SCHED_MM_CID
-	tsk->mm_cid = -1;
-	tsk->last_mm_cid = -1;
+	tsk->mm_cid = MM_CID_UNSET;
+	tsk->last_mm_cid = MM_CID_UNSET;
 	tsk->mm_cid_active = 0;
-	tsk->migrate_from_cpu = -1;
 #endif
 	return tsk;
 
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -2128,8 +2128,6 @@ void activate_task(struct rq *rq, struct
 {
 	if (task_on_rq_migrating(p))
 		flags |= ENQUEUE_MIGRATED;
-	if (flags & ENQUEUE_MIGRATED)
-		sched_mm_cid_migrate_to(rq, p);
 
 	enqueue_task(rq, p, flags);
 
@@ -3329,7 +3327,6 @@ void set_task_cpu(struct task_struct *p,
 		if (p->sched_class->migrate_task_rq)
 			p->sched_class->migrate_task_rq(p, new_cpu);
 		p->se.nr_migrations++;
-		sched_mm_cid_migrate_from(p);
 		perf_event_task_migrate(p);
 	}
 
@@ -5312,8 +5309,7 @@ context_switch(struct rq *rq, struct tas
 		}
 	}
 
-	/* switch_mm_cid() requires the memory barriers above. */
-	switch_mm_cid(rq, prev, next);
+	switch_mm_cid(prev, next);
 
 	/*
 	 * Tell rseq that the task was scheduled in. Must be after
@@ -5604,7 +5600,6 @@ void sched_tick(void)
 		resched_latency = cpu_resched_latency(rq);
 	calc_global_load_tick(rq);
 	sched_core_tick(rq);
-	task_tick_mm_cid(rq, donor);
 	scx_tick(rq);
 
 	rq_unlock(rq, &rf);
@@ -10376,522 +10371,47 @@ void call_trace_sched_update_nr_running(
 }
 
 #ifdef CONFIG_SCHED_MM_CID
-
 /*
- * @cid_lock: Guarantee forward-progress of cid allocation.
- *
- * Concurrency ID allocation within a bitmap is mostly lock-free. The cid_lock
- * is only used when contention is detected by the lock-free allocation so
- * forward progress can be guaranteed.
- */
-DEFINE_RAW_SPINLOCK(cid_lock);
-
-/*
- * @use_cid_lock: Select cid allocation behavior: lock-free vs spinlock.
- *
- * When @use_cid_lock is 0, the cid allocation is lock-free. When contention is
- * detected, it is set to 1 to ensure that all newly coming allocations are
- * serialized by @cid_lock until the allocation which detected contention
- * completes and sets @use_cid_lock back to 0. This guarantees forward progress
- * of a cid allocation.
- */
-int use_cid_lock;
-
-/*
- * mm_cid remote-clear implements a lock-free algorithm to clear per-mm/cpu cid
- * concurrently with respect to the execution of the source runqueue context
- * switch.
- *
- * There is one basic properties we want to guarantee here:
- *
- * (1) Remote-clear should _never_ mark a per-cpu cid UNSET when it is actively
- * used by a task. That would lead to concurrent allocation of the cid and
- * userspace corruption.
- *
- * Provide this guarantee by introducing a Dekker memory ordering to guarantee
- * that a pair of loads observe at least one of a pair of stores, which can be
- * shown as:
- *
- *      X = Y = 0
- *
- *      w[X]=1          w[Y]=1
- *      MB              MB
- *      r[Y]=y          r[X]=x
- *
- * Which guarantees that x==0 && y==0 is impossible. But rather than using
- * values 0 and 1, this algorithm cares about specific state transitions of the
- * runqueue current task (as updated by the scheduler context switch), and the
- * per-mm/cpu cid value.
- *
- * Let's introduce task (Y) which has task->mm == mm and task (N) which has
- * task->mm != mm for the rest of the discussion. There are two scheduler state
- * transitions on context switch we care about:
- *
- * (TSA) Store to rq->curr with transition from (N) to (Y)
- *
- * (TSB) Store to rq->curr with transition from (Y) to (N)
- *
- * On the remote-clear side, there is one transition we care about:
- *
- * (TMA) cmpxchg to *pcpu_cid to set the LAZY flag
- *
- * There is also a transition to UNSET state which can be performed from all
- * sides (scheduler, remote-clear). It is always performed with a cmpxchg which
- * guarantees that only a single thread will succeed:
- *
- * (TMB) cmpxchg to *pcpu_cid to mark UNSET
- *
- * Just to be clear, what we do _not_ want to happen is a transition to UNSET
- * when a thread is actively using the cid (property (1)).
- *
- * Let's looks at the relevant combinations of TSA/TSB, and TMA transitions.
- *
- * Scenario A) (TSA)+(TMA) (from next task perspective)
- *
- * CPU0                                      CPU1
- *
- * Context switch CS-1                       Remote-clear
- *   - store to rq->curr: (N)->(Y) (TSA)     - cmpxchg to *pcpu_id to LAZY (TMA)
- *                                             (implied barrier after cmpxchg)
- *   - switch_mm_cid()
- *     - memory barrier (see switch_mm_cid()
- *       comment explaining how this barrier
- *       is combined with other scheduler
- *       barriers)
- *     - mm_cid_get (next)
- *       - READ_ONCE(*pcpu_cid)              - rcu_dereference(src_rq->curr)
- *
- * This Dekker ensures that either task (Y) is observed by the
- * rcu_dereference() or the LAZY flag is observed by READ_ONCE(), or both are
- * observed.
- *
- * If task (Y) store is observed by rcu_dereference(), it means that there is
- * still an active task on the cpu. Remote-clear will therefore not transition
- * to UNSET, which fulfills property (1).
- *
- * If task (Y) is not observed, but the lazy flag is observed by READ_ONCE(),
- * it will move its state to UNSET, which clears the percpu cid perhaps
- * uselessly (which is not an issue for correctness). Because task (Y) is not
- * observed, CPU1 can move ahead to set the state to UNSET. Because moving
- * state to UNSET is done with a cmpxchg expecting that the old state has the
- * LAZY flag set, only one thread will successfully UNSET.
- *
- * If both states (LAZY flag and task (Y)) are observed, the thread on CPU0
- * will observe the LAZY flag and transition to UNSET (perhaps uselessly), and
- * CPU1 will observe task (Y) and do nothing more, which is fine.
- *
- * What we are effectively preventing with this Dekker is a scenario where
- * neither LAZY flag nor store (Y) are observed, which would fail property (1)
- * because this would UNSET a cid which is actively used.
+ * When a task exits, the MM CID held by the task is not longer required as
+ * the task cannot return to user space.
  */
-
-void sched_mm_cid_migrate_from(struct task_struct *t)
-{
-	t->migrate_from_cpu = task_cpu(t);
-}
-
-static
-int __sched_mm_cid_migrate_from_fetch_cid(struct rq *src_rq,
-					  struct task_struct *t,
-					  struct mm_cid *src_pcpu_cid)
-{
-	struct mm_struct *mm = t->mm;
-	struct task_struct *src_task;
-	int src_cid, last_mm_cid;
-
-	if (!mm)
-		return -1;
-
-	last_mm_cid = t->last_mm_cid;
-	/*
-	 * If the migrated task has no last cid, or if the current
-	 * task on src rq uses the cid, it means the source cid does not need
-	 * to be moved to the destination cpu.
-	 */
-	if (last_mm_cid == -1)
-		return -1;
-	src_cid = READ_ONCE(src_pcpu_cid->cid);
-	if (!mm_cid_is_valid(src_cid) || last_mm_cid != src_cid)
-		return -1;
-
-	/*
-	 * If we observe an active task using the mm on this rq, it means we
-	 * are not the last task to be migrated from this cpu for this mm, so
-	 * there is no need to move src_cid to the destination cpu.
-	 */
-	guard(rcu)();
-	src_task = rcu_dereference(src_rq->curr);
-	if (READ_ONCE(src_task->mm_cid_active) && src_task->mm == mm) {
-		t->last_mm_cid = -1;
-		return -1;
-	}
-
-	return src_cid;
-}
-
-static
-int __sched_mm_cid_migrate_from_try_steal_cid(struct rq *src_rq,
-					      struct task_struct *t,
-					      struct mm_cid *src_pcpu_cid,
-					      int src_cid)
-{
-	struct task_struct *src_task;
-	struct mm_struct *mm = t->mm;
-	int lazy_cid;
-
-	if (src_cid == -1)
-		return -1;
-
-	/*
-	 * Attempt to clear the source cpu cid to move it to the destination
-	 * cpu.
-	 */
-	lazy_cid = mm_cid_set_lazy_put(src_cid);
-	if (!try_cmpxchg(&src_pcpu_cid->cid, &src_cid, lazy_cid))
-		return -1;
-
-	/*
-	 * The implicit barrier after cmpxchg per-mm/cpu cid before loading
-	 * rq->curr->mm matches the scheduler barrier in context_switch()
-	 * between store to rq->curr and load of prev and next task's
-	 * per-mm/cpu cid.
-	 *
-	 * The implicit barrier after cmpxchg per-mm/cpu cid before loading
-	 * rq->curr->mm_cid_active matches the barrier in
-	 * sched_mm_cid_exit_signals(), sched_mm_cid_before_execve(), and
-	 * sched_mm_cid_after_execve() between store to t->mm_cid_active and
-	 * load of per-mm/cpu cid.
-	 */
-
-	/*
-	 * If we observe an active task using the mm on this rq after setting
-	 * the lazy-put flag, this task will be responsible for transitioning
-	 * from lazy-put flag set to MM_CID_UNSET.
-	 */
-	scoped_guard (rcu) {
-		src_task = rcu_dereference(src_rq->curr);
-		if (READ_ONCE(src_task->mm_cid_active) && src_task->mm == mm) {
-			/*
-			 * We observed an active task for this mm, there is therefore
-			 * no point in moving this cid to the destination cpu.
-			 */
-			t->last_mm_cid = -1;
-			return -1;
-		}
-	}
-
-	/*
-	 * The src_cid is unused, so it can be unset.
-	 */
-	if (!try_cmpxchg(&src_pcpu_cid->cid, &lazy_cid, MM_CID_UNSET))
-		return -1;
-	WRITE_ONCE(src_pcpu_cid->recent_cid, MM_CID_UNSET);
-	return src_cid;
-}
-
-/*
- * Migration to dst cpu. Called with dst_rq lock held.
- * Interrupts are disabled, which keeps the window of cid ownership without the
- * source rq lock held small.
- */
-void sched_mm_cid_migrate_to(struct rq *dst_rq, struct task_struct *t)
-{
-	struct mm_cid *src_pcpu_cid, *dst_pcpu_cid;
-	struct mm_struct *mm = t->mm;
-	int src_cid, src_cpu;
-	bool dst_cid_is_set;
-	struct rq *src_rq;
-
-	lockdep_assert_rq_held(dst_rq);
-
-	if (!mm)
-		return;
-	src_cpu = t->migrate_from_cpu;
-	if (src_cpu == -1) {
-		t->last_mm_cid = -1;
-		return;
-	}
-	/*
-	 * Move the src cid if the dst cid is unset. This keeps id
-	 * allocation closest to 0 in cases where few threads migrate around
-	 * many CPUs.
-	 *
-	 * If destination cid or recent cid is already set, we may have
-	 * to just clear the src cid to ensure compactness in frequent
-	 * migrations scenarios.
-	 *
-	 * It is not useful to clear the src cid when the number of threads is
-	 * greater or equal to the number of allowed CPUs, because user-space
-	 * can expect that the number of allowed cids can reach the number of
-	 * allowed CPUs.
-	 */
-	dst_pcpu_cid = per_cpu_ptr(mm->pcpu_cid, cpu_of(dst_rq));
-	dst_cid_is_set = !mm_cid_is_unset(READ_ONCE(dst_pcpu_cid->cid)) ||
-			 !mm_cid_is_unset(READ_ONCE(dst_pcpu_cid->recent_cid));
-	if (dst_cid_is_set && atomic_read(&mm->mm_users) >= READ_ONCE(mm->nr_cpus_allowed))
-		return;
-	src_pcpu_cid = per_cpu_ptr(mm->pcpu_cid, src_cpu);
-	src_rq = cpu_rq(src_cpu);
-	src_cid = __sched_mm_cid_migrate_from_fetch_cid(src_rq, t, src_pcpu_cid);
-	if (src_cid == -1)
-		return;
-	src_cid = __sched_mm_cid_migrate_from_try_steal_cid(src_rq, t, src_pcpu_cid,
-							    src_cid);
-	if (src_cid == -1)
-		return;
-	if (dst_cid_is_set) {
-		__mm_cid_put(mm, src_cid);
-		return;
-	}
-	/* Move src_cid to dst cpu. */
-	mm_cid_snapshot_time(dst_rq, mm);
-	WRITE_ONCE(dst_pcpu_cid->cid, src_cid);
-	WRITE_ONCE(dst_pcpu_cid->recent_cid, src_cid);
-}
-
-static void sched_mm_cid_remote_clear(struct mm_struct *mm, struct mm_cid *pcpu_cid,
-				      int cpu)
-{
-	struct rq *rq = cpu_rq(cpu);
-	struct task_struct *t;
-	int cid, lazy_cid;
-
-	cid = READ_ONCE(pcpu_cid->cid);
-	if (!mm_cid_is_valid(cid))
-		return;
-
-	/*
-	 * Clear the cpu cid if it is set to keep cid allocation compact.  If
-	 * there happens to be other tasks left on the source cpu using this
-	 * mm, the next task using this mm will reallocate its cid on context
-	 * switch.
-	 */
-	lazy_cid = mm_cid_set_lazy_put(cid);
-	if (!try_cmpxchg(&pcpu_cid->cid, &cid, lazy_cid))
-		return;
-
-	/*
-	 * The implicit barrier after cmpxchg per-mm/cpu cid before loading
-	 * rq->curr->mm matches the scheduler barrier in context_switch()
-	 * between store to rq->curr and load of prev and next task's
-	 * per-mm/cpu cid.
-	 *
-	 * The implicit barrier after cmpxchg per-mm/cpu cid before loading
-	 * rq->curr->mm_cid_active matches the barrier in
-	 * sched_mm_cid_exit_signals(), sched_mm_cid_before_execve(), and
-	 * sched_mm_cid_after_execve() between store to t->mm_cid_active and
-	 * load of per-mm/cpu cid.
-	 */
-
-	/*
-	 * If we observe an active task using the mm on this rq after setting
-	 * the lazy-put flag, that task will be responsible for transitioning
-	 * from lazy-put flag set to MM_CID_UNSET.
-	 */
-	scoped_guard (rcu) {
-		t = rcu_dereference(rq->curr);
-		if (READ_ONCE(t->mm_cid_active) && t->mm == mm)
-			return;
-	}
-
-	/*
-	 * The cid is unused, so it can be unset.
-	 * Disable interrupts to keep the window of cid ownership without rq
-	 * lock small.
-	 */
-	scoped_guard (irqsave) {
-		if (try_cmpxchg(&pcpu_cid->cid, &lazy_cid, MM_CID_UNSET))
-			__mm_cid_put(mm, cid);
-	}
-}
-
-static void sched_mm_cid_remote_clear_old(struct mm_struct *mm, int cpu)
-{
-	struct rq *rq = cpu_rq(cpu);
-	struct mm_cid *pcpu_cid;
-	struct task_struct *curr;
-	u64 rq_clock;
-
-	/*
-	 * rq->clock load is racy on 32-bit but one spurious clear once in a
-	 * while is irrelevant.
-	 */
-	rq_clock = READ_ONCE(rq->clock);
-	pcpu_cid = per_cpu_ptr(mm->pcpu_cid, cpu);
-
-	/*
-	 * In order to take care of infrequently scheduled tasks, bump the time
-	 * snapshot associated with this cid if an active task using the mm is
-	 * observed on this rq.
-	 */
-	scoped_guard (rcu) {
-		curr = rcu_dereference(rq->curr);
-		if (READ_ONCE(curr->mm_cid_active) && curr->mm == mm) {
-			WRITE_ONCE(pcpu_cid->time, rq_clock);
-			return;
-		}
-	}
-
-	if (rq_clock < pcpu_cid->time + SCHED_MM_CID_PERIOD_NS)
-		return;
-	sched_mm_cid_remote_clear(mm, pcpu_cid, cpu);
-}
-
-static void sched_mm_cid_remote_clear_weight(struct mm_struct *mm, int cpu,
-					     int weight)
-{
-	struct mm_cid *pcpu_cid;
-	int cid;
-
-	pcpu_cid = per_cpu_ptr(mm->pcpu_cid, cpu);
-	cid = READ_ONCE(pcpu_cid->cid);
-	if (!mm_cid_is_valid(cid) || cid < weight)
-		return;
-	sched_mm_cid_remote_clear(mm, pcpu_cid, cpu);
-}
-
-static void task_mm_cid_work(struct callback_head *work)
-{
-	unsigned long now = jiffies, old_scan, next_scan;
-	struct task_struct *t = current;
-	struct cpumask *cidmask;
-	struct mm_struct *mm;
-	int weight, cpu;
-
-	WARN_ON_ONCE(t != container_of(work, struct task_struct, cid_work));
-
-	work->next = work;	/* Prevent double-add */
-	if (t->flags & PF_EXITING)
-		return;
-	mm = t->mm;
-	if (!mm)
-		return;
-	old_scan = READ_ONCE(mm->mm_cid_next_scan);
-	next_scan = now + msecs_to_jiffies(MM_CID_SCAN_DELAY);
-	if (!old_scan) {
-		unsigned long res;
-
-		res = cmpxchg(&mm->mm_cid_next_scan, old_scan, next_scan);
-		if (res != old_scan)
-			old_scan = res;
-		else
-			old_scan = next_scan;
-	}
-	if (time_before(now, old_scan))
-		return;
-	if (!try_cmpxchg(&mm->mm_cid_next_scan, &old_scan, next_scan))
-		return;
-	cidmask = mm_cidmask(mm);
-	/* Clear cids that were not recently used. */
-	for_each_possible_cpu(cpu)
-		sched_mm_cid_remote_clear_old(mm, cpu);
-	weight = cpumask_weight(cidmask);
-	/*
-	 * Clear cids that are greater or equal to the cidmask weight to
-	 * recompact it.
-	 */
-	for_each_possible_cpu(cpu)
-		sched_mm_cid_remote_clear_weight(mm, cpu, weight);
-}
-
-void init_sched_mm_cid(struct task_struct *t)
-{
-	struct mm_struct *mm = t->mm;
-	int mm_users = 0;
-
-	if (mm) {
-		mm_users = atomic_read(&mm->mm_users);
-		if (mm_users == 1)
-			mm->mm_cid_next_scan = jiffies + msecs_to_jiffies(MM_CID_SCAN_DELAY);
-	}
-	t->cid_work.next = &t->cid_work;	/* Protect against double add */
-	init_task_work(&t->cid_work, task_mm_cid_work);
-}
-
-void task_tick_mm_cid(struct rq *rq, struct task_struct *curr)
-{
-	struct callback_head *work = &curr->cid_work;
-	unsigned long now = jiffies;
-
-	if (!curr->mm || (curr->flags & (PF_EXITING | PF_KTHREAD)) ||
-	    work->next != work)
-		return;
-	if (time_before(now, READ_ONCE(curr->mm->mm_cid_next_scan)))
-		return;
-
-	/* No page allocation under rq lock */
-	task_work_add(curr, work, TWA_RESUME);
-}
-
 void sched_mm_cid_exit_signals(struct task_struct *t)
 {
 	struct mm_struct *mm = t->mm;
-	struct rq *rq;
 
-	if (!mm)
+	if (!mm || !t->mm_cid_active)
 		return;
 
-	preempt_disable();
-	rq = this_rq();
-	guard(rq_lock_irqsave)(rq);
-	preempt_enable_no_resched();	/* holding spinlock */
-	WRITE_ONCE(t->mm_cid_active, 0);
-	/*
-	 * Store t->mm_cid_active before loading per-mm/cpu cid.
-	 * Matches barrier in sched_mm_cid_remote_clear_old().
-	 */
-	smp_mb();
-	mm_cid_put(mm);
-	t->last_mm_cid = t->mm_cid = -1;
+	guard(preempt)();
+	t->mm_cid_active = 0;
+	if (t->mm_cid != MM_CID_UNSET) {
+		cpumask_clear_cpu(t->mm_cid, mm_cidmask(mm));
+		t->mm_cid = MM_CID_UNSET;
+	}
 }
 
+/* Deactivate MM CID allocation across execve() */
 void sched_mm_cid_before_execve(struct task_struct *t)
 {
-	struct mm_struct *mm = t->mm;
-	struct rq *rq;
-
-	if (!mm)
-		return;
-
-	preempt_disable();
-	rq = this_rq();
-	guard(rq_lock_irqsave)(rq);
-	preempt_enable_no_resched();	/* holding spinlock */
-	WRITE_ONCE(t->mm_cid_active, 0);
-	/*
-	 * Store t->mm_cid_active before loading per-mm/cpu cid.
-	 * Matches barrier in sched_mm_cid_remote_clear_old().
-	 */
-	smp_mb();
-	mm_cid_put(mm);
-	t->last_mm_cid = t->mm_cid = -1;
+	sched_mm_cid_exit_signals(t);
 }
 
+/* Reactivate MM CID after successful execve() */
 void sched_mm_cid_after_execve(struct task_struct *t)
 {
 	struct mm_struct *mm = t->mm;
-	struct rq *rq;
 
 	if (!mm)
 		return;
 
-	preempt_disable();
-	rq = this_rq();
-	scoped_guard (rq_lock_irqsave, rq) {
-		preempt_enable_no_resched();	/* holding spinlock */
-		WRITE_ONCE(t->mm_cid_active, 1);
-		/*
-		 * Store t->mm_cid_active before loading per-mm/cpu cid.
-		 * Matches barrier in sched_mm_cid_remote_clear_old().
-		 */
-		smp_mb();
-		t->last_mm_cid = t->mm_cid = mm_cid_get(rq, t, mm);
-	}
+	guard(preempt)();
+	t->mm_cid_active = 1;
+	mm_cid_select(t);
 }
 
 void sched_mm_cid_fork(struct task_struct *t)
 {
-	WARN_ON_ONCE(!t->mm || t->mm_cid != -1);
+	WARN_ON_ONCE(!t->mm || t->mm_cid != MM_CID_UNSET);
 	t->mm_cid_active = 1;
 }
 #endif /* CONFIG_SCHED_MM_CID */
--- a/kernel/sched/sched.h
+++ b/kernel/sched/sched.h
@@ -3539,286 +3539,83 @@ extern void sched_dynamic_update(int mod
 extern const char *preempt_modes[];
 
 #ifdef CONFIG_SCHED_MM_CID
-
-#define SCHED_MM_CID_PERIOD_NS	(100ULL * 1000000)	/* 100ms */
-#define MM_CID_SCAN_DELAY	100			/* 100ms */
-
-extern raw_spinlock_t cid_lock;
-extern int use_cid_lock;
-
-extern void sched_mm_cid_migrate_from(struct task_struct *t);
-extern void sched_mm_cid_migrate_to(struct rq *dst_rq, struct task_struct *t);
-extern void task_tick_mm_cid(struct rq *rq, struct task_struct *curr);
-extern void init_sched_mm_cid(struct task_struct *t);
-
-static inline void __mm_cid_put(struct mm_struct *mm, int cid)
-{
-	if (cid < 0)
-		return;
-	cpumask_clear_cpu(cid, mm_cidmask(mm));
-}
-
-/*
- * The per-mm/cpu cid can have the MM_CID_LAZY_PUT flag set or transition to
- * the MM_CID_UNSET state without holding the rq lock, but the rq lock needs to
- * be held to transition to other states.
- *
- * State transitions synchronized with cmpxchg or try_cmpxchg need to be
- * consistent across CPUs, which prevents use of this_cpu_cmpxchg.
- */
-static inline void mm_cid_put_lazy(struct task_struct *t)
+static inline void init_sched_mm_cid(struct task_struct *t)
 {
 	struct mm_struct *mm = t->mm;
-	struct mm_cid __percpu *pcpu_cid = mm->pcpu_cid;
-	int cid;
+	unsigned int max_cid;
 
-	lockdep_assert_irqs_disabled();
-	cid = __this_cpu_read(pcpu_cid->cid);
-	if (!mm_cid_is_lazy_put(cid) ||
-	    !try_cmpxchg(&this_cpu_ptr(pcpu_cid)->cid, &cid, MM_CID_UNSET))
+	if (!mm)
 		return;
-	__mm_cid_put(mm, mm_cid_clear_lazy_put(cid));
-}
-
-static inline int mm_cid_pcpu_unset(struct mm_struct *mm)
-{
-	struct mm_cid __percpu *pcpu_cid = mm->pcpu_cid;
-	int cid, res;
 
-	lockdep_assert_irqs_disabled();
-	cid = __this_cpu_read(pcpu_cid->cid);
-	for (;;) {
-		if (mm_cid_is_unset(cid))
-			return MM_CID_UNSET;
-		/*
-		 * Attempt transition from valid or lazy-put to unset.
-		 */
-		res = cmpxchg(&this_cpu_ptr(pcpu_cid)->cid, cid, MM_CID_UNSET);
-		if (res == cid)
-			break;
-		cid = res;
-	}
-	return cid;
+	/* Preset last_mm_cid */
+	max_cid = min_t(int, READ_ONCE(mm->nr_cpus_allowed), atomic_read(&mm->mm_users));
+	t->last_mm_cid = max_cid - 1;
 }
 
-static inline void mm_cid_put(struct mm_struct *mm)
+static inline bool __mm_cid_get(struct task_struct *t, unsigned int cid, unsigned int max_cids)
 {
-	int cid;
+	struct mm_struct *mm = t->mm;
 
-	lockdep_assert_irqs_disabled();
-	cid = mm_cid_pcpu_unset(mm);
-	if (cid == MM_CID_UNSET)
-		return;
-	__mm_cid_put(mm, mm_cid_clear_lazy_put(cid));
+	if (cid >= max_cids)
+		return false;
+	if (cpumask_test_and_set_cpu(cid, mm_cidmask(mm)))
+		return false;
+	t->mm_cid = t->last_mm_cid = cid;
+	__this_cpu_write(mm->pcpu_cid->cid, cid);
+	return true;
 }
 
-static inline int __mm_cid_try_get(struct task_struct *t, struct mm_struct *mm)
+static inline bool mm_cid_get(struct task_struct *t)
 {
-	struct cpumask *cidmask = mm_cidmask(mm);
-	struct mm_cid __percpu *pcpu_cid = mm->pcpu_cid;
-	int cid, max_nr_cid, allowed_max_nr_cid;
+	struct mm_struct *mm = t->mm;
+	unsigned int max_cids;
 
-	/*
-	 * After shrinking the number of threads or reducing the number
-	 * of allowed cpus, reduce the value of max_nr_cid so expansion
-	 * of cid allocation will preserve cache locality if the number
-	 * of threads or allowed cpus increase again.
-	 */
-	max_nr_cid = atomic_read(&mm->max_nr_cid);
-	while ((allowed_max_nr_cid = min_t(int, READ_ONCE(mm->nr_cpus_allowed),
-					   atomic_read(&mm->mm_users))),
-	       max_nr_cid > allowed_max_nr_cid) {
-		/* atomic_try_cmpxchg loads previous mm->max_nr_cid into max_nr_cid. */
-		if (atomic_try_cmpxchg(&mm->max_nr_cid, &max_nr_cid, allowed_max_nr_cid)) {
-			max_nr_cid = allowed_max_nr_cid;
-			break;
-		}
-	}
-	/* Try to re-use recent cid. This improves cache locality. */
-	cid = __this_cpu_read(pcpu_cid->recent_cid);
-	if (!mm_cid_is_unset(cid) && cid < max_nr_cid &&
-	    !cpumask_test_and_set_cpu(cid, cidmask))
-		return cid;
-	/*
-	 * Expand cid allocation if the maximum number of concurrency
-	 * IDs allocated (max_nr_cid) is below the number cpus allowed
-	 * and number of threads. Expanding cid allocation as much as
-	 * possible improves cache locality.
-	 */
-	cid = max_nr_cid;
-	while (cid < READ_ONCE(mm->nr_cpus_allowed) && cid < atomic_read(&mm->mm_users)) {
-		/* atomic_try_cmpxchg loads previous mm->max_nr_cid into cid. */
-		if (!atomic_try_cmpxchg(&mm->max_nr_cid, &cid, cid + 1))
-			continue;
-		if (!cpumask_test_and_set_cpu(cid, cidmask))
-			return cid;
-	}
-	/*
-	 * Find the first available concurrency id.
-	 * Retry finding first zero bit if the mask is temporarily
-	 * filled. This only happens during concurrent remote-clear
-	 * which owns a cid without holding a rq lock.
-	 */
-	for (;;) {
-		cid = cpumask_first_zero(cidmask);
-		if (cid < READ_ONCE(mm->nr_cpus_allowed))
-			break;
-		cpu_relax();
-	}
-	if (cpumask_test_and_set_cpu(cid, cidmask))
-		return -1;
+	max_cids = min_t(int, READ_ONCE(mm->nr_cpus_allowed), atomic_read(&mm->mm_users));
 
-	return cid;
-}
+	/* Try to reuse the last CID of this task */
+	if (__mm_cid_get(t, t->last_mm_cid, max_cids))
+		return true;
 
-/*
- * Save a snapshot of the current runqueue time of this cpu
- * with the per-cpu cid value, allowing to estimate how recently it was used.
- */
-static inline void mm_cid_snapshot_time(struct rq *rq, struct mm_struct *mm)
-{
-	struct mm_cid *pcpu_cid = per_cpu_ptr(mm->pcpu_cid, cpu_of(rq));
+	/* Try to reuse the last CID of this mm on this CPU */
+	if (__mm_cid_get(t, __this_cpu_read(mm->pcpu_cid->cid), max_cids))
+		return true;
 
-	lockdep_assert_rq_held(rq);
-	WRITE_ONCE(pcpu_cid->time, rq->clock);
+	/* Try the first zero bit in the cidmask. */
+	return __mm_cid_get(t, cpumask_first_zero(mm_cidmask(mm)), max_cids);
 }
 
-static inline int __mm_cid_get(struct rq *rq, struct task_struct *t,
-			       struct mm_struct *mm)
+static inline void mm_cid_select(struct task_struct *t)
 {
-	int cid;
-
-	/*
-	 * All allocations (even those using the cid_lock) are lock-free. If
-	 * use_cid_lock is set, hold the cid_lock to perform cid allocation to
-	 * guarantee forward progress.
-	 */
-	if (!READ_ONCE(use_cid_lock)) {
-		cid = __mm_cid_try_get(t, mm);
-		if (cid >= 0)
-			goto end;
-		raw_spin_lock(&cid_lock);
-	} else {
-		raw_spin_lock(&cid_lock);
-		cid = __mm_cid_try_get(t, mm);
-		if (cid >= 0)
-			goto unlock;
-	}
-
-	/*
-	 * cid concurrently allocated. Retry while forcing following
-	 * allocations to use the cid_lock to ensure forward progress.
-	 */
-	WRITE_ONCE(use_cid_lock, 1);
-	/*
-	 * Set use_cid_lock before allocation. Only care about program order
-	 * because this is only required for forward progress.
-	 */
-	barrier();
-	/*
-	 * Retry until it succeeds. It is guaranteed to eventually succeed once
-	 * all newcoming allocations observe the use_cid_lock flag set.
-	 */
-	do {
-		cid = __mm_cid_try_get(t, mm);
-		cpu_relax();
-	} while (cid < 0);
 	/*
-	 * Allocate before clearing use_cid_lock. Only care about
-	 * program order because this is for forward progress.
+	 * mm_cid_get() can fail when the maximum CID, which is determined
+	 * by min(mm->nr_cpus_allowed, mm->mm_users) changes concurrently.
+	 * That's a transient failure as there cannot be more tasks
+	 * concurrently on a CPU (or about to be scheduled in) than that.
 	 */
-	barrier();
-	WRITE_ONCE(use_cid_lock, 0);
-unlock:
-	raw_spin_unlock(&cid_lock);
-end:
-	mm_cid_snapshot_time(rq, mm);
-
-	return cid;
-}
-
-static inline int mm_cid_get(struct rq *rq, struct task_struct *t,
-			     struct mm_struct *mm)
-{
-	struct mm_cid __percpu *pcpu_cid = mm->pcpu_cid;
-	int cid;
-
-	lockdep_assert_rq_held(rq);
-	cid = __this_cpu_read(pcpu_cid->cid);
-	if (mm_cid_is_valid(cid)) {
-		mm_cid_snapshot_time(rq, mm);
-		return cid;
-	}
-	if (mm_cid_is_lazy_put(cid)) {
-		if (try_cmpxchg(&this_cpu_ptr(pcpu_cid)->cid, &cid, MM_CID_UNSET))
-			__mm_cid_put(mm, mm_cid_clear_lazy_put(cid));
+	for (;;) {
+		if (mm_cid_get(t))
+			break;
 	}
-	cid = __mm_cid_get(rq, t, mm);
-	__this_cpu_write(pcpu_cid->cid, cid);
-	__this_cpu_write(pcpu_cid->recent_cid, cid);
-
-	return cid;
 }
 
-static inline void switch_mm_cid(struct rq *rq,
-				 struct task_struct *prev,
-				 struct task_struct *next)
+static inline void switch_mm_cid(struct task_struct *prev, struct task_struct *next)
 {
-	/*
-	 * Provide a memory barrier between rq->curr store and load of
-	 * {prev,next}->mm->pcpu_cid[cpu] on rq->curr->mm transition.
-	 *
-	 * Should be adapted if context_switch() is modified.
-	 */
-	if (!next->mm) {                                // to kernel
-		/*
-		 * user -> kernel transition does not guarantee a barrier, but
-		 * we can use the fact that it performs an atomic operation in
-		 * mmgrab().
-		 */
-		if (prev->mm)                           // from user
-			smp_mb__after_mmgrab();
-		/*
-		 * kernel -> kernel transition does not change rq->curr->mm
-		 * state. It stays NULL.
-		 */
-	} else {                                        // to user
-		/*
-		 * kernel -> user transition does not provide a barrier
-		 * between rq->curr store and load of {prev,next}->mm->pcpu_cid[cpu].
-		 * Provide it here.
-		 */
-		if (!prev->mm) {                        // from kernel
-			smp_mb();
-		} else {				// from user
-			/*
-			 * user->user transition relies on an implicit
-			 * memory barrier in switch_mm() when
-			 * current->mm changes. If the architecture
-			 * switch_mm() does not have an implicit memory
-			 * barrier, it is emitted here.  If current->mm
-			 * is unchanged, no barrier is needed.
-			 */
-			smp_mb__after_switch_mm();
-		}
-	}
 	if (prev->mm_cid_active) {
-		mm_cid_snapshot_time(rq, prev->mm);
-		mm_cid_put_lazy(prev);
-		prev->mm_cid = -1;
+		if (prev->mm_cid != MM_CID_UNSET)
+			cpumask_clear_cpu(prev->mm_cid, mm_cidmask(prev->mm));
+		prev->mm_cid = MM_CID_UNSET;
 	}
+
 	if (next->mm_cid_active) {
-		next->last_mm_cid = next->mm_cid = mm_cid_get(rq, next, next->mm);
+		mm_cid_select(next);
 		rseq_sched_set_task_mm_cid(next, next->mm_cid);
 	}
 }
 
 #else /* !CONFIG_SCHED_MM_CID: */
-static inline void switch_mm_cid(struct rq *rq, struct task_struct *prev, struct task_struct *next) { }
-static inline void sched_mm_cid_migrate_from(struct task_struct *t) { }
-static inline void sched_mm_cid_migrate_to(struct rq *dst_rq, struct task_struct *t) { }
-static inline void task_tick_mm_cid(struct rq *rq, struct task_struct *curr) { }
 static inline void init_sched_mm_cid(struct task_struct *t) { }
+static inline void mm_cid_select(struct task_struct *t) { }
+static inline void switch_mm_cid(struct task_struct *prev, struct task_struct *next) { }
 #endif /* !CONFIG_SCHED_MM_CID */
 
 extern u64 avg_vruntime(struct cfs_rq *cfs_rq);


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

* [patch V3 02/20] sched/mmcid: Use proper data structures
  2025-10-29 13:08 [patch V3 00/20] sched: Rewrite MM CID management Thomas Gleixner
  2025-10-29 13:08 ` [patch V3 01/20] sched/mmcid: Revert the complex " Thomas Gleixner
@ 2025-10-29 13:08 ` Thomas Gleixner
  2025-10-29 15:31   ` Mathieu Desnoyers
  2025-10-29 13:08 ` [patch V3 03/20] sched/mmcid: Cacheline align MM CID storage Thomas Gleixner
                   ` (19 subsequent siblings)
  21 siblings, 1 reply; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-29 13:08 UTC (permalink / raw)
  To: LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov, Shrikanth Hegde

Having a lot of CID functionality specific members in struct task_struct
and struct mm_struct is not really making the code easier to read.

Encapsulate the CID specific parts in data structures and keep them
seperate from the stuff they are embedded in.

No functional change.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 include/linux/mm_types.h   |   56 +++++++++++----------------------------------
 include/linux/rseq_types.h |   42 +++++++++++++++++++++++++++++++++
 include/linux/sched.h      |   11 +-------
 init/init_task.c           |    3 ++
 kernel/fork.c              |    6 ++--
 kernel/sched/core.c        |   16 ++++++------
 kernel/sched/sched.h       |   26 ++++++++++----------
 7 files changed, 85 insertions(+), 75 deletions(-)

--- a/include/linux/mm_types.h
+++ b/include/linux/mm_types.h
@@ -20,6 +20,7 @@
 #include <linux/seqlock.h>
 #include <linux/percpu_counter.h>
 #include <linux/types.h>
+#include <linux/rseq_types.h>
 #include <linux/bitmap.h>
 
 #include <asm/mmu.h>
@@ -922,10 +923,6 @@ struct vm_area_struct {
 #define vma_policy(vma) NULL
 #endif
 
-struct mm_cid {
-	unsigned int cid;
-};
-
 /*
  * Opaque type representing current mm_struct flag state. Must be accessed via
  * mm_flags_xxx() helper functions.
@@ -987,30 +984,9 @@ struct mm_struct {
 		 */
 		atomic_t mm_users;
 
-#ifdef CONFIG_SCHED_MM_CID
-		/**
-		 * @pcpu_cid: Per-cpu current cid.
-		 *
-		 * Keep track of the currently allocated mm_cid for each cpu.
-		 * The per-cpu mm_cid values are serialized by their respective
-		 * runqueue locks.
-		 */
-		struct mm_cid __percpu *pcpu_cid;
-		/**
-		 * @nr_cpus_allowed: Number of CPUs allowed for mm.
-		 *
-		 * Number of CPUs allowed in the union of all mm's
-		 * threads allowed CPUs.
-		 */
-		unsigned int nr_cpus_allowed;
-		/**
-		 * @cpus_allowed_lock: Lock protecting mm cpus_allowed.
-		 *
-		 * Provide mutual exclusion for mm cpus_allowed and
-		 * mm nr_cpus_allowed updates.
-		 */
-		raw_spinlock_t cpus_allowed_lock;
-#endif
+		/* MM CID related storage */
+		struct mm_mm_cid mm_cid;
+
 #ifdef CONFIG_MMU
 		atomic_long_t pgtables_bytes;	/* size of all page tables */
 #endif
@@ -1352,9 +1328,6 @@ static inline void vma_iter_init(struct
 }
 
 #ifdef CONFIG_SCHED_MM_CID
-
-#define	MM_CID_UNSET	(~0U)
-
 /*
  * mm_cpus_allowed: Union of all mm's threads allowed CPUs.
  */
@@ -1383,20 +1356,20 @@ static inline void mm_init_cid(struct mm
 	int i;
 
 	for_each_possible_cpu(i) {
-		struct mm_cid *pcpu_cid = per_cpu_ptr(mm->pcpu_cid, i);
+		struct mm_cid_pcpu *pcpu = per_cpu_ptr(mm->mm_cid.pcpu, i);
 
-		pcpu_cid->cid = MM_CID_UNSET;
+		pcpu->cid = MM_CID_UNSET;
 	}
-	mm->nr_cpus_allowed = p->nr_cpus_allowed;
-	raw_spin_lock_init(&mm->cpus_allowed_lock);
+	mm->mm_cid.nr_cpus_allowed = p->nr_cpus_allowed;
+	raw_spin_lock_init(&mm->mm_cid.lock);
 	cpumask_copy(mm_cpus_allowed(mm), &p->cpus_mask);
 	cpumask_clear(mm_cidmask(mm));
 }
 
 static inline int mm_alloc_cid_noprof(struct mm_struct *mm, struct task_struct *p)
 {
-	mm->pcpu_cid = alloc_percpu_noprof(struct mm_cid);
-	if (!mm->pcpu_cid)
+	mm->mm_cid.pcpu = alloc_percpu_noprof(struct mm_cid_pcpu);
+	if (!mm->mm_cid.pcpu)
 		return -ENOMEM;
 	mm_init_cid(mm, p);
 	return 0;
@@ -1405,8 +1378,8 @@ static inline int mm_alloc_cid_noprof(st
 
 static inline void mm_destroy_cid(struct mm_struct *mm)
 {
-	free_percpu(mm->pcpu_cid);
-	mm->pcpu_cid = NULL;
+	free_percpu(mm->mm_cid.pcpu);
+	mm->mm_cid.pcpu = NULL;
 }
 
 static inline unsigned int mm_cid_size(void)
@@ -1421,10 +1394,9 @@ static inline void mm_set_cpus_allowed(s
 	if (!mm)
 		return;
 	/* The mm_cpus_allowed is the union of each thread allowed CPUs masks. */
-	raw_spin_lock(&mm->cpus_allowed_lock);
+	guard(raw_spinlock)(&mm->mm_cid.lock);
 	cpumask_or(mm_allowed, mm_allowed, cpumask);
-	WRITE_ONCE(mm->nr_cpus_allowed, cpumask_weight(mm_allowed));
-	raw_spin_unlock(&mm->cpus_allowed_lock);
+	WRITE_ONCE(mm->mm_cid.nr_cpus_allowed, cpumask_weight(mm_allowed));
 }
 #else /* CONFIG_SCHED_MM_CID */
 static inline void mm_init_cid(struct mm_struct *mm, struct task_struct *p) { }
--- a/include/linux/rseq_types.h
+++ b/include/linux/rseq_types.h
@@ -90,4 +90,46 @@ struct rseq_data {
 struct rseq_data { };
 #endif /* !CONFIG_RSEQ */
 
+#ifdef CONFIG_SCHED_MM_CID
+
+#define MM_CID_UNSET	(~0U)
+
+/**
+ * struct sched_mm_cid - Storage for per task MM CID data
+ * @active:	MM CID is active for the task
+ * @cid:	The CID associated to the task
+ * @last_cid:	The last CID associated to the task
+ */
+struct sched_mm_cid {
+	unsigned int		active;
+	unsigned int		cid;
+	unsigned int		last_cid;
+};
+
+/**
+ * struct mm_cid_pcpu - Storage for per CPU MM_CID data
+ * @cid:	The CID associated to the CPU
+ */
+struct mm_cid_pcpu {
+	unsigned int	cid;
+};
+
+/**
+ * struct mm_mm_cid - Storage for per MM CID data
+ * @pcpu:		Per CPU storage for CIDs associated to a CPU
+ * @nr_cpus_allowed:	The number of CPUs in the per MM allowed CPUs map. The map
+ *			is growth only.
+ * @lock:		Spinlock to protect all fields except @pcpu. It also protects
+ *			the MM cid cpumask and the MM cidmask bitmap.
+ */
+struct mm_mm_cid {
+	struct mm_cid_pcpu	__percpu *pcpu;
+	unsigned int		nr_cpus_allowed;
+	raw_spinlock_t		lock;
+};
+#else /* CONFIG_SCHED_MM_CID */
+struct mm_mm_cid { };
+struct sched_mm_cid { };
+#endif /* !CONFIG_SCHED_MM_CID */
+
 #endif
--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -1407,14 +1407,7 @@ struct task_struct {
 #endif /* CONFIG_NUMA_BALANCING */
 
 	struct rseq_data		rseq;
-
-#ifdef CONFIG_SCHED_MM_CID
-	int				mm_cid;		/* Current cid in mm */
-	int				last_mm_cid;	/* Most recent cid in mm */
-	int				migrate_from_cpu;
-	int				mm_cid_active;	/* Whether cid bitmap is active */
-	struct callback_head		cid_work;
-#endif
+	struct sched_mm_cid		mm_cid;
 
 	struct tlbflush_unmap_batch	tlb_ubc;
 
@@ -2308,7 +2301,7 @@ void sched_mm_cid_fork(struct task_struc
 void sched_mm_cid_exit_signals(struct task_struct *t);
 static inline int task_mm_cid(struct task_struct *t)
 {
-	return t->mm_cid;
+	return t->mm_cid.cid;
 }
 #else
 static inline void sched_mm_cid_before_execve(struct task_struct *t) { }
--- a/init/init_task.c
+++ b/init/init_task.c
@@ -223,6 +223,9 @@ struct task_struct init_task __aligned(L
 #ifdef CONFIG_SECCOMP_FILTER
 	.seccomp	= { .filter_count = ATOMIC_INIT(0) },
 #endif
+#ifdef CONFIG_SCHED_MM_CID
+	.mm_cid		= { .cid = MM_CID_UNSET, },
+#endif
 };
 EXPORT_SYMBOL(init_task);
 
--- a/kernel/fork.c
+++ b/kernel/fork.c
@@ -955,9 +955,9 @@ static struct task_struct *dup_task_stru
 #endif
 
 #ifdef CONFIG_SCHED_MM_CID
-	tsk->mm_cid = MM_CID_UNSET;
-	tsk->last_mm_cid = MM_CID_UNSET;
-	tsk->mm_cid_active = 0;
+	tsk->mm_cid.cid = MM_CID_UNSET;
+	tsk->mm_cid.last_cid = MM_CID_UNSET;
+	tsk->mm_cid.active = 0;
 #endif
 	return tsk;
 
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -10379,14 +10379,14 @@ void sched_mm_cid_exit_signals(struct ta
 {
 	struct mm_struct *mm = t->mm;
 
-	if (!mm || !t->mm_cid_active)
+	if (!mm || !t->mm_cid.active)
 		return;
 
 	guard(preempt)();
-	t->mm_cid_active = 0;
-	if (t->mm_cid != MM_CID_UNSET) {
-		cpumask_clear_cpu(t->mm_cid, mm_cidmask(mm));
-		t->mm_cid = MM_CID_UNSET;
+	t->mm_cid.active = 0;
+	if (t->mm_cid.cid != MM_CID_UNSET) {
+		cpumask_clear_cpu(t->mm_cid.cid, mm_cidmask(mm));
+		t->mm_cid.cid = MM_CID_UNSET;
 	}
 }
 
@@ -10405,14 +10405,14 @@ void sched_mm_cid_after_execve(struct ta
 		return;
 
 	guard(preempt)();
-	t->mm_cid_active = 1;
+	t->mm_cid.active = 1;
 	mm_cid_select(t);
 }
 
 void sched_mm_cid_fork(struct task_struct *t)
 {
-	WARN_ON_ONCE(!t->mm || t->mm_cid != MM_CID_UNSET);
-	t->mm_cid_active = 1;
+	WARN_ON_ONCE(!t->mm || t->mm_cid.cid != MM_CID_UNSET);
+	t->mm_cid.active = 1;
 }
 #endif /* CONFIG_SCHED_MM_CID */
 
--- a/kernel/sched/sched.h
+++ b/kernel/sched/sched.h
@@ -3548,8 +3548,8 @@ static inline void init_sched_mm_cid(str
 		return;
 
 	/* Preset last_mm_cid */
-	max_cid = min_t(int, READ_ONCE(mm->nr_cpus_allowed), atomic_read(&mm->mm_users));
-	t->last_mm_cid = max_cid - 1;
+	max_cid = min_t(int, READ_ONCE(mm->mm_cid.nr_cpus_allowed), atomic_read(&mm->mm_users));
+	t->mm_cid.last_cid = max_cid - 1;
 }
 
 static inline bool __mm_cid_get(struct task_struct *t, unsigned int cid, unsigned int max_cids)
@@ -3560,8 +3560,8 @@ static inline bool __mm_cid_get(struct t
 		return false;
 	if (cpumask_test_and_set_cpu(cid, mm_cidmask(mm)))
 		return false;
-	t->mm_cid = t->last_mm_cid = cid;
-	__this_cpu_write(mm->pcpu_cid->cid, cid);
+	t->mm_cid.cid = t->mm_cid.last_cid = cid;
+	__this_cpu_write(mm->mm_cid.pcpu->cid, cid);
 	return true;
 }
 
@@ -3570,14 +3570,14 @@ static inline bool mm_cid_get(struct tas
 	struct mm_struct *mm = t->mm;
 	unsigned int max_cids;
 
-	max_cids = min_t(int, READ_ONCE(mm->nr_cpus_allowed), atomic_read(&mm->mm_users));
+	max_cids = min_t(int, READ_ONCE(mm->mm_cid.nr_cpus_allowed), atomic_read(&mm->mm_users));
 
 	/* Try to reuse the last CID of this task */
-	if (__mm_cid_get(t, t->last_mm_cid, max_cids))
+	if (__mm_cid_get(t, t->mm_cid.last_cid, max_cids))
 		return true;
 
 	/* Try to reuse the last CID of this mm on this CPU */
-	if (__mm_cid_get(t, __this_cpu_read(mm->pcpu_cid->cid), max_cids))
+	if (__mm_cid_get(t, __this_cpu_read(mm->mm_cid.pcpu->cid), max_cids))
 		return true;
 
 	/* Try the first zero bit in the cidmask. */
@@ -3600,15 +3600,15 @@ static inline void mm_cid_select(struct
 
 static inline void switch_mm_cid(struct task_struct *prev, struct task_struct *next)
 {
-	if (prev->mm_cid_active) {
-		if (prev->mm_cid != MM_CID_UNSET)
-			cpumask_clear_cpu(prev->mm_cid, mm_cidmask(prev->mm));
-		prev->mm_cid = MM_CID_UNSET;
+	if (prev->mm_cid.active) {
+		if (prev->mm_cid.cid != MM_CID_UNSET)
+			cpumask_clear_cpu(prev->mm_cid.cid, mm_cidmask(prev->mm));
+		prev->mm_cid.cid = MM_CID_UNSET;
 	}
 
-	if (next->mm_cid_active) {
+	if (next->mm_cid.active) {
 		mm_cid_select(next);
-		rseq_sched_set_task_mm_cid(next, next->mm_cid);
+		rseq_sched_set_task_mm_cid(next, next->mm_cid.cid);
 	}
 }
 


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

* [patch V3 03/20] sched/mmcid: Cacheline align MM CID storage
  2025-10-29 13:08 [patch V3 00/20] sched: Rewrite MM CID management Thomas Gleixner
  2025-10-29 13:08 ` [patch V3 01/20] sched/mmcid: Revert the complex " Thomas Gleixner
  2025-10-29 13:08 ` [patch V3 02/20] sched/mmcid: Use proper data structures Thomas Gleixner
@ 2025-10-29 13:08 ` Thomas Gleixner
  2025-10-29 15:39   ` Mathieu Desnoyers
  2025-10-29 13:09 ` [patch V3 04/20] sched: Fixup whitespace damage Thomas Gleixner
                   ` (18 subsequent siblings)
  21 siblings, 1 reply; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-29 13:08 UTC (permalink / raw)
  To: LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov, Shrikanth Hegde

Both the per CPU storage and the data in mm_struct are heavily used in
context switch. As they can end up next to other frequently modified data,
they are subject to false sharing.

Make them cache line aligned.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 include/linux/rseq_types.h |    4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

--- a/include/linux/rseq_types.h
+++ b/include/linux/rseq_types.h
@@ -112,7 +112,7 @@ struct sched_mm_cid {
  */
 struct mm_cid_pcpu {
 	unsigned int	cid;
-};
+}____cacheline_aligned_in_smp;
 
 /**
  * struct mm_mm_cid - Storage for per MM CID data
@@ -126,7 +126,7 @@ struct mm_mm_cid {
 	struct mm_cid_pcpu	__percpu *pcpu;
 	unsigned int		nr_cpus_allowed;
 	raw_spinlock_t		lock;
-};
+}____cacheline_aligned_in_smp;
 #else /* CONFIG_SCHED_MM_CID */
 struct mm_mm_cid { };
 struct sched_mm_cid { };


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

* [patch V3 04/20] sched: Fixup whitespace damage
  2025-10-29 13:08 [patch V3 00/20] sched: Rewrite MM CID management Thomas Gleixner
                   ` (2 preceding siblings ...)
  2025-10-29 13:08 ` [patch V3 03/20] sched/mmcid: Cacheline align MM CID storage Thomas Gleixner
@ 2025-10-29 13:09 ` Thomas Gleixner
  2025-10-29 15:42   ` Mathieu Desnoyers
  2025-10-29 13:09 ` [patch V3 05/20] sched/mmcid: Move scheduler code out of global header Thomas Gleixner
                   ` (17 subsequent siblings)
  21 siblings, 1 reply; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-29 13:09 UTC (permalink / raw)
  To: LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov, Shrikanth Hegde

With whitespace checks enabled in the editor this makes eyes bleed.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 kernel/sched/core.c |   11 ++++-------
 1 file changed, 4 insertions(+), 7 deletions(-)

--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -5277,19 +5277,16 @@ context_switch(struct rq *rq, struct tas
 	 *
 	 * kernel ->   user   switch + mmdrop_lazy_tlb() active
 	 *   user ->   user   switch
-	 *
-	 * switch_mm_cid() needs to be updated if the barriers provided
-	 * by context_switch() are modified.
 	 */
-	if (!next->mm) {                                // to kernel
+	if (!next->mm) {				// to kernel
 		enter_lazy_tlb(prev->active_mm, next);
 
 		next->active_mm = prev->active_mm;
-		if (prev->mm)                           // from user
+		if (prev->mm)				// from user
 			mmgrab_lazy_tlb(prev->active_mm);
 		else
 			prev->active_mm = NULL;
-	} else {                                        // to user
+	} else {					// to user
 		membarrier_switch_mm(rq, prev->active_mm, next->mm);
 		/*
 		 * sys_membarrier() requires an smp_mb() between setting
@@ -5302,7 +5299,7 @@ context_switch(struct rq *rq, struct tas
 		switch_mm_irqs_off(prev->active_mm, next->mm, next);
 		lru_gen_use_mm(next->mm);
 
-		if (!prev->mm) {                        // from kernel
+		if (!prev->mm) {			// from kernel
 			/* will mmdrop_lazy_tlb() in finish_task_switch(). */
 			rq->prev_mm = prev->active_mm;
 			prev->active_mm = NULL;


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

* [patch V3 05/20] sched/mmcid: Move scheduler code out of global header
  2025-10-29 13:08 [patch V3 00/20] sched: Rewrite MM CID management Thomas Gleixner
                   ` (3 preceding siblings ...)
  2025-10-29 13:09 ` [patch V3 04/20] sched: Fixup whitespace damage Thomas Gleixner
@ 2025-10-29 13:09 ` Thomas Gleixner
  2025-10-29 15:43   ` Mathieu Desnoyers
  2025-10-29 13:09 ` [patch V3 06/20] sched/mmcid: Prevent pointless work in mm_update_cpus_allowed() Thomas Gleixner
                   ` (16 subsequent siblings)
  21 siblings, 1 reply; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-29 13:09 UTC (permalink / raw)
  To: LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov, Shrikanth Hegde

This is only used in the scheduler core code, so there is no point to have
it in a global header.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 include/linux/mm_types.h |   13 -------------
 kernel/sched/core.c      |   20 ++++++++++++++++++--
 2 files changed, 18 insertions(+), 15 deletions(-)

--- a/include/linux/mm_types.h
+++ b/include/linux/mm_types.h
@@ -1387,27 +1387,14 @@ static inline unsigned int mm_cid_size(v
 	return 2 * cpumask_size();	/* mm_cpus_allowed(), mm_cidmask(). */
 }
 
-static inline void mm_set_cpus_allowed(struct mm_struct *mm, const struct cpumask *cpumask)
-{
-	struct cpumask *mm_allowed = mm_cpus_allowed(mm);
-
-	if (!mm)
-		return;
-	/* The mm_cpus_allowed is the union of each thread allowed CPUs masks. */
-	guard(raw_spinlock)(&mm->mm_cid.lock);
-	cpumask_or(mm_allowed, mm_allowed, cpumask);
-	WRITE_ONCE(mm->mm_cid.nr_cpus_allowed, cpumask_weight(mm_allowed));
-}
 #else /* CONFIG_SCHED_MM_CID */
 static inline void mm_init_cid(struct mm_struct *mm, struct task_struct *p) { }
 static inline int mm_alloc_cid(struct mm_struct *mm, struct task_struct *p) { return 0; }
 static inline void mm_destroy_cid(struct mm_struct *mm) { }
-
 static inline unsigned int mm_cid_size(void)
 {
 	return 0;
 }
-static inline void mm_set_cpus_allowed(struct mm_struct *mm, const struct cpumask *cpumask) { }
 #endif /* CONFIG_SCHED_MM_CID */
 
 struct mmu_gather;
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -2669,6 +2669,8 @@ int push_cpu_stop(void *arg)
 	return 0;
 }
 
+static inline void mm_update_cpus_allowed(struct mm_struct *mm, const cpumask_t *affmask);
+
 /*
  * sched_class::set_cpus_allowed must do the below, but is not required to
  * actually call this function.
@@ -2728,7 +2730,7 @@ static void
 		put_prev_task(rq, p);
 
 	p->sched_class->set_cpus_allowed(p, ctx);
-	mm_set_cpus_allowed(p->mm, ctx->new_mask);
+	mm_update_cpus_allowed(p->mm, ctx->new_mask);
 
 	if (queued)
 		enqueue_task(rq, p, ENQUEUE_RESTORE | ENQUEUE_NOCLOCK);
@@ -10370,6 +10372,18 @@ void call_trace_sched_update_nr_running(
  * When a task exits, the MM CID held by the task is not longer required as
  * the task cannot return to user space.
  */
+static inline void mm_update_cpus_allowed(struct mm_struct *mm, const struct cpumask *affmsk)
+{
+	struct cpumask *mm_allowed = mm_cpus_allowed(mm);
+
+	if (!mm)
+		return;
+	/* The mm_cpus_allowed is the union of each thread allowed CPUs masks. */
+	guard(raw_spinlock)(&mm->mm_cid.lock);
+	cpumask_or(mm_allowed, mm_allowed, affmsk);
+	WRITE_ONCE(mm->mm_cid.nr_cpus_allowed, cpumask_weight(mm_allowed));
+}
+
 void sched_mm_cid_exit_signals(struct task_struct *t)
 {
 	struct mm_struct *mm = t->mm;
@@ -10409,7 +10423,9 @@ void sched_mm_cid_fork(struct task_struc
 	WARN_ON_ONCE(!t->mm || t->mm_cid.cid != MM_CID_UNSET);
 	t->mm_cid.active = 1;
 }
-#endif /* CONFIG_SCHED_MM_CID */
+#else /* CONFIG_SCHED_MM_CID */
+static inline void mm_update_cpus_allowed(struct mm_struct *mm, const struct cpumask *affmsk) { }
+#endif /* !CONFIG_SCHED_MM_CID */
 
 #ifdef CONFIG_SCHED_CLASS_EXT
 void sched_deq_and_put_task(struct task_struct *p, int queue_flags,


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

* [patch V3 06/20] sched/mmcid: Prevent pointless work in mm_update_cpus_allowed()
  2025-10-29 13:08 [patch V3 00/20] sched: Rewrite MM CID management Thomas Gleixner
                   ` (4 preceding siblings ...)
  2025-10-29 13:09 ` [patch V3 05/20] sched/mmcid: Move scheduler code out of global header Thomas Gleixner
@ 2025-10-29 13:09 ` Thomas Gleixner
  2025-10-29 15:45   ` Mathieu Desnoyers
  2025-10-29 13:09 ` [patch V3 07/20] cpumask: Introduce cpumask_weighted_or() Thomas Gleixner
                   ` (15 subsequent siblings)
  21 siblings, 1 reply; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-29 13:09 UTC (permalink / raw)
  To: LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov, Shrikanth Hegde

mm_update_cpus_allowed() is not required to be invoked for affinity changes
due to migrate_disable() and migrate_enable().

migrate_disable() restricts the task temporarily to a CPU on which the task
was already allowed to run, so nothing changes. migrate_enable() restores
the actual task affinity mask.

If that mask changed between migrate_disable() and migrate_enable() then
that change was already accounted for.

Move the invocation to the proper place to avoid that.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
V2: Remove the nr_cpu_ids optimization which does not really work - PeterZ
---
 kernel/sched/core.c |   11 ++++++++---
 1 file changed, 8 insertions(+), 3 deletions(-)

--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -2684,6 +2684,7 @@ void set_cpus_allowed_common(struct task
 
 	cpumask_copy(&p->cpus_mask, ctx->new_mask);
 	p->nr_cpus_allowed = cpumask_weight(ctx->new_mask);
+	mm_update_cpus_allowed(p->mm, ctx->new_mask);
 
 	/*
 	 * Swap in a new user_cpus_ptr if SCA_USER flag set
@@ -2730,7 +2731,6 @@ static void
 		put_prev_task(rq, p);
 
 	p->sched_class->set_cpus_allowed(p, ctx);
-	mm_update_cpus_allowed(p->mm, ctx->new_mask);
 
 	if (queued)
 		enqueue_task(rq, p, ENQUEUE_RESTORE | ENQUEUE_NOCLOCK);
@@ -10374,12 +10374,17 @@ void call_trace_sched_update_nr_running(
  */
 static inline void mm_update_cpus_allowed(struct mm_struct *mm, const struct cpumask *affmsk)
 {
-	struct cpumask *mm_allowed = mm_cpus_allowed(mm);
+	struct cpumask *mm_allowed;
 
 	if (!mm)
 		return;
-	/* The mm_cpus_allowed is the union of each thread allowed CPUs masks. */
+
+	/*
+	 * mm::mm_cid::mm_cpus_allowed is the superset of each threads
+	 * allowed CPUs mask which means it can only grow.
+	 */
 	guard(raw_spinlock)(&mm->mm_cid.lock);
+	mm_allowed = mm_cpus_allowed(mm);
 	cpumask_or(mm_allowed, mm_allowed, affmsk);
 	WRITE_ONCE(mm->mm_cid.nr_cpus_allowed, cpumask_weight(mm_allowed));
 }


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

* [patch V3 07/20] cpumask: Introduce cpumask_weighted_or()
  2025-10-29 13:08 [patch V3 00/20] sched: Rewrite MM CID management Thomas Gleixner
                   ` (5 preceding siblings ...)
  2025-10-29 13:09 ` [patch V3 06/20] sched/mmcid: Prevent pointless work in mm_update_cpus_allowed() Thomas Gleixner
@ 2025-10-29 13:09 ` Thomas Gleixner
  2025-10-29 15:49   ` Mathieu Desnoyers
  2025-11-03  9:15   ` Shrikanth Hegde
  2025-10-29 13:09 ` [patch V3 08/20] sched/mmcid: Use cpumask_weighted_or() Thomas Gleixner
                   ` (14 subsequent siblings)
  21 siblings, 2 replies; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-29 13:09 UTC (permalink / raw)
  To: LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov, Shrikanth Hegde

CID management OR's two cpumasks and then calculates the weight on the
result. That's inefficient as that has to walk the same stuff twice. As
this is done with runqueue lock held, there is a real benefit of speeding
this up. Depending on the system this results in 10-20% less cycles spent
with runqueue lock held for a 4K cpumask.

Provide cpumask_weighted_or() and the corresponding bitmap functions which
return the weight of the OR result right away.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Yury Norov (NVIDIA) <yury.norov@gmail.com>
---
V3: Rename again - Yury
V2: Rename and use the BITMAP_WEIGHT() macro - Yury
---
 include/linux/bitmap.h  |   15 +++++++++++++++
 include/linux/cpumask.h |   16 ++++++++++++++++
 lib/bitmap.c            |    6 ++++++
 3 files changed, 37 insertions(+)

--- a/include/linux/bitmap.h
+++ b/include/linux/bitmap.h
@@ -45,6 +45,7 @@ struct device;
  *  bitmap_copy(dst, src, nbits)                *dst = *src
  *  bitmap_and(dst, src1, src2, nbits)          *dst = *src1 & *src2
  *  bitmap_or(dst, src1, src2, nbits)           *dst = *src1 | *src2
+ *  bitmap_weighted_or(dst, src1, src2, nbits)	*dst = *src1 | *src2. Returns Hamming Weight of dst
  *  bitmap_xor(dst, src1, src2, nbits)          *dst = *src1 ^ *src2
  *  bitmap_andnot(dst, src1, src2, nbits)       *dst = *src1 & ~(*src2)
  *  bitmap_complement(dst, src, nbits)          *dst = ~(*src)
@@ -165,6 +166,8 @@ bool __bitmap_and(unsigned long *dst, co
 		 const unsigned long *bitmap2, unsigned int nbits);
 void __bitmap_or(unsigned long *dst, const unsigned long *bitmap1,
 		 const unsigned long *bitmap2, unsigned int nbits);
+unsigned int __bitmap_weighted_or(unsigned long *dst, const unsigned long *bitmap1,
+				  const unsigned long *bitmap2, unsigned int nbits);
 void __bitmap_xor(unsigned long *dst, const unsigned long *bitmap1,
 		  const unsigned long *bitmap2, unsigned int nbits);
 bool __bitmap_andnot(unsigned long *dst, const unsigned long *bitmap1,
@@ -338,6 +341,18 @@ void bitmap_or(unsigned long *dst, const
 }
 
 static __always_inline
+unsigned int bitmap_weighted_or(unsigned long *dst, const unsigned long *src1,
+				const unsigned long *src2, unsigned int nbits)
+{
+	if (small_const_nbits(nbits)) {
+		*dst = *src1 | *src2;
+		return hweight_long(*dst & BITMAP_LAST_WORD_MASK(nbits));
+	} else {
+		return __bitmap_weighted_or(dst, src1, src2, nbits);
+	}
+}
+
+static __always_inline
 void bitmap_xor(unsigned long *dst, const unsigned long *src1,
 		const unsigned long *src2, unsigned int nbits)
 {
--- a/include/linux/cpumask.h
+++ b/include/linux/cpumask.h
@@ -729,6 +729,22 @@ void cpumask_or(struct cpumask *dstp, co
 }
 
 /**
+ * cpumask_weighted_or - *dstp = *src1p | *src2p and return the weight of the result
+ * @dstp: the cpumask result
+ * @src1p: the first input
+ * @src2p: the second input
+ *
+ * Return: The number of bits set in the resulting cpumask @dstp
+ */
+static __always_inline
+unsigned int cpumask_weighted_or(struct cpumask *dstp, const struct cpumask *src1p,
+				 const struct cpumask *src2p)
+{
+	return bitmap_weighted_or(cpumask_bits(dstp), cpumask_bits(src1p),
+				  cpumask_bits(src2p), small_cpumask_bits);
+}
+
+/**
  * cpumask_xor - *dstp = *src1p ^ *src2p
  * @dstp: the cpumask result
  * @src1p: the first input
--- a/lib/bitmap.c
+++ b/lib/bitmap.c
@@ -355,6 +355,12 @@ unsigned int __bitmap_weight_andnot(cons
 }
 EXPORT_SYMBOL(__bitmap_weight_andnot);
 
+unsigned int __bitmap_weighted_or(unsigned long *dst, const unsigned long *bitmap1,
+				  const unsigned long *bitmap2, unsigned int bits)
+{
+	return BITMAP_WEIGHT(({dst[idx] = bitmap1[idx] | bitmap2[idx]; dst[idx]; }), bits);
+}
+
 void __bitmap_set(unsigned long *map, unsigned int start, int len)
 {
 	unsigned long *p = map + BIT_WORD(start);


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

* [patch V3 08/20] sched/mmcid: Use cpumask_weighted_or()
  2025-10-29 13:08 [patch V3 00/20] sched: Rewrite MM CID management Thomas Gleixner
                   ` (6 preceding siblings ...)
  2025-10-29 13:09 ` [patch V3 07/20] cpumask: Introduce cpumask_weighted_or() Thomas Gleixner
@ 2025-10-29 13:09 ` Thomas Gleixner
  2025-10-29 15:51   ` Mathieu Desnoyers
  2025-10-29 13:09 ` [patch V3 09/20] cpumask: Cache num_possible_cpus() Thomas Gleixner
                   ` (13 subsequent siblings)
  21 siblings, 1 reply; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-29 13:09 UTC (permalink / raw)
  To: LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov, Shrikanth Hegde

Use cpumask_weighted_or() instead of cpumask_or() and cpumask_weight() on
the result, which walks the same bitmap twice. Results in 10-20% less
cycles, which reduces the runqueue lock hold time.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 kernel/sched/core.c |    5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)

--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -10375,6 +10375,7 @@ void call_trace_sched_update_nr_running(
 static inline void mm_update_cpus_allowed(struct mm_struct *mm, const struct cpumask *affmsk)
 {
 	struct cpumask *mm_allowed;
+	unsigned int weight;
 
 	if (!mm)
 		return;
@@ -10385,8 +10386,8 @@ static inline void mm_update_cpus_allowe
 	 */
 	guard(raw_spinlock)(&mm->mm_cid.lock);
 	mm_allowed = mm_cpus_allowed(mm);
-	cpumask_or(mm_allowed, mm_allowed, affmsk);
-	WRITE_ONCE(mm->mm_cid.nr_cpus_allowed, cpumask_weight(mm_allowed));
+	weight = cpumask_weighted_or(mm_allowed, mm_allowed, affmsk);
+	WRITE_ONCE(mm->mm_cid.nr_cpus_allowed, weight);
 }
 
 void sched_mm_cid_exit_signals(struct task_struct *t)


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

* [patch V3 09/20] cpumask: Cache num_possible_cpus()
  2025-10-29 13:08 [patch V3 00/20] sched: Rewrite MM CID management Thomas Gleixner
                   ` (7 preceding siblings ...)
  2025-10-29 13:09 ` [patch V3 08/20] sched/mmcid: Use cpumask_weighted_or() Thomas Gleixner
@ 2025-10-29 13:09 ` Thomas Gleixner
  2025-10-29 15:54   ` Mathieu Desnoyers
  2025-11-03 10:06   ` Shrikanth Hegde
  2025-10-29 13:09 ` [patch V3 10/20] sched/mmcid: Convert mm CID mask to a bitmap Thomas Gleixner
                   ` (12 subsequent siblings)
  21 siblings, 2 replies; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-29 13:09 UTC (permalink / raw)
  To: LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov, Shrikanth Hegde

Reevaluating num_possible_cpus() over and over does not make sense. That
becomes a constant after init as cpu_possible_mask is marked ro_after_init.

Cache the value during initialization and provide that for consumption.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Yury Norov <yury.norov@gmail.com>
---
V2: New patch
---
 include/linux/cpumask.h |   10 ++++++++--
 kernel/cpu.c            |   15 +++++++++++++++
 2 files changed, 23 insertions(+), 2 deletions(-)

--- a/include/linux/cpumask.h
+++ b/include/linux/cpumask.h
@@ -126,6 +126,7 @@ extern struct cpumask __cpu_dying_mask;
 #define cpu_dying_mask    ((const struct cpumask *)&__cpu_dying_mask)
 
 extern atomic_t __num_online_cpus;
+extern unsigned int __num_possible_cpus;
 
 extern cpumask_t cpus_booted_once_mask;
 
@@ -1152,13 +1153,13 @@ void init_cpu_possible(const struct cpum
 #define __assign_cpu(cpu, mask, val)	\
 	__assign_bit(cpumask_check(cpu), cpumask_bits(mask), (val))
 
-#define set_cpu_possible(cpu, possible)	assign_cpu((cpu), &__cpu_possible_mask, (possible))
 #define set_cpu_enabled(cpu, enabled)	assign_cpu((cpu), &__cpu_enabled_mask, (enabled))
 #define set_cpu_present(cpu, present)	assign_cpu((cpu), &__cpu_present_mask, (present))
 #define set_cpu_active(cpu, active)	assign_cpu((cpu), &__cpu_active_mask, (active))
 #define set_cpu_dying(cpu, dying)	assign_cpu((cpu), &__cpu_dying_mask, (dying))
 
 void set_cpu_online(unsigned int cpu, bool online);
+void set_cpu_possible(unsigned int cpu, bool possible);
 
 /**
  * to_cpumask - convert a NR_CPUS bitmap to a struct cpumask *
@@ -1211,7 +1212,12 @@ static __always_inline unsigned int num_
 {
 	return raw_atomic_read(&__num_online_cpus);
 }
-#define num_possible_cpus()	cpumask_weight(cpu_possible_mask)
+
+static __always_inline unsigned int num_possible_cpus(void)
+{
+	return __num_possible_cpus;
+}
+
 #define num_enabled_cpus()	cpumask_weight(cpu_enabled_mask)
 #define num_present_cpus()	cpumask_weight(cpu_present_mask)
 #define num_active_cpus()	cpumask_weight(cpu_active_mask)
--- a/kernel/cpu.c
+++ b/kernel/cpu.c
@@ -3108,6 +3108,9 @@ EXPORT_SYMBOL(__cpu_dying_mask);
 atomic_t __num_online_cpus __read_mostly;
 EXPORT_SYMBOL(__num_online_cpus);
 
+unsigned int __num_possible_cpus __ro_after_init = NR_CPUS;
+EXPORT_SYMBOL(__num_possible_cpus);
+
 void init_cpu_present(const struct cpumask *src)
 {
 	cpumask_copy(&__cpu_present_mask, src);
@@ -3116,6 +3119,7 @@ void init_cpu_present(const struct cpuma
 void init_cpu_possible(const struct cpumask *src)
 {
 	cpumask_copy(&__cpu_possible_mask, src);
+	__num_possible_cpus = cpumask_weight(&__cpu_possible_mask);
 }
 
 void set_cpu_online(unsigned int cpu, bool online)
@@ -3139,6 +3143,17 @@ void set_cpu_online(unsigned int cpu, bo
 	}
 }
 
+void set_cpu_possible(unsigned int cpu, bool possible)
+{
+	if (possible) {
+		if (!cpumask_test_and_set_cpu(cpu, &__cpu_possible_mask))
+			__num_possible_cpus++;
+	} else {
+		if (cpumask_test_and_clear_cpu(cpu, &__cpu_possible_mask))
+			__num_possible_cpus--;
+	}
+}
+
 /*
  * Activate the first processor.
  */


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

* [patch V3 10/20] sched/mmcid: Convert mm CID mask to a bitmap
  2025-10-29 13:08 [patch V3 00/20] sched: Rewrite MM CID management Thomas Gleixner
                   ` (8 preceding siblings ...)
  2025-10-29 13:09 ` [patch V3 09/20] cpumask: Cache num_possible_cpus() Thomas Gleixner
@ 2025-10-29 13:09 ` Thomas Gleixner
  2025-10-30 13:59   ` Mathieu Desnoyers
  2025-10-29 13:09 ` [patch V3 11/20] signal: Move MMCID exit out of sighand lock Thomas Gleixner
                   ` (11 subsequent siblings)
  21 siblings, 1 reply; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-29 13:09 UTC (permalink / raw)
  To: LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov, Shrikanth Hegde

This is truly a bitmap and just conveniently uses a cpumask because the
maximum size of the bitmap is nr_cpu_ids.

But that prevents to do searches for a zero bit in a limited range, which
is helpful to provide an efficient mechanism to consolidate the CID space
when the number of users decreases.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Acked-by: Yury Norov (NVIDIA) <yury.norov@gmail.com>
---
V3: Zero the bitmap with length of bitmap_size(nr_possible_cpus()) - Shrikanth
---
 include/linux/mm_types.h |    6 +++---
 kernel/sched/core.c      |    2 +-
 kernel/sched/sched.h     |    6 +++---
 3 files changed, 7 insertions(+), 7 deletions(-)

--- a/include/linux/mm_types.h
+++ b/include/linux/mm_types.h
@@ -1342,13 +1342,13 @@ static inline cpumask_t *mm_cpus_allowed
 }
 
 /* Accessor for struct mm_struct's cidmask. */
-static inline cpumask_t *mm_cidmask(struct mm_struct *mm)
+static inline unsigned long *mm_cidmask(struct mm_struct *mm)
 {
 	unsigned long cid_bitmap = (unsigned long)mm_cpus_allowed(mm);
 
 	/* Skip mm_cpus_allowed */
 	cid_bitmap += cpumask_size();
-	return (struct cpumask *)cid_bitmap;
+	return (unsigned long *)cid_bitmap;
 }
 
 static inline void mm_init_cid(struct mm_struct *mm, struct task_struct *p)
@@ -1363,7 +1363,7 @@ static inline void mm_init_cid(struct mm
 	mm->mm_cid.nr_cpus_allowed = p->nr_cpus_allowed;
 	raw_spin_lock_init(&mm->mm_cid.lock);
 	cpumask_copy(mm_cpus_allowed(mm), &p->cpus_mask);
-	cpumask_clear(mm_cidmask(mm));
+	bitmap_zero(mm_cidmask(mm), bitmap_size(num_possible_cpus()));
 }
 
 static inline int mm_alloc_cid_noprof(struct mm_struct *mm, struct task_struct *p)
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -10400,7 +10400,7 @@ void sched_mm_cid_exit_signals(struct ta
 	guard(preempt)();
 	t->mm_cid.active = 0;
 	if (t->mm_cid.cid != MM_CID_UNSET) {
-		cpumask_clear_cpu(t->mm_cid.cid, mm_cidmask(mm));
+		clear_bit(t->mm_cid.cid, mm_cidmask(mm));
 		t->mm_cid.cid = MM_CID_UNSET;
 	}
 }
--- a/kernel/sched/sched.h
+++ b/kernel/sched/sched.h
@@ -3558,7 +3558,7 @@ static inline bool __mm_cid_get(struct t
 
 	if (cid >= max_cids)
 		return false;
-	if (cpumask_test_and_set_cpu(cid, mm_cidmask(mm)))
+	if (test_and_set_bit(cid, mm_cidmask(mm)))
 		return false;
 	t->mm_cid.cid = t->mm_cid.last_cid = cid;
 	__this_cpu_write(mm->mm_cid.pcpu->cid, cid);
@@ -3581,7 +3581,7 @@ static inline bool mm_cid_get(struct tas
 		return true;
 
 	/* Try the first zero bit in the cidmask. */
-	return __mm_cid_get(t, cpumask_first_zero(mm_cidmask(mm)), max_cids);
+	return __mm_cid_get(t, find_first_zero_bit(mm_cidmask(mm), num_possible_cpus()), max_cids);
 }
 
 static inline void mm_cid_select(struct task_struct *t)
@@ -3602,7 +3602,7 @@ static inline void switch_mm_cid(struct
 {
 	if (prev->mm_cid.active) {
 		if (prev->mm_cid.cid != MM_CID_UNSET)
-			cpumask_clear_cpu(prev->mm_cid.cid, mm_cidmask(prev->mm));
+			clear_bit(prev->mm_cid.cid, mm_cidmask(prev->mm));
 		prev->mm_cid.cid = MM_CID_UNSET;
 	}
 


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

* [patch V3 11/20] signal: Move MMCID exit out of sighand lock
  2025-10-29 13:08 [patch V3 00/20] sched: Rewrite MM CID management Thomas Gleixner
                   ` (9 preceding siblings ...)
  2025-10-29 13:09 ` [patch V3 10/20] sched/mmcid: Convert mm CID mask to a bitmap Thomas Gleixner
@ 2025-10-29 13:09 ` Thomas Gleixner
  2025-10-29 13:09 ` [patch V3 12/20] sched/mmcid: Move initialization out of line Thomas Gleixner
                   ` (10 subsequent siblings)
  21 siblings, 0 replies; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-29 13:09 UTC (permalink / raw)
  To: LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov, Shrikanth Hegde

There is no need anymore to keep this under sighand lock as the current
code and the upcoming replacement are not depending on the exit state of a
task anymore.

That allows to use a mutex in the exit path.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 include/linux/sched.h |    4 ++--
 kernel/exit.c         |    1 +
 kernel/sched/core.c   |    4 ++--
 kernel/signal.c       |    2 --
 4 files changed, 5 insertions(+), 6 deletions(-)

--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -2298,7 +2298,7 @@ static __always_inline void alloc_tag_re
 void sched_mm_cid_before_execve(struct task_struct *t);
 void sched_mm_cid_after_execve(struct task_struct *t);
 void sched_mm_cid_fork(struct task_struct *t);
-void sched_mm_cid_exit_signals(struct task_struct *t);
+void sched_mm_cid_exit(struct task_struct *t);
 static inline int task_mm_cid(struct task_struct *t)
 {
 	return t->mm_cid.cid;
@@ -2307,7 +2307,7 @@ static inline int task_mm_cid(struct tas
 static inline void sched_mm_cid_before_execve(struct task_struct *t) { }
 static inline void sched_mm_cid_after_execve(struct task_struct *t) { }
 static inline void sched_mm_cid_fork(struct task_struct *t) { }
-static inline void sched_mm_cid_exit_signals(struct task_struct *t) { }
+static inline void sched_mm_cid_exit(struct task_struct *t) { }
 static inline int task_mm_cid(struct task_struct *t)
 {
 	/*
--- a/kernel/exit.c
+++ b/kernel/exit.c
@@ -910,6 +910,7 @@ void __noreturn do_exit(long code)
 	user_events_exit(tsk);
 
 	io_uring_files_cancel();
+	sched_mm_cid_exit(tsk);
 	exit_signals(tsk);  /* sets PF_EXITING */
 
 	seccomp_filter_release(tsk);
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -10392,7 +10392,7 @@ static inline void mm_update_cpus_allowe
 	WRITE_ONCE(mm->mm_cid.nr_cpus_allowed, weight);
 }
 
-void sched_mm_cid_exit_signals(struct task_struct *t)
+void sched_mm_cid_exit(struct task_struct *t)
 {
 	struct mm_struct *mm = t->mm;
 
@@ -10410,7 +10410,7 @@ void sched_mm_cid_exit_signals(struct ta
 /* Deactivate MM CID allocation across execve() */
 void sched_mm_cid_before_execve(struct task_struct *t)
 {
-	sched_mm_cid_exit_signals(t);
+	sched_mm_cid_exit(t);
 }
 
 /* Reactivate MM CID after successful execve() */
--- a/kernel/signal.c
+++ b/kernel/signal.c
@@ -3125,7 +3125,6 @@ void exit_signals(struct task_struct *ts
 	cgroup_threadgroup_change_begin(tsk);
 
 	if (thread_group_empty(tsk) || (tsk->signal->flags & SIGNAL_GROUP_EXIT)) {
-		sched_mm_cid_exit_signals(tsk);
 		tsk->flags |= PF_EXITING;
 		cgroup_threadgroup_change_end(tsk);
 		return;
@@ -3136,7 +3135,6 @@ void exit_signals(struct task_struct *ts
 	 * From now this task is not visible for group-wide signals,
 	 * see wants_signal(), do_signal_stop().
 	 */
-	sched_mm_cid_exit_signals(tsk);
 	tsk->flags |= PF_EXITING;
 
 	cgroup_threadgroup_change_end(tsk);


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

* [patch V3 12/20] sched/mmcid: Move initialization out of line
  2025-10-29 13:08 [patch V3 00/20] sched: Rewrite MM CID management Thomas Gleixner
                   ` (10 preceding siblings ...)
  2025-10-29 13:09 ` [patch V3 11/20] signal: Move MMCID exit out of sighand lock Thomas Gleixner
@ 2025-10-29 13:09 ` Thomas Gleixner
  2025-10-30 14:02   ` Mathieu Desnoyers
  2025-10-29 13:09 ` [patch V3 13/20] sched/mmcid: Provide precomputed maximal value Thomas Gleixner
                   ` (9 subsequent siblings)
  21 siblings, 1 reply; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-29 13:09 UTC (permalink / raw)
  To: LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov, Shrikanth Hegde

It's getting bigger soon, so just move it out of line to the rest of the
code.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 include/linux/mm_types.h |   15 +--------------
 kernel/sched/core.c      |   14 ++++++++++++++
 2 files changed, 15 insertions(+), 14 deletions(-)

--- a/include/linux/mm_types.h
+++ b/include/linux/mm_types.h
@@ -1351,20 +1351,7 @@ static inline unsigned long *mm_cidmask(
 	return (unsigned long *)cid_bitmap;
 }
 
-static inline void mm_init_cid(struct mm_struct *mm, struct task_struct *p)
-{
-	int i;
-
-	for_each_possible_cpu(i) {
-		struct mm_cid_pcpu *pcpu = per_cpu_ptr(mm->mm_cid.pcpu, i);
-
-		pcpu->cid = MM_CID_UNSET;
-	}
-	mm->mm_cid.nr_cpus_allowed = p->nr_cpus_allowed;
-	raw_spin_lock_init(&mm->mm_cid.lock);
-	cpumask_copy(mm_cpus_allowed(mm), &p->cpus_mask);
-	bitmap_zero(mm_cidmask(mm), bitmap_size(num_possible_cpus()));
-}
+void mm_init_cid(struct mm_struct *mm, struct task_struct *p);
 
 static inline int mm_alloc_cid_noprof(struct mm_struct *mm, struct task_struct *p)
 {
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -10429,6 +10429,20 @@ void sched_mm_cid_fork(struct task_struc
 	WARN_ON_ONCE(!t->mm || t->mm_cid.cid != MM_CID_UNSET);
 	t->mm_cid.active = 1;
 }
+
+void mm_init_cid(struct mm_struct *mm, struct task_struct *p)
+{
+	struct mm_cid_pcpu __percpu *pcpu = mm->mm_cid.pcpu;
+	int cpu;
+
+	for_each_possible_cpu(cpu)
+		per_cpu_ptr(pcpu, cpu)->cid = MM_CID_UNSET;
+
+	mm->mm_cid.nr_cpus_allowed = p->nr_cpus_allowed;
+	raw_spin_lock_init(&mm->mm_cid.lock);
+	cpumask_copy(mm_cpus_allowed(mm), &p->cpus_mask);
+	bitmap_zero(mm_cidmask(mm), bitmap_size(num_possible_cpus()));
+}
 #else /* CONFIG_SCHED_MM_CID */
 static inline void mm_update_cpus_allowed(struct mm_struct *mm, const struct cpumask *affmsk) { }
 #endif /* !CONFIG_SCHED_MM_CID */


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

* [patch V3 13/20] sched/mmcid: Provide precomputed maximal value
  2025-10-29 13:08 [patch V3 00/20] sched: Rewrite MM CID management Thomas Gleixner
                   ` (11 preceding siblings ...)
  2025-10-29 13:09 ` [patch V3 12/20] sched/mmcid: Move initialization out of line Thomas Gleixner
@ 2025-10-29 13:09 ` Thomas Gleixner
  2025-10-30 14:23   ` Mathieu Desnoyers
  2025-10-29 13:09 ` [patch V3 14/20] sched/mmcid: Serialize sched_mm_cid_fork()/exit() with a mutex Thomas Gleixner
                   ` (8 subsequent siblings)
  21 siblings, 1 reply; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-29 13:09 UTC (permalink / raw)
  To: LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov, Shrikanth Hegde

Reading mm::mm_users and mm:::mm_cid::nr_cpus_allowed everytime to compute
the maximal CID value is just wasteful as that value is only changing on
fork(), exit() and eventually when the affinity changes.

So it can be easily precomputed at those points and provided in mm::mm_cid
for consumption in the hot path.

But there is an issue with using mm::mm_users for accounting because that
does not necessarily reflect the number of user space tasks as other kernel
code can take temporary references on the MM which skew the picture.

Solve that by adding a users counter to struct mm_mm_cid, which is modified
by fork() and exit() and used for precomputing under mm_mm_cid::lock.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 include/linux/rseq_types.h |    6 ++++
 kernel/fork.c              |    1 
 kernel/sched/core.c        |   59 ++++++++++++++++++++++++++++++++-------------
 kernel/sched/sched.h       |    3 --
 4 files changed, 50 insertions(+), 19 deletions(-)

--- a/include/linux/rseq_types.h
+++ b/include/linux/rseq_types.h
@@ -117,14 +117,20 @@ struct mm_cid_pcpu {
 /**
  * struct mm_mm_cid - Storage for per MM CID data
  * @pcpu:		Per CPU storage for CIDs associated to a CPU
+ * @max_cids:		The exclusive maximum CID value for allocation and convergance
  * @nr_cpus_allowed:	The number of CPUs in the per MM allowed CPUs map. The map
  *			is growth only.
+ * @users:		The number of tasks sharing this MM. Seperate from mm::mm_users
+ *			as that is modified by mmget()/mm_put() by other entities which
+ *			do not actually share the MM.
  * @lock:		Spinlock to protect all fields except @pcpu. It also protects
  *			the MM cid cpumask and the MM cidmask bitmap.
  */
 struct mm_mm_cid {
 	struct mm_cid_pcpu	__percpu *pcpu;
+	unsigned int		max_cids;
 	unsigned int		nr_cpus_allowed;
+	unsigned int		users;
 	raw_spinlock_t		lock;
 }____cacheline_aligned_in_smp;
 #else /* CONFIG_SCHED_MM_CID */
--- a/kernel/fork.c
+++ b/kernel/fork.c
@@ -2455,6 +2455,7 @@ static bool need_futex_hash_allocate_def
 	exit_task_namespaces(p);
 bad_fork_cleanup_mm:
 	if (p->mm) {
+		sched_mm_cid_exit(p);
 		mm_clear_owner(p->mm, p);
 		mmput(p->mm);
 	}
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -4485,7 +4485,6 @@ static void __sched_fork(u64 clone_flags
 	init_numa_balancing(clone_flags, p);
 	p->wake_entry.u_flags = CSD_TYPE_TTWU;
 	p->migration_pending = NULL;
-	init_sched_mm_cid(p);
 }
 
 DEFINE_STATIC_KEY_FALSE(sched_numa_balancing);
@@ -10369,15 +10368,27 @@ void call_trace_sched_update_nr_running(
 
 #ifdef CONFIG_SCHED_MM_CID
 /*
- * When a task exits, the MM CID held by the task is not longer required as
- * the task cannot return to user space.
+ * Update the CID range properties when the constraints change. Invoked via
+ * fork(), exit() and affinity changes
  */
+static void mm_update_max_cids(struct mm_struct *mm)
+{
+	struct mm_mm_cid *mc = &mm->mm_cid;
+	unsigned int max_cids;
+
+	lockdep_assert_held(&mm->mm_cid.lock);
+
+	/* Calculate the new maximum constraint */
+	max_cids = min(mc->nr_cpus_allowed, mc->users);
+	WRITE_ONCE(mc->max_cids, max_cids);
+}
+
 static inline void mm_update_cpus_allowed(struct mm_struct *mm, const struct cpumask *affmsk)
 {
 	struct cpumask *mm_allowed;
 	unsigned int weight;
 
-	if (!mm)
+	if (!mm || !READ_ONCE(mm->mm_cid.users))
 		return;
 
 	/*
@@ -10387,9 +10398,30 @@ static inline void mm_update_cpus_allowe
 	guard(raw_spinlock)(&mm->mm_cid.lock);
 	mm_allowed = mm_cpus_allowed(mm);
 	weight = cpumask_weighted_or(mm_allowed, mm_allowed, affmsk);
+	if (weight == mm->mm_cid.nr_cpus_allowed)
+		return;
 	WRITE_ONCE(mm->mm_cid.nr_cpus_allowed, weight);
+	mm_update_max_cids(mm);
+}
+
+void sched_mm_cid_fork(struct task_struct *t)
+{
+	struct mm_struct *mm = t->mm;
+
+	WARN_ON_ONCE(!mm || t->mm_cid.cid != MM_CID_UNSET);
+
+	guard(raw_spinlock)(&mm->mm_cid.lock);
+	t->mm_cid.active = 1;
+	mm->mm_cid.users++;
+	/* Preset last_cid for mm_cid_select() */
+	t->mm_cid.last_cid = READ_ONCE(mm->mm_cid.max_cids) - 1;
+	mm_update_max_cids(mm);
 }
 
+/*
+ * When a task exits, the MM CID held by the task is not longer required as
+ * the task cannot return to user space.
+ */
 void sched_mm_cid_exit(struct task_struct *t)
 {
 	struct mm_struct *mm = t->mm;
@@ -10397,12 +10429,14 @@ void sched_mm_cid_exit(struct task_struc
 	if (!mm || !t->mm_cid.active)
 		return;
 
-	guard(preempt)();
+	guard(raw_spinlock)(&mm->mm_cid.lock);
 	t->mm_cid.active = 0;
+	mm->mm_cid.users--;
 	if (t->mm_cid.cid != MM_CID_UNSET) {
 		clear_bit(t->mm_cid.cid, mm_cidmask(mm));
 		t->mm_cid.cid = MM_CID_UNSET;
 	}
+	mm_update_max_cids(mm);
 }
 
 /* Deactivate MM CID allocation across execve() */
@@ -10414,22 +10448,11 @@ void sched_mm_cid_before_execve(struct t
 /* Reactivate MM CID after successful execve() */
 void sched_mm_cid_after_execve(struct task_struct *t)
 {
-	struct mm_struct *mm = t->mm;
-
-	if (!mm)
-		return;
-
+	sched_mm_cid_fork(t);
 	guard(preempt)();
-	t->mm_cid.active = 1;
 	mm_cid_select(t);
 }
 
-void sched_mm_cid_fork(struct task_struct *t)
-{
-	WARN_ON_ONCE(!t->mm || t->mm_cid.cid != MM_CID_UNSET);
-	t->mm_cid.active = 1;
-}
-
 void mm_init_cid(struct mm_struct *mm, struct task_struct *p)
 {
 	struct mm_cid_pcpu __percpu *pcpu = mm->mm_cid.pcpu;
@@ -10438,7 +10461,9 @@ void mm_init_cid(struct mm_struct *mm, s
 	for_each_possible_cpu(cpu)
 		per_cpu_ptr(pcpu, cpu)->cid = MM_CID_UNSET;
 
+	mm->mm_cid.max_cids = 0;
 	mm->mm_cid.nr_cpus_allowed = p->nr_cpus_allowed;
+	mm->mm_cid.users = 0;
 	raw_spin_lock_init(&mm->mm_cid.lock);
 	cpumask_copy(mm_cpus_allowed(mm), &p->cpus_mask);
 	bitmap_zero(mm_cidmask(mm), bitmap_size(num_possible_cpus()));
--- a/kernel/sched/sched.h
+++ b/kernel/sched/sched.h
@@ -3570,7 +3570,7 @@ static inline bool mm_cid_get(struct tas
 	struct mm_struct *mm = t->mm;
 	unsigned int max_cids;
 
-	max_cids = min_t(int, READ_ONCE(mm->mm_cid.nr_cpus_allowed), atomic_read(&mm->mm_users));
+	max_cids = READ_ONCE(mm->mm_cid.max_cids);
 
 	/* Try to reuse the last CID of this task */
 	if (__mm_cid_get(t, t->mm_cid.last_cid, max_cids))
@@ -3613,7 +3613,6 @@ static inline void switch_mm_cid(struct
 }
 
 #else /* !CONFIG_SCHED_MM_CID: */
-static inline void init_sched_mm_cid(struct task_struct *t) { }
 static inline void mm_cid_select(struct task_struct *t) { }
 static inline void switch_mm_cid(struct task_struct *prev, struct task_struct *next) { }
 #endif /* !CONFIG_SCHED_MM_CID */


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

* [patch V3 14/20] sched/mmcid: Serialize sched_mm_cid_fork()/exit() with a mutex
  2025-10-29 13:08 [patch V3 00/20] sched: Rewrite MM CID management Thomas Gleixner
                   ` (12 preceding siblings ...)
  2025-10-29 13:09 ` [patch V3 13/20] sched/mmcid: Provide precomputed maximal value Thomas Gleixner
@ 2025-10-29 13:09 ` Thomas Gleixner
  2025-10-30 14:25   ` Mathieu Desnoyers
  2025-10-29 13:09 ` [patch V3 15/20] sched/mmcid: Introduce per task/CPU ownership infrastrcuture Thomas Gleixner
                   ` (7 subsequent siblings)
  21 siblings, 1 reply; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-29 13:09 UTC (permalink / raw)
  To: LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov, Shrikanth Hegde

Prepare for the new CID management scheme which puts the CID ownership
transition into the fork() and exit() slow path by serializing
sched_mm_cid_fork()/exit() with it, so task list and cpu mask walks can be
done in interruptible and preemptible code.

The contention on it is not worse than on other concurrency controls in the
fork()/exit() machinery.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 include/linux/rseq_types.h |    2 ++
 kernel/sched/core.c        |   22 ++++++++++++++++++++++
 2 files changed, 24 insertions(+)

--- a/include/linux/rseq_types.h
+++ b/include/linux/rseq_types.h
@@ -125,6 +125,7 @@ struct mm_cid_pcpu {
  *			do not actually share the MM.
  * @lock:		Spinlock to protect all fields except @pcpu. It also protects
  *			the MM cid cpumask and the MM cidmask bitmap.
+ * @mutex:		Mutex to serialize forks and exits related to this mm
  */
 struct mm_mm_cid {
 	struct mm_cid_pcpu	__percpu *pcpu;
@@ -132,6 +133,7 @@ struct mm_mm_cid {
 	unsigned int		nr_cpus_allowed;
 	unsigned int		users;
 	raw_spinlock_t		lock;
+	struct mutex		mutex;
 }____cacheline_aligned_in_smp;
 #else /* CONFIG_SCHED_MM_CID */
 struct mm_mm_cid { };
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -10368,6 +10368,25 @@ void call_trace_sched_update_nr_running(
 
 #ifdef CONFIG_SCHED_MM_CID
 /*
+ * Concurrency IDentifier management
+ *
+ * Serialization rules:
+ *
+ * mm::mm_cid::mutex:	Serializes fork() and exit() and therefore
+ *			protects mm::mm_cid::users.
+ *
+ * mm::mm_cid::lock:	Serializes mm_update_max_cids() and
+ *			mm_update_cpus_allowed(). Nests in mm_cid::mutex
+ *			and runqueue lock.
+ *
+ * The mm_cidmask bitmap is not protected by any of the mm::mm_cid locks
+ * and can only be modified with atomic operations.
+ *
+ * The mm::mm_cid:pcpu per CPU storage is protected by the CPUs runqueue
+ * lock.
+ */
+
+/*
  * Update the CID range properties when the constraints change. Invoked via
  * fork(), exit() and affinity changes
  */
@@ -10410,6 +10429,7 @@ void sched_mm_cid_fork(struct task_struc
 
 	WARN_ON_ONCE(!mm || t->mm_cid.cid != MM_CID_UNSET);
 
+	guard(mutex)(&mm->mm_cid.mutex);
 	guard(raw_spinlock)(&mm->mm_cid.lock);
 	t->mm_cid.active = 1;
 	mm->mm_cid.users++;
@@ -10429,6 +10449,7 @@ void sched_mm_cid_exit(struct task_struc
 	if (!mm || !t->mm_cid.active)
 		return;
 
+	guard(mutex)(&mm->mm_cid.mutex);
 	guard(raw_spinlock)(&mm->mm_cid.lock);
 	t->mm_cid.active = 0;
 	mm->mm_cid.users--;
@@ -10465,6 +10486,7 @@ void mm_init_cid(struct mm_struct *mm, s
 	mm->mm_cid.nr_cpus_allowed = p->nr_cpus_allowed;
 	mm->mm_cid.users = 0;
 	raw_spin_lock_init(&mm->mm_cid.lock);
+	mutex_init(&mm->mm_cid.mutex);
 	cpumask_copy(mm_cpus_allowed(mm), &p->cpus_mask);
 	bitmap_zero(mm_cidmask(mm), bitmap_size(num_possible_cpus()));
 }


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

* [patch V3 15/20] sched/mmcid: Introduce per task/CPU ownership infrastrcuture
  2025-10-29 13:08 [patch V3 00/20] sched: Rewrite MM CID management Thomas Gleixner
                   ` (13 preceding siblings ...)
  2025-10-29 13:09 ` [patch V3 14/20] sched/mmcid: Serialize sched_mm_cid_fork()/exit() with a mutex Thomas Gleixner
@ 2025-10-29 13:09 ` Thomas Gleixner
  2025-10-30 14:45   ` Mathieu Desnoyers
  2025-10-29 13:09 ` [patch V3 16/20] sched/mmcid: Provide new scheduler CID mechanism Thomas Gleixner
                   ` (6 subsequent siblings)
  21 siblings, 1 reply; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-29 13:09 UTC (permalink / raw)
  To: LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov, Shrikanth Hegde

The MM CID management has two fundamental requirements:

  1) It has to guarantee that at no given point in time the same CID is
     used by concurrent tasks in userspace.

  2) The CID space must not exceed the number of possible CPUs in a
     system. While most allocators (glibc, tcmalloc, jemalloc) do not
     care about that, there seems to be at least some LTTng library
     depending on it.

The CID space compaction itself is not a functional correctness
requirement, it is only a useful optimization mechanism to reduce the
memory foot print in unused user space pools.

The optimal CID space is:

    min(nr_tasks, nr_cpus_allowed);

Where @nr_tasks is the number of actual user space threads associated to
the mm and @nr_cpus_allowed is the superset of all task affinities. It is
growth only as it would be insane to take a racy snapshot of all task
affinities when the affinity of one task changes just do redo it 2
milliseconds later when the next task changes it's affinity.

That means that as long as the number of tasks is lower or equal than the
number of CPUs allowed, each task owns a CID. If the number of tasks
exceeds the number of CPUs allowed it switches to per CPU mode, where the
CPUs own the CIDs and the tasks borrow them as long as they are scheduled
in.

For transition periods CIDs can go beyond the optimal space as long as they
don't go beyond the number of possible CPUs.

The current upstream implementation tries to keep the CID with the task
even in overcommit situations, which complicates task migration. It also
has to do the CID space consolidation work from a task work in the exit to
user space path. As that work is assigned to a random task related to a MM
this can inflict unwanted exit latencies.

This can be done differently by implementing a strict CID ownership
mechanism. Either the CIDs are owned by the tasks or by the CPUs. The
latter provides less locality when tasks are heavily migrating, but there
is no justification to optimize for overcommit scenarios and thereby
penalazing everyone else.

Provide the basic infrastructure to implement this:

  - Change the UNSET marker to BIT(31) from ~0U
  - Add the ONCPU marker as BIT(30)
  - Add the TRANSIT marker as BIT(29)

That allows to check for ownership trivialy and provides a simple check for
UNSET as well. The TRANSIT marker is required to prevent CID space
exhaustion when switching from per CPU to per task mode.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
V2: Add the TRANSIT(ion) bit
---
 include/linux/rseq_types.h |    4 ++-
 include/linux/sched.h      |    6 ++--
 kernel/sched/core.c        |   10 +++++++
 kernel/sched/sched.h       |   59 +++++++++++++++++++++++++++++++++++++++++++++
 4 files changed, 75 insertions(+), 4 deletions(-)

--- a/include/linux/rseq_types.h
+++ b/include/linux/rseq_types.h
@@ -92,7 +92,9 @@ struct rseq_data { };
 
 #ifdef CONFIG_SCHED_MM_CID
 
-#define MM_CID_UNSET	(~0U)
+#define MM_CID_UNSET	BIT(31)
+#define MM_CID_ONCPU	BIT(30)
+#define MM_CID_TRANSIT	BIT(29)
 
 /**
  * struct sched_mm_cid - Storage for per task MM CID data
--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -2299,16 +2299,16 @@ void sched_mm_cid_before_execve(struct t
 void sched_mm_cid_after_execve(struct task_struct *t);
 void sched_mm_cid_fork(struct task_struct *t);
 void sched_mm_cid_exit(struct task_struct *t);
-static inline int task_mm_cid(struct task_struct *t)
+static __always_inline int task_mm_cid(struct task_struct *t)
 {
-	return t->mm_cid.cid;
+	return t->mm_cid.cid & ~(MM_CID_ONCPU | MM_CID_TRANSIT);
 }
 #else
 static inline void sched_mm_cid_before_execve(struct task_struct *t) { }
 static inline void sched_mm_cid_after_execve(struct task_struct *t) { }
 static inline void sched_mm_cid_fork(struct task_struct *t) { }
 static inline void sched_mm_cid_exit(struct task_struct *t) { }
-static inline int task_mm_cid(struct task_struct *t)
+static __always_inline int task_mm_cid(struct task_struct *t)
 {
 	/*
 	 * Use the processor id as a fall-back when the mm cid feature is
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -10384,6 +10384,16 @@ void call_trace_sched_update_nr_running(
  *
  * The mm::mm_cid:pcpu per CPU storage is protected by the CPUs runqueue
  * lock.
+ *
+ * CID ownership:
+ *
+ * A CID is either owned by a task (stored in task_struct::mm_cid.cid) or
+ * by a CPU (stored in mm::mm_cid.pcpu::cid). CIDs owned by CPUs have the
+ * MM_CID_ONCPU bit set. During transitioning from CPU to task ownership
+ * mode, MM_CID_TRANSIT is set on the per task CIDs. When this bit is set
+ * the task needs to drop the CID into the pool when scheduling out.  Both
+ * bits (ONCPU and TRANSIT) are filtered out by task_cid() when the CID is
+ * actualy handed over to user space in the RSEQ memory.
  */
 
 /*
--- a/kernel/sched/sched.h
+++ b/kernel/sched/sched.h
@@ -3539,6 +3539,65 @@ extern void sched_dynamic_update(int mod
 extern const char *preempt_modes[];
 
 #ifdef CONFIG_SCHED_MM_CID
+
+static __always_inline bool cid_on_cpu(unsigned int cid)
+{
+	return cid & MM_CID_ONCPU;
+}
+
+static __always_inline bool cid_in_transit(unsigned int cid)
+{
+	return cid & MM_CID_TRANSIT;
+}
+
+static __always_inline unsigned int cpu_cid_to_cid(unsigned int cid)
+{
+	return cid & ~MM_CID_ONCPU;
+}
+
+static __always_inline unsigned int cid_to_cpu_cid(unsigned int cid)
+{
+	return cid | MM_CID_ONCPU;
+}
+
+static __always_inline unsigned int cid_to_transit_cid(unsigned int cid)
+{
+	return cid | MM_CID_TRANSIT;
+}
+
+static __always_inline unsigned int cid_from_transit_cid(unsigned int cid)
+{
+	return cid & ~MM_CID_TRANSIT;
+}
+
+static __always_inline bool cid_on_task(unsigned int cid)
+{
+	/* True if none of the MM_CID_ONCPU, MM_CID_TRANSIT, MM_CID_UNSET bits is set */
+	return cid < MM_CID_TRANSIT;
+}
+
+static __always_inline void mm_drop_cid(struct mm_struct *mm, unsigned int cid)
+{
+	clear_bit(cid, mm_cidmask(mm));
+}
+
+static __always_inline void mm_unset_cid_on_task(struct task_struct *t)
+{
+	unsigned int cid = t->mm_cid.cid;
+
+	t->mm_cid.cid = MM_CID_UNSET;
+	if (cid_on_task(cid))
+		mm_drop_cid(t->mm, cid);
+}
+
+static __always_inline void mm_drop_cid_on_cpu(struct mm_struct *mm, struct mm_cid_pcpu *pcp)
+{
+	/* Clear the ONCPU bit, but do not set UNSET in the per CPU storage */
+	pcp->cid = cpu_cid_to_cid(pcp->cid);
+	mm_drop_cid(mm, pcp->cid);
+}
+
+/* Active implementation */
 static inline void init_sched_mm_cid(struct task_struct *t)
 {
 	struct mm_struct *mm = t->mm;


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

* [patch V3 16/20] sched/mmcid: Provide new scheduler CID mechanism
  2025-10-29 13:08 [patch V3 00/20] sched: Rewrite MM CID management Thomas Gleixner
                   ` (14 preceding siblings ...)
  2025-10-29 13:09 ` [patch V3 15/20] sched/mmcid: Introduce per task/CPU ownership infrastrcuture Thomas Gleixner
@ 2025-10-29 13:09 ` Thomas Gleixner
  2025-10-30 15:09   ` Mathieu Desnoyers
  2025-10-29 13:09 ` [patch V3 17/20] sched/mmcid: Provide CID ownership mode fixup functions Thomas Gleixner
                   ` (5 subsequent siblings)
  21 siblings, 1 reply; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-29 13:09 UTC (permalink / raw)
  To: LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov, Shrikanth Hegde

The MM CID management has two fundamental requirements:

  1) It has to guarantee that at no given point in time the same CID is
     used by concurrent tasks in userspace.

  2) The CID space must not exceed the number of possible CPUs in a
     system. While most allocators (glibc, tcmalloc, jemalloc) do not
     care about that, there seems to be at least some LTTng library
     depending on it.

The CID space compaction itself is not a functional correctness
requirement, it is only a useful optimization mechanism to reduce the
memory foot print in unused user space pools.

The optimal CID space is:

    min(nr_tasks, nr_cpus_allowed);

Where @nr_tasks is the number of actual user space threads associated to
the mm and @nr_cpus_allowed is the superset of all task affinities. It is
growth only as it would be insane to take a racy snapshot of all task
affinities when the affinity of one task changes just do redo it 2
milliseconds later when the next task changes it's affinity.

That means that as long as the number of tasks is lower or equal than the
number of CPUs allowed, each task owns a CID. If the number of tasks
exceeds the number of CPUs allowed it switches to per CPU mode, where the
CPUs own the CIDs and the tasks borrow them as long as they are scheduled
in.

For transition periods CIDs can go beyond the optimal space as long as they
don't go beyond the number of possible CPUs.

The current upstream implementation tries to keep the CID with the task
even in overcommit situations, which complicates task migration. It also
has to do the CID space consolidation work from a task work in the exit to
user space path. As that work is assigned to a random task related to a MM
this can inflict unwanted exit latencies.

Implement the context switch parts of a strict ownership mechanism to
address this.

This removes most of the work from the task which schedules out. Only
during transitioning from per CPU to per task ownership it is required to
drop the CID when leaving the CPU to prevent CID space exhaustion. Other
than that scheduling out is just a single check and branch.

The task which schedules in has to check whether:

    1) The ownership mode changed
    2) The CID is within the optimal CID space

In stable situations this results in zero work. The only short disruption
is when ownership mode changes or when the associated CID is not in the
optimal CID space. The latter only happens when tasks exit and therefore
the optimal CID space shrinks.

That mechanism is strictly optimized for the common case where no change
happens. The only case where it actually causes a temporary one time spike
is on mode changes when and only when a lot of tasks related to a MM
schedule exactly at the same time and have eventually to compete on
allocating a CID from the bitmap.

In the sysbench test case which triggered the spinlock contention in the
initial CID code, __schedule() drops significantly in perf top on a 128
Core (256 threads) machine when running sysbench with 255 threads, which
fits into the task mode limit of 256 together with the parent thread:

  Upstream  rseq/perf branch  +CID rework  
  0.42%     0.37%             0.32%          [k] __schedule

Increasing the number of threads to 256, which puts the test process into
per CPU mode looks about the same.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
V3: Move cpu_relax() out of the for() construct - Shrikanth
V2: Adjust to the mode transition changes
---
 include/linux/rseq.h       |    8 +-
 include/linux/rseq_types.h |    5 +
 kernel/sched/core.c        |    1 
 kernel/sched/sched.h       |  150 ++++++++++++++++++++++++++++++++++++++++++++-
 4 files changed, 159 insertions(+), 5 deletions(-)

--- a/include/linux/rseq.h
+++ b/include/linux/rseq.h
@@ -73,13 +73,13 @@ static __always_inline void rseq_sched_s
 }
 
 /*
- * Invoked from __set_task_cpu() when a task migrates to enforce an IDs
- * update.
+ * Invoked from __set_task_cpu() when a task migrates or from
+ * mm_cid_schedin() when the CID changes to enforce an IDs update.
  *
  * This does not raise TIF_NOTIFY_RESUME as that happens in
  * rseq_sched_switch_event().
  */
-static __always_inline void rseq_sched_set_task_cpu(struct task_struct *t, unsigned int cpu)
+static __always_inline void rseq_sched_set_ids_changed(struct task_struct *t)
 {
 	t->rseq.event.ids_changed = true;
 }
@@ -169,7 +169,7 @@ static inline void rseq_fork(struct task
 static inline void rseq_handle_slowpath(struct pt_regs *regs) { }
 static inline void rseq_signal_deliver(struct ksignal *ksig, struct pt_regs *regs) { }
 static inline void rseq_sched_switch_event(struct task_struct *t) { }
-static inline void rseq_sched_set_task_cpu(struct task_struct *t, unsigned int cpu) { }
+static inline void rseq_sched_set_ids_changed(struct task_struct *t) { }
 static inline void rseq_sched_set_task_mm_cid(struct task_struct *t, unsigned int cid) { }
 static inline void rseq_force_update(void) { }
 static inline void rseq_virt_userspace_exit(void) { }
--- a/include/linux/rseq_types.h
+++ b/include/linux/rseq_types.h
@@ -119,6 +119,8 @@ struct mm_cid_pcpu {
 /**
  * struct mm_mm_cid - Storage for per MM CID data
  * @pcpu:		Per CPU storage for CIDs associated to a CPU
+ * @percpu:		Set, when CIDs are in per CPU mode
+ * @transit:		Set to MM_CID_TRANSIT during a mode change transition phase
  * @max_cids:		The exclusive maximum CID value for allocation and convergance
  * @nr_cpus_allowed:	The number of CPUs in the per MM allowed CPUs map. The map
  *			is growth only.
@@ -130,7 +132,10 @@ struct mm_cid_pcpu {
  * @mutex:		Mutex to serialize forks and exits related to this mm
  */
 struct mm_mm_cid {
+	/* Hotpath read mostly members */
 	struct mm_cid_pcpu	__percpu *pcpu;
+	unsigned int		percpu;
+	unsigned int		transit;
 	unsigned int		max_cids;
 	unsigned int		nr_cpus_allowed;
 	unsigned int		users;
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -10493,6 +10493,7 @@ void mm_init_cid(struct mm_struct *mm, s
 		per_cpu_ptr(pcpu, cpu)->cid = MM_CID_UNSET;
 
 	mm->mm_cid.max_cids = 0;
+	mm->mm_cid.percpu = 0;
 	mm->mm_cid.nr_cpus_allowed = p->nr_cpus_allowed;
 	mm->mm_cid.users = 0;
 	raw_spin_lock_init(&mm->mm_cid.lock);
--- a/kernel/sched/sched.h
+++ b/kernel/sched/sched.h
@@ -2208,7 +2208,7 @@ static inline void __set_task_cpu(struct
 	smp_wmb();
 	WRITE_ONCE(task_thread_info(p)->cpu, cpu);
 	p->wake_cpu = cpu;
-	rseq_sched_set_task_cpu(p, cpu);
+	rseq_sched_set_ids_changed(p);
 #endif /* CONFIG_SMP */
 }
 
@@ -3597,6 +3597,153 @@ static __always_inline void mm_drop_cid_
 	mm_drop_cid(mm, pcp->cid);
 }
 
+static inline unsigned int __mm_get_cid(struct mm_struct *mm, unsigned int max_cids)
+{
+	unsigned int cid = find_first_zero_bit(mm_cidmask(mm), max_cids);
+
+	if (cid >= max_cids)
+		return MM_CID_UNSET;
+	if (test_and_set_bit(cid, mm_cidmask(mm)))
+		return MM_CID_UNSET;
+	return cid;
+}
+
+static inline unsigned int mm_get_cid(struct mm_struct *mm)
+{
+	unsigned int cid = __mm_get_cid(mm, READ_ONCE(mm->mm_cid.max_cids));
+
+	while (cid == MM_CID_UNSET) {
+		cpu_relax();
+		cid = __mm_get_cid(mm, num_possible_cpus());
+	}
+	return cid;
+}
+
+static inline unsigned int mm_cid_converge(struct mm_struct *mm, unsigned int orig_cid,
+					   unsigned int max_cids)
+{
+	unsigned int new_cid, cid = cpu_cid_to_cid(orig_cid);
+
+	/* Is it in the optimal CID space? */
+	if (likely(cid < max_cids))
+		return orig_cid;
+
+	/* Try to find one in the optimal space. Otherwise keep the provided. */
+	new_cid = __mm_get_cid(mm, max_cids);
+	if (new_cid != MM_CID_UNSET) {
+		mm_drop_cid(mm, cid);
+		/* Preserve the ONCPU mode of the original CID */
+		return new_cid | (orig_cid & MM_CID_ONCPU);
+	}
+	return orig_cid;
+}
+
+static __always_inline void mm_cid_update_task_cid(struct task_struct *t, unsigned int cid)
+{
+	if (t->mm_cid.cid != cid) {
+		t->mm_cid.cid = cid;
+		rseq_sched_set_ids_changed(t);
+	}
+}
+
+static __always_inline void mm_cid_update_pcpu_cid(struct mm_struct *mm, unsigned int cid)
+{
+	__this_cpu_write(mm->mm_cid.pcpu->cid, cid);
+}
+
+static __always_inline void mm_cid_from_cpu(struct task_struct *t, unsigned int cpu_cid)
+{
+	unsigned int max_cids, tcid = t->mm_cid.cid;
+	struct mm_struct *mm = t->mm;
+
+	max_cids = READ_ONCE(mm->mm_cid.max_cids);
+	/* Optimize for the common case where both have the ONCPU bit set */
+	if (likely(cid_on_cpu(cpu_cid & tcid))) {
+		if (likely(cpu_cid_to_cid(cpu_cid) < max_cids)) {
+			mm_cid_update_task_cid(t, cpu_cid);
+			return;
+		}
+		/* Try to converge into the optimal CID space */
+		cpu_cid = mm_cid_converge(mm, cpu_cid, max_cids);
+	} else {
+		/* Hand over or drop the task owned CID */
+		if (cid_on_task(tcid)) {
+			if (cid_on_cpu(cpu_cid))
+				mm_unset_cid_on_task(t);
+			else
+				cpu_cid = cid_to_cpu_cid(tcid);
+		}
+		/* Still nothing, allocate a new one */
+		if (!cid_on_cpu(cpu_cid))
+			cpu_cid = cid_to_cpu_cid(mm_get_cid(mm));
+	}
+	mm_cid_update_pcpu_cid(mm, cpu_cid);
+	mm_cid_update_task_cid(t, cpu_cid);
+}
+
+static __always_inline void mm_cid_from_task(struct task_struct *t, unsigned int cpu_cid)
+{
+	unsigned int max_cids, tcid = t->mm_cid.cid;
+	struct mm_struct *mm = t->mm;
+
+	max_cids = READ_ONCE(mm->mm_cid.max_cids);
+	/* Optimize for the common case, where both have the ONCPU bit clear */
+	if (likely(cid_on_task(tcid | cpu_cid))) {
+		if (likely(tcid < max_cids)) {
+			mm_cid_update_pcpu_cid(mm, tcid);
+			return;
+		}
+		/* Try to converge into the optimal CID space */
+		tcid = mm_cid_converge(mm, tcid, max_cids);
+	} else {
+		/* Hand over or drop the CPU owned CID */
+		if (cid_on_cpu(cpu_cid)) {
+			if (cid_on_task(tcid))
+				mm_drop_cid_on_cpu(mm, this_cpu_ptr(mm->mm_cid.pcpu));
+			else
+				tcid = cpu_cid_to_cid(cpu_cid);
+		}
+		/* Still nothing, allocate a new one */
+		if (!cid_on_task(tcid))
+			tcid = mm_get_cid(mm);
+		/* Set the transition mode flag if required */
+		tcid |= READ_ONCE(mm->mm_cid.transit);
+	}
+	mm_cid_update_pcpu_cid(mm, tcid);
+	mm_cid_update_task_cid(t, tcid);
+}
+
+static __always_inline void mm_cid_schedin(struct task_struct *next)
+{
+	struct mm_struct *mm = next->mm;
+	unsigned int cpu_cid;
+
+	if (!next->mm_cid.active)
+		return;
+
+	cpu_cid = __this_cpu_read(mm->mm_cid.pcpu->cid);
+	if (likely(!READ_ONCE(mm->mm_cid.percpu)))
+		mm_cid_from_task(next, cpu_cid);
+	else
+		mm_cid_from_cpu(next, cpu_cid);
+}
+
+static __always_inline void mm_cid_schedout(struct task_struct *prev)
+{
+	/* During mode transitions CIDs are temporary and need to be dropped */
+	if (likely(!cid_in_transit(prev->mm_cid.cid)))
+		return;
+
+	mm_drop_cid(prev->mm, cid_from_transit_cid(prev->mm_cid.cid));
+	prev->mm_cid.cid = MM_CID_UNSET;
+}
+
+static inline void mm_cid_switch_to(struct task_struct *prev, struct task_struct *next)
+{
+	mm_cid_schedout(prev);
+	mm_cid_schedin(next);
+}
+
 /* Active implementation */
 static inline void init_sched_mm_cid(struct task_struct *t)
 {
@@ -3674,6 +3821,7 @@ static inline void switch_mm_cid(struct
 #else /* !CONFIG_SCHED_MM_CID: */
 static inline void mm_cid_select(struct task_struct *t) { }
 static inline void switch_mm_cid(struct task_struct *prev, struct task_struct *next) { }
+static inline void mm_cid_switch_to(struct task_struct *prev, struct task_struct *next) { }
 #endif /* !CONFIG_SCHED_MM_CID */
 
 extern u64 avg_vruntime(struct cfs_rq *cfs_rq);


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

* [patch V3 17/20] sched/mmcid: Provide CID ownership mode fixup functions
  2025-10-29 13:08 [patch V3 00/20] sched: Rewrite MM CID management Thomas Gleixner
                   ` (15 preceding siblings ...)
  2025-10-29 13:09 ` [patch V3 16/20] sched/mmcid: Provide new scheduler CID mechanism Thomas Gleixner
@ 2025-10-29 13:09 ` Thomas Gleixner
  2025-10-30 15:51   ` Mathieu Desnoyers
  2025-10-29 13:09 ` [patch V3 18/20] irqwork: Move data struct to a types header Thomas Gleixner
                   ` (4 subsequent siblings)
  21 siblings, 1 reply; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-29 13:09 UTC (permalink / raw)
  To: LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov, Shrikanth Hegde

CIDs are either owned by tasks or by CPUs. The ownership mode depends on
the number of tasks related to a MM and the number of CPUs on which these
tasks are theoretically allowed to run on. Theoretically because that
number is the superset of CPU affinities of all tasks which only grows and
never shrinks.

Switching to per CPU mode happens when the user count becomes greater than
the maximum number of CIDs, which is calculated by:

	opt_cids = min(mm_cid::nr_cpus_allowed, mm_cid::users);
	max_cids = min(1.25 * opt_cids, nr_cpu_ids);

The +25% allowance is useful for tight CPU masks in scenarios where only a
few threads are created and destroyed to avoid frequent mode
switches. Though this allowance shrinks, the closer opt_cids becomes to
nr_cpu_ids, which is the (unfortunate) hard ABI limit.

At the point of switching to per CPU mode the new user is not yet visible
in the system, so the task which initiated the fork() runs the fixup
function: mm_cid_fixup_tasks_to_cpu() walks the thread list and either
transfers each tasks owned CID to the CPU the task runs on or drops it into
the CID pool if a task is not on a CPU at that point in time. Tasks which
schedule in before the task walk reaches them do the handover in
mm_cid_schedin(). When mm_cid_fixup_tasks_to_cpus() completes it's
guaranteed that no task related to that MM owns a CID anymore.

Switching back to task mode happens when the user count goes below the
threshold which was recorded on the per CPU mode switch:

	pcpu_thrs = min(opt_cids - (opt_cids / 4), nr_cpu_ids / 2);

This threshold is updated when a affinity change increases the number of
allowed CPUs for the MM, which might cause a switch back to per task mode.

If the switch back was initiated by a exiting task, then that task runs the
fixup function. If it was initiated by a affinity change, then it's run
either in the deferred update function in context of a workqueue or by a
task which forks a new one or by a task which exits. Whatever happens
first. mm_cid_fixup_cpus_to_task() walks through the possible CPUs and
either transfers the CPU owned CIDs to a related task which runs on the CPU
or drops it into the pool. Tasks which schedule in on a CPU which the walk
did not cover yet do the handover themself.

This transition from CPU to per task ownership happens in two phases:

 1) mm:mm_cid.transit contains MM_CID_TRANSIT. This is OR'ed on the task
    CID and denotes that the CID is only temporarily owned by the
    task. When it schedules out the task drops the CID back into the
    pool if this bit is set.

 2) The initiating context walks the per CPU space and after completion
    clears mm:mm_cid.transit. After that point the CIDs are strictly
    task owned again.

This two phase transition is required to prevent CID space exhaustion
during the transition as a direct transfer of ownership would fail if
two tasks are scheduled in on the same CPU before the fixup freed per
CPU CIDs.

When mm_cid_fixup_cpus_to_tasks() completes it's guaranteed that no CID
related to that MM is owned by a CPU anymore.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
V2: Fix CPU to task owner mode
---
 include/linux/rseq_types.h |   10 +
 kernel/sched/core.c        |  277 +++++++++++++++++++++++++++++++++++++++++----
 2 files changed, 261 insertions(+), 26 deletions(-)

--- a/include/linux/rseq_types.h
+++ b/include/linux/rseq_types.h
@@ -127,8 +127,9 @@ struct mm_cid_pcpu {
  * @users:		The number of tasks sharing this MM. Seperate from mm::mm_users
  *			as that is modified by mmget()/mm_put() by other entities which
  *			do not actually share the MM.
- * @lock:		Spinlock to protect all fields except @pcpu. It also protects
- *			the MM cid cpumask and the MM cidmask bitmap.
+ * @pcpu_thrs:		Threshold for switching back from per CPU mode
+ * @update_deferred:	A deferred switch back to per task mode is pending.
+ * @lock:		Spinlock to protect against affinity setting which can't take @mutex
  * @mutex:		Mutex to serialize forks and exits related to this mm
  */
 struct mm_mm_cid {
@@ -137,8 +138,13 @@ struct mm_mm_cid {
 	unsigned int		percpu;
 	unsigned int		transit;
 	unsigned int		max_cids;
+
+	/* Low frequency modified */
 	unsigned int		nr_cpus_allowed;
 	unsigned int		users;
+	unsigned int		pcpu_thrs;
+	unsigned int		update_deferred;
+
 	raw_spinlock_t		lock;
 	struct mutex		mutex;
 }____cacheline_aligned_in_smp;
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -10396,43 +10396,269 @@ void call_trace_sched_update_nr_running(
  * the task needs to drop the CID into the pool when scheduling out.  Both
  * bits (ONCPU and TRANSIT) are filtered out by task_cid() when the CID is
  * actualy handed over to user space in the RSEQ memory.
+ *
+ * Mode switching:
+ *
+ * Switching to per CPU mode happens when the user count becomes greater
+ * than the maximum number of CIDs, which is calculated by:
+ *
+ *	opt_cids = min(mm_cid::nr_cpus_allowed, mm_cid::users);
+ *	max_cids = min(1.25 * opt_cids, num_possible_cpus());
+ *
+ * The +25% allowance is useful for tight CPU masks in scenarios where only
+ * a few threads are created and destroyed to avoid frequent mode
+ * switches. Though this allowance shrinks, the closer opt_cids becomes to
+ * num_possible_cpus(), which is the (unfortunate) hard ABI limit.
+ *
+ * At the point of switching to per CPU mode the new user is not yet
+ * visible in the system, so the task which initiated the fork() runs the
+ * fixup function: mm_cid_fixup_tasks_to_cpu() walks the thread list and
+ * either transfers each tasks owned CID to the CPU the task runs on or
+ * drops it into the CID pool if a task is not on a CPU at that point in
+ * time. Tasks which schedule in before the task walk reaches them do the
+ * handover in mm_cid_schedin(). When mm_cid_fixup_tasks_to_cpus() completes
+ * it's guaranteed that no task related to that MM owns a CID anymore.
+ *
+ * Switching back to task mode happens when the user count goes below the
+ * threshold which was recorded on the per CPU mode switch:
+ *
+ *	pcpu_thrs = min(opt_cids - (opt_cids / 4), num_possible_cpus() / 2);
+ *
+ * This threshold is updated when a affinity change increases the number of
+ * allowed CPUs for the MM, which might cause a switch back to per task
+ * mode.
+ *
+ * If the switch back was initiated by a exiting task, then that task runs
+ * the fixup function. If it was initiated by a affinity change, then it's
+ * run either in the deferred update function in context of a workqueue or
+ * by a task which forks a new one or by a task which exits. Whatever
+ * happens first. mm_cid_fixup_cpus_to_task() walks through the possible
+ * CPUs and either transfers the CPU owned CIDs to a related task which
+ * runs on the CPU or drops it into the pool. Tasks which schedule in on a
+ * CPU which the walk did not cover yet do the handover themself.
+ *
+ * This transition from CPU to per task ownership happens in two phases:
+ *
+ *  1) mm:mm_cid.transit contains MM_CID_TRANSIT This is OR'ed on the task
+ *     CID and denotes that the CID is only temporarily owned by the
+ *     task. When it schedules out the task drops the CID back into the
+ *     pool if this bit is set.
+ *
+ *  2) The initiating context walks the per CPU space and after completion
+ *     clears mm:mm_cid.transit. So after that point the CIDs are strictly
+ *     task owned again.
+ *
+ * This two phase transition is required to prevent CID space exhaustion
+ * during the transition as a direct transfer of ownership would fail if
+ * two tasks are scheduled in on the same CPU before the fixup freed per
+ * CPU CIDs.
+ *
+ * When mm_cid_fixup_cpus_to_tasks() completes it's guaranteed that no CID
+ * related to that MM is owned by a CPU anymore.
  */
 
 /*
  * Update the CID range properties when the constraints change. Invoked via
  * fork(), exit() and affinity changes
  */
-static void mm_update_max_cids(struct mm_struct *mm)
+static void __mm_update_max_cids(struct mm_mm_cid *mc)
+{
+	unsigned int opt_cids, max_cids;
+
+	/* Calculate the new optimal constraint */
+	opt_cids = min(mc->nr_cpus_allowed, mc->users);
+
+	/* Adjust the maximum CIDs to +25% limited by the number of possible CPUs */
+	max_cids = min(opt_cids + (opt_cids / 4), num_possible_cpus());
+	WRITE_ONCE(mc->max_cids, max_cids);
+}
+
+static inline unsigned int mm_cid_calc_pcpu_thrs(struct mm_mm_cid *mc)
+{
+	unsigned int opt_cids;
+
+	opt_cids = min(mc->nr_cpus_allowed, mc->users);
+	return min(opt_cids - opt_cids / 4, num_possible_cpus() / 2);
+}
+
+static bool mm_update_max_cids(struct mm_struct *mm)
 {
 	struct mm_mm_cid *mc = &mm->mm_cid;
-	unsigned int max_cids;
 
 	lockdep_assert_held(&mm->mm_cid.lock);
 
-	/* Calculate the new maximum constraint */
-	max_cids = min(mc->nr_cpus_allowed, mc->users);
-	WRITE_ONCE(mc->max_cids, max_cids);
+	/* Clear deferred mode switch flag. A change is handled by the caller */
+	mc->update_deferred = false;
+	__mm_update_max_cids(mc);
+
+	/* Check whether owner mode must be changed */
+	if (!mc->percpu) {
+		/* Enable per CPU mode when the number of users is above max_cids */
+		if (mc->users > mc->max_cids)
+			mc->pcpu_thrs = mm_cid_calc_pcpu_thrs(mc);
+	} else {
+		/* Switch back to per task if user count under threshold */
+		if (mc->users < mc->pcpu_thrs)
+			mc->pcpu_thrs = 0;
+	}
+
+	/* Mode change required? */
+	if (!!mc->percpu == !!mc->pcpu_thrs)
+		return false;
+	/* When switching back to per TASK mode, set the transition flag */
+	if (!mc->pcpu_thrs)
+		WRITE_ONCE(mc->transit, MM_CID_TRANSIT);
+	WRITE_ONCE(mc->percpu, !!mc->pcpu_thrs);
+	return true;
 }
 
 static inline void mm_update_cpus_allowed(struct mm_struct *mm, const struct cpumask *affmsk)
 {
 	struct cpumask *mm_allowed;
+	struct mm_mm_cid *mc;
 	unsigned int weight;
 
 	if (!mm || !READ_ONCE(mm->mm_cid.users))
 		return;
-
 	/*
 	 * mm::mm_cid::mm_cpus_allowed is the superset of each threads
 	 * allowed CPUs mask which means it can only grow.
 	 */
-	guard(raw_spinlock)(&mm->mm_cid.lock);
+	mc = &mm->mm_cid;
+	guard(raw_spinlock)(&mc->lock);
 	mm_allowed = mm_cpus_allowed(mm);
 	weight = cpumask_weighted_or(mm_allowed, mm_allowed, affmsk);
-	if (weight == mm->mm_cid.nr_cpus_allowed)
+	if (weight == mc->nr_cpus_allowed)
+		return;
+
+	WRITE_ONCE(mc->nr_cpus_allowed, weight);
+	__mm_update_max_cids(mc);
+	if (!mc->percpu)
 		return;
-	WRITE_ONCE(mm->mm_cid.nr_cpus_allowed, weight);
-	mm_update_max_cids(mm);
+
+	/* Adjust the threshold to the wider set */
+	mc->pcpu_thrs = mm_cid_calc_pcpu_thrs(mc);
+
+	/* Scheduling of deferred mode switch goes here */
+}
+
+static inline void mm_cid_transit_to_task(struct task_struct *t, struct mm_cid_pcpu *pcp)
+{
+	if (cid_on_cpu(t->mm_cid.cid)) {
+		unsigned int cid = cpu_cid_to_cid(t->mm_cid.cid);
+
+		t->mm_cid.cid = cid_to_transit_cid(cid);
+		pcp->cid = t->mm_cid.cid;
+	}
+}
+
+static void __maybe_unused mm_cid_fixup_cpus_to_tasks(struct mm_struct *mm)
+{
+	unsigned int cpu;
+
+	/* Walk the CPUs and fixup all stale CIDs */
+	for_each_possible_cpu(cpu) {
+		struct mm_cid_pcpu *pcp = per_cpu_ptr(mm->mm_cid.pcpu, cpu);
+		struct rq *rq = cpu_rq(cpu);
+
+		/* Remote access to mm::mm_cid::pcpu requires rq_lock */
+		guard(rq_lock_irq)(rq);
+		/* Is the CID still owned by the CPU? */
+		if (cid_on_cpu(pcp->cid)) {
+			/*
+			 * If rq->curr has @mm, transfer it with the
+			 * transition bit set. Otherwise drop it.
+			 */
+			if (rq->curr->mm == mm && rq->curr->mm_cid.active)
+				mm_cid_transit_to_task(rq->curr, pcp);
+			else
+				mm_drop_cid_on_cpu(mm, pcp);
+
+		} else if (rq->curr->mm == mm && rq->curr->mm_cid.active) {
+			unsigned int cid = rq->curr->mm_cid.cid;
+
+			/* Ensure it has the transition bit set */
+			if (!cid_in_transit(cid)) {
+				cid = cid_to_transit_cid(cid);
+				rq->curr->mm_cid.cid = cid;
+				pcp->cid = cid;
+			}
+		}
+	}
+	/* Clear the transition bit */
+	WRITE_ONCE(mm->mm_cid.transit, 0);
+}
+
+static inline void mm_cid_transfer_to_cpu(struct task_struct *t, struct mm_cid_pcpu *pcp)
+{
+	if (cid_on_task(t->mm_cid.cid)) {
+		t->mm_cid.cid = cid_to_cpu_cid(t->mm_cid.cid);
+		pcp->cid = t->mm_cid.cid;
+	}
+}
+
+static bool mm_cid_fixup_task_to_cpu(struct task_struct *t, struct mm_struct *mm)
+{
+	/* Remote access to mm::mm_cid::pcpu requires rq_lock */
+	guard(task_rq_lock)(t);
+	/* If the task is not active it is not in the users count */
+	if (!t->mm_cid.active)
+		return false;
+	if (cid_on_task(t->mm_cid.cid)) {
+		/* If running on the CPU, transfer the CID, otherwise drop it */
+		if (task_rq(t)->curr == t)
+			mm_cid_transfer_to_cpu(t, per_cpu_ptr(mm->mm_cid.pcpu, task_cpu(t)));
+		else
+			mm_unset_cid_on_task(t);
+	}
+	return true;
+}
+
+static void __maybe_unused mm_cid_fixup_tasks_to_cpus(void)
+{
+	struct mm_struct *mm = current->mm;
+	struct task_struct *p, *t;
+	unsigned int users;
+
+	/*
+	 * This can obviously race with a concurrent affinity change, which
+	 * increases the number of allowed CPUs for this mm, but that does
+	 * not affect the mode and only changes the CID constraints. A
+	 * possible switch back to per task mode happens either in the
+	 * deferred handler function or in the next fork()/exit().
+	 *
+	 * The caller has already transferred. The newly incoming task is
+	 * already accounted for, but not yet visible.
+	 */
+	users = mm->mm_cid.users - 2;
+	if (!users)
+		return;
+
+	guard(rcu)();
+	for_other_threads(current, t) {
+		if (mm_cid_fixup_task_to_cpu(t, mm))
+			users--;
+	}
+
+	if (!users)
+		return;
+
+	/* Happens only for VM_CLONE processes. */
+	for_each_process_thread(p, t) {
+		if (t == current || t->mm != mm)
+			continue;
+		if (mm_cid_fixup_task_to_cpu(t, mm)) {
+			if (--users == 0)
+				return;
+		}
+	}
+}
+
+static bool sched_mm_cid_add_user(struct task_struct *t, struct mm_struct *mm)
+{
+	t->mm_cid.active = 1;
+	mm->mm_cid.users++;
+	return mm_update_max_cids(mm);
 }
 
 void sched_mm_cid_fork(struct task_struct *t)
@@ -10442,12 +10668,19 @@ void sched_mm_cid_fork(struct task_struc
 	WARN_ON_ONCE(!mm || t->mm_cid.cid != MM_CID_UNSET);
 
 	guard(mutex)(&mm->mm_cid.mutex);
-	guard(raw_spinlock)(&mm->mm_cid.lock);
-	t->mm_cid.active = 1;
-	mm->mm_cid.users++;
-	/* Preset last_cid for mm_cid_select() */
-	t->mm_cid.last_cid = READ_ONCE(mm->mm_cid.max_cids) - 1;
-	mm_update_max_cids(mm);
+	scoped_guard(raw_spinlock, &mm->mm_cid.lock) {
+		sched_mm_cid_add_user(t, mm);
+		/* Preset last_cid for mm_cid_select() */
+		t->mm_cid.last_cid = mm->mm_cid.max_cids - 1;
+	}
+}
+
+static bool sched_mm_cid_remove_user(struct task_struct *t)
+{
+	t->mm_cid.active = 0;
+	mm_unset_cid_on_task(t);
+	t->mm->mm_cid.users--;
+	return mm_update_max_cids(t->mm);
 }
 
 /*
@@ -10462,14 +10695,8 @@ void sched_mm_cid_exit(struct task_struc
 		return;
 
 	guard(mutex)(&mm->mm_cid.mutex);
-	guard(raw_spinlock)(&mm->mm_cid.lock);
-	t->mm_cid.active = 0;
-	mm->mm_cid.users--;
-	if (t->mm_cid.cid != MM_CID_UNSET) {
-		clear_bit(t->mm_cid.cid, mm_cidmask(mm));
-		t->mm_cid.cid = MM_CID_UNSET;
-	}
-	mm_update_max_cids(mm);
+	scoped_guard(raw_spinlock, &mm->mm_cid.lock)
+		sched_mm_cid_remove_user(t);
 }
 
 /* Deactivate MM CID allocation across execve() */
@@ -10498,6 +10725,8 @@ void mm_init_cid(struct mm_struct *mm, s
 	mm->mm_cid.percpu = 0;
 	mm->mm_cid.nr_cpus_allowed = p->nr_cpus_allowed;
 	mm->mm_cid.users = 0;
+	mm->mm_cid.pcpu_thrs = 0;
+	mm->mm_cid.update_deferred = 0;
 	raw_spin_lock_init(&mm->mm_cid.lock);
 	mutex_init(&mm->mm_cid.mutex);
 	cpumask_copy(mm_cpus_allowed(mm), &p->cpus_mask);


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

* [patch V3 18/20] irqwork: Move data struct to a types header
  2025-10-29 13:08 [patch V3 00/20] sched: Rewrite MM CID management Thomas Gleixner
                   ` (16 preceding siblings ...)
  2025-10-29 13:09 ` [patch V3 17/20] sched/mmcid: Provide CID ownership mode fixup functions Thomas Gleixner
@ 2025-10-29 13:09 ` Thomas Gleixner
  2025-10-30 15:52   ` Mathieu Desnoyers
  2025-10-29 13:09 ` [patch V3 19/20] sched/mmcid: Implement deferred mode change Thomas Gleixner
                   ` (3 subsequent siblings)
  21 siblings, 1 reply; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-29 13:09 UTC (permalink / raw)
  To: LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov, Shrikanth Hegde

... to avoid header recursion hell.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 include/linux/irq_work.h       |    9 ++-------
 include/linux/irq_work_types.h |   14 ++++++++++++++
 2 files changed, 16 insertions(+), 7 deletions(-)

--- a/include/linux/irq_work.h
+++ b/include/linux/irq_work.h
@@ -2,8 +2,9 @@
 #ifndef _LINUX_IRQ_WORK_H
 #define _LINUX_IRQ_WORK_H
 
-#include <linux/smp_types.h>
+#include <linux/irq_work_types.h>
 #include <linux/rcuwait.h>
+#include <linux/smp_types.h>
 
 /*
  * An entry can be in one of four states:
@@ -14,12 +15,6 @@
  * busy      NULL, 2 -> {free, claimed} : callback in progress, can be claimed
  */
 
-struct irq_work {
-	struct __call_single_node node;
-	void (*func)(struct irq_work *);
-	struct rcuwait irqwait;
-};
-
 #define __IRQ_WORK_INIT(_func, _flags) (struct irq_work){	\
 	.node = { .u_flags = (_flags), },			\
 	.func = (_func),					\
--- /dev/null
+++ b/include/linux/irq_work_types.h
@@ -0,0 +1,14 @@
+/* SPDX-License-Identifier: GPL-2.0 */
+#ifndef _LINUX_IRQ_WORK_TYPES_H
+#define _LINUX_IRQ_WORK_TYPES_H
+
+#include <linux/smp_types.h>
+#include <linux/types.h>
+
+struct irq_work {
+	struct __call_single_node	node;
+	void				(*func)(struct irq_work *);
+	struct rcuwait			irqwait;
+};
+
+#endif


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

* [patch V3 19/20] sched/mmcid: Implement deferred mode change
  2025-10-29 13:08 [patch V3 00/20] sched: Rewrite MM CID management Thomas Gleixner
                   ` (17 preceding siblings ...)
  2025-10-29 13:09 ` [patch V3 18/20] irqwork: Move data struct to a types header Thomas Gleixner
@ 2025-10-29 13:09 ` Thomas Gleixner
  2025-10-30 15:56   ` Mathieu Desnoyers
  2025-10-29 13:09 ` [patch V3 20/20] sched/mmcid: Switch over to the new mechanism Thomas Gleixner
                   ` (2 subsequent siblings)
  21 siblings, 1 reply; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-29 13:09 UTC (permalink / raw)
  To: LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov, Shrikanth Hegde

When affinity changes cause an increase of the number of CPUs allowed for
tasks which are related to a MM, that might results in a situation where
the ownership mode can go back from per CPU mode to per task mode.

As affinity changes happen with runqueue lock held there is no way to do
the actual mode change and required fixup right there.

Add the infrastructure to defer it to a workqueue. The scheduled work can
race with a fork() or exit(). Whatever happens first takes care of it.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 include/linux/rseq_types.h |    8 ++++++
 kernel/sched/core.c        |   58 +++++++++++++++++++++++++++++++++++++++------
 2 files changed, 59 insertions(+), 7 deletions(-)

--- a/include/linux/rseq_types.h
+++ b/include/linux/rseq_types.h
@@ -2,7 +2,9 @@
 #ifndef _LINUX_RSEQ_TYPES_H
 #define _LINUX_RSEQ_TYPES_H
 
+#include <linux/irq_work_types.h>
 #include <linux/types.h>
+#include <linux/workqueue_types.h>
 
 #ifdef CONFIG_RSEQ
 struct rseq;
@@ -129,6 +131,8 @@ struct mm_cid_pcpu {
  *			do not actually share the MM.
  * @pcpu_thrs:		Threshold for switching back from per CPU mode
  * @update_deferred:	A deferred switch back to per task mode is pending.
+ * @irq_work:		irq_work to handle the affinity mode change case
+ * @work:		Regular work to handle the affinity mode change case
  * @lock:		Spinlock to protect against affinity setting which can't take @mutex
  * @mutex:		Mutex to serialize forks and exits related to this mm
  */
@@ -145,6 +149,10 @@ struct mm_mm_cid {
 	unsigned int		pcpu_thrs;
 	unsigned int		update_deferred;
 
+	/* Rarely used. Moves @lock and @mutex into the second cacheline */
+	struct irq_work		irq_work;
+	struct work_struct	work;
+
 	raw_spinlock_t		lock;
 	struct mutex		mutex;
 }____cacheline_aligned_in_smp;
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -10536,8 +10536,17 @@ static inline void mm_update_cpus_allowe
 
 	/* Adjust the threshold to the wider set */
 	mc->pcpu_thrs = mm_cid_calc_pcpu_thrs(mc);
+	/* Switch back to per task mode? */
+	if (mc->users >= mc->pcpu_thrs)
+		return;
+
+	/* Don't queue twice */
+	if (mc->update_deferred)
+		return;
 
-	/* Scheduling of deferred mode switch goes here */
+	/* Queue the irq work, which schedules the real work */
+	mc->update_deferred = true;
+	irq_work_queue(&mc->irq_work);
 }
 
 static inline void mm_cid_transit_to_task(struct task_struct *t, struct mm_cid_pcpu *pcp)
@@ -10550,7 +10559,7 @@ static inline void mm_cid_transit_to_tas
 	}
 }
 
-static void __maybe_unused mm_cid_fixup_cpus_to_tasks(struct mm_struct *mm)
+static void mm_cid_fixup_cpus_to_tasks(struct mm_struct *mm)
 {
 	unsigned int cpu;
 
@@ -10711,14 +10720,47 @@ void sched_mm_cid_after_execve(struct ta
 	mm_cid_select(t);
 }
 
-void mm_init_cid(struct mm_struct *mm, struct task_struct *p)
+static void mm_cid_work_fn(struct work_struct *work)
 {
-	struct mm_cid_pcpu __percpu *pcpu = mm->mm_cid.pcpu;
-	int cpu;
+	struct mm_struct *mm = container_of(work, struct mm_struct, mm_cid.work);
+
+	/* Make it compile, but not functional yet */
+	if (!IS_ENABLED(CONFIG_NEW_MM_CID))
+		return;
 
-	for_each_possible_cpu(cpu)
-		per_cpu_ptr(pcpu, cpu)->cid = MM_CID_UNSET;
+	guard(mutex)(&mm->mm_cid.mutex);
+	/* Did the last user task exit already? */
+	if (!mm->mm_cid.users)
+		return;
 
+	scoped_guard(raw_spinlock_irq, &mm->mm_cid.lock) {
+		/* Have fork() or exit() handled it already? */
+		if (!mm->mm_cid.update_deferred)
+			return;
+		/* This clears mm_cid::update_deferred */
+		if (!mm_update_max_cids(mm))
+			return;
+		/* Affinity changes can only switch back to task mode */
+		if (WARN_ON_ONCE(mm->mm_cid.percpu))
+			return;
+	}
+	mm_cid_fixup_cpus_to_tasks(mm);
+}
+
+static void mm_cid_irq_work(struct irq_work *work)
+{
+	struct mm_struct *mm = container_of(work, struct mm_struct, mm_cid.irq_work);
+
+	/*
+	 * Needs to be unconditional because mm_cid::lock cannot be held
+	 * when scheduling work as mm_update_cpus_allowed() nests inside
+	 * rq::lock and schedule_work() might end up in wakeup...
+	 */
+	schedule_work(&mm->mm_cid.work);
+}
+
+void mm_init_cid(struct mm_struct *mm, struct task_struct *p)
+{
 	mm->mm_cid.max_cids = 0;
 	mm->mm_cid.percpu = 0;
 	mm->mm_cid.nr_cpus_allowed = p->nr_cpus_allowed;
@@ -10727,6 +10769,8 @@ void mm_init_cid(struct mm_struct *mm, s
 	mm->mm_cid.update_deferred = 0;
 	raw_spin_lock_init(&mm->mm_cid.lock);
 	mutex_init(&mm->mm_cid.mutex);
+	mm->mm_cid.irq_work = IRQ_WORK_INIT_HARD(mm_cid_irq_work);
+	INIT_WORK(&mm->mm_cid.work, mm_cid_work_fn);
 	cpumask_copy(mm_cpus_allowed(mm), &p->cpus_mask);
 	bitmap_zero(mm_cidmask(mm), bitmap_size(num_possible_cpus()));
 }


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

* [patch V3 20/20] sched/mmcid: Switch over to the new mechanism
  2025-10-29 13:08 [patch V3 00/20] sched: Rewrite MM CID management Thomas Gleixner
                   ` (18 preceding siblings ...)
  2025-10-29 13:09 ` [patch V3 19/20] sched/mmcid: Implement deferred mode change Thomas Gleixner
@ 2025-10-29 13:09 ` Thomas Gleixner
  2025-10-30 16:07   ` Mathieu Desnoyers
  2025-11-24 12:10   ` Mark Brown
  2025-10-30  5:00 ` [patch V3 00/20] sched: Rewrite MM CID management Shrikanth Hegde
  2025-11-10 17:09 ` Gabriele Monaco
  21 siblings, 2 replies; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-29 13:09 UTC (permalink / raw)
  To: LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov, Shrikanth Hegde

Now that all pieces are in place, change the implementations of
sched_mm_cid_fork() and sched_mm_cid_exit() to adhere to the new strict
ownership scheme and switch context_switch() over to use the new
mm_cid_schedin() functionality.

The common case is that there is no mode change required, which makes
fork() and exit() just update the user count and the constraints.

In case that a new user would exceed the CID space limit the fork() context
handles the transition to per CPU mode with mm::mm_cid::mutex held. exit()
handles the transition back to per task mode when the user count drops
below the switch back threshold. fork() might also be forced to handle a
deferred switch back to per task mode, when a affinity change increased the
number of allowed CPUs enough.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 include/linux/rseq.h       |   19 -------
 include/linux/rseq_types.h |    8 +--
 kernel/fork.c              |    1 
 kernel/sched/core.c        |  115 ++++++++++++++++++++++++++++++++++++++-------
 kernel/sched/sched.h       |   76 -----------------------------
 5 files changed, 103 insertions(+), 116 deletions(-)

--- a/include/linux/rseq.h
+++ b/include/linux/rseq.h
@@ -84,24 +84,6 @@ static __always_inline void rseq_sched_s
 	t->rseq.event.ids_changed = true;
 }
 
-/*
- * Invoked from switch_mm_cid() in context switch when the task gets a MM
- * CID assigned.
- *
- * This does not raise TIF_NOTIFY_RESUME as that happens in
- * rseq_sched_switch_event().
- */
-static __always_inline void rseq_sched_set_task_mm_cid(struct task_struct *t, unsigned int cid)
-{
-	/*
-	 * Requires a comparison as the switch_mm_cid() code does not
-	 * provide a conditional for it readily. So avoid excessive updates
-	 * when nothing changes.
-	 */
-	if (t->rseq.ids.mm_cid != cid)
-		t->rseq.event.ids_changed = true;
-}
-
 /* Enforce a full update after RSEQ registration and when execve() failed */
 static inline void rseq_force_update(void)
 {
@@ -170,7 +152,6 @@ static inline void rseq_handle_slowpath(
 static inline void rseq_signal_deliver(struct ksignal *ksig, struct pt_regs *regs) { }
 static inline void rseq_sched_switch_event(struct task_struct *t) { }
 static inline void rseq_sched_set_ids_changed(struct task_struct *t) { }
-static inline void rseq_sched_set_task_mm_cid(struct task_struct *t, unsigned int cid) { }
 static inline void rseq_force_update(void) { }
 static inline void rseq_virt_userspace_exit(void) { }
 static inline void rseq_fork(struct task_struct *t, u64 clone_flags) { }
--- a/include/linux/rseq_types.h
+++ b/include/linux/rseq_types.h
@@ -101,18 +101,18 @@ struct rseq_data { };
 /**
  * struct sched_mm_cid - Storage for per task MM CID data
  * @active:	MM CID is active for the task
- * @cid:	The CID associated to the task
- * @last_cid:	The last CID associated to the task
+ * @cid:	The CID associated to the task either permanently or
+ *		borrowed from the CPU
  */
 struct sched_mm_cid {
 	unsigned int		active;
 	unsigned int		cid;
-	unsigned int		last_cid;
 };
 
 /**
  * struct mm_cid_pcpu - Storage for per CPU MM_CID data
- * @cid:	The CID associated to the CPU
+ * @cid:	The CID associated to the CPU either permanently or
+ *		while a task with a CID is running
  */
 struct mm_cid_pcpu {
 	unsigned int	cid;
--- a/kernel/fork.c
+++ b/kernel/fork.c
@@ -956,7 +956,6 @@ static struct task_struct *dup_task_stru
 
 #ifdef CONFIG_SCHED_MM_CID
 	tsk->mm_cid.cid = MM_CID_UNSET;
-	tsk->mm_cid.last_cid = MM_CID_UNSET;
 	tsk->mm_cid.active = 0;
 #endif
 	return tsk;
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -5307,7 +5307,7 @@ context_switch(struct rq *rq, struct tas
 		}
 	}
 
-	switch_mm_cid(prev, next);
+	mm_cid_switch_to(prev, next);
 
 	/*
 	 * Tell rseq that the task was scheduled in. Must be after
@@ -10623,7 +10623,7 @@ static bool mm_cid_fixup_task_to_cpu(str
 	return true;
 }
 
-static void __maybe_unused mm_cid_fixup_tasks_to_cpus(void)
+static void mm_cid_fixup_tasks_to_cpus(void)
 {
 	struct mm_struct *mm = current->mm;
 	struct task_struct *p, *t;
@@ -10673,25 +10673,81 @@ static bool sched_mm_cid_add_user(struct
 void sched_mm_cid_fork(struct task_struct *t)
 {
 	struct mm_struct *mm = t->mm;
+	bool percpu;
 
 	WARN_ON_ONCE(!mm || t->mm_cid.cid != MM_CID_UNSET);
 
 	guard(mutex)(&mm->mm_cid.mutex);
-	scoped_guard(raw_spinlock, &mm->mm_cid.lock) {
-		sched_mm_cid_add_user(t, mm);
-		/* Preset last_cid for mm_cid_select() */
-		t->mm_cid.last_cid = mm->mm_cid.max_cids - 1;
+	scoped_guard(raw_spinlock_irq, &mm->mm_cid.lock) {
+		struct mm_cid_pcpu *pcp = this_cpu_ptr(mm->mm_cid.pcpu);
+
+		/* First user ? */
+		if (!mm->mm_cid.users) {
+			sched_mm_cid_add_user(t, mm);
+			t->mm_cid.cid = mm_get_cid(mm);
+			/* Required for execve() */
+			pcp->cid = t->mm_cid.cid;
+			return;
+		}
+
+		if (!sched_mm_cid_add_user(t, mm)) {
+			if (!mm->mm_cid.percpu)
+				t->mm_cid.cid = mm_get_cid(mm);
+			return;
+		}
+
+		/* Handle the mode change and transfer current's CID */
+		percpu = !!mm->mm_cid.percpu;
+		if (!percpu)
+			mm_cid_transit_to_task(current, pcp);
+		else
+			mm_cid_transfer_to_cpu(current, pcp);
+	}
+
+	if (percpu) {
+		mm_cid_fixup_tasks_to_cpus();
+	} else {
+		mm_cid_fixup_cpus_to_tasks(mm);
+		t->mm_cid.cid = mm_get_cid(mm);
 	}
 }
 
 static bool sched_mm_cid_remove_user(struct task_struct *t)
 {
 	t->mm_cid.active = 0;
-	mm_unset_cid_on_task(t);
+	scoped_guard(preempt) {
+		/* Clear the transition bit */
+		t->mm_cid.cid = cid_from_transit_cid(t->mm_cid.cid);
+		mm_unset_cid_on_task(t);
+	}
 	t->mm->mm_cid.users--;
 	return mm_update_max_cids(t->mm);
 }
 
+static bool __sched_mm_cid_exit(struct task_struct *t)
+{
+	struct mm_struct *mm = t->mm;
+
+	if (!sched_mm_cid_remove_user(t))
+		return false;
+	/*
+	 * Contrary to fork() this only deals with a switch back to per
+	 * task mode either because the above decreased users or an
+	 * affinity change increased the number of allowed CPUs and the
+	 * deferred fixup did not run yet.
+	 */
+	if (WARN_ON_ONCE(mm->mm_cid.percpu))
+		return false;
+	/*
+	 * A failed fork(2) cleanup never gets here, so @current must have
+	 * the same MM as @t. That's true for exit() and the failed
+	 * pthread_create() cleanup case.
+	 */
+	if (WARN_ON_ONCE(current->mm != mm))
+		return false;
+	return true;
+}
+
 /*
  * When a task exits, the MM CID held by the task is not longer required as
  * the task cannot return to user space.
@@ -10702,10 +10758,43 @@ void sched_mm_cid_exit(struct task_struc
 
 	if (!mm || !t->mm_cid.active)
 		return;
+	/*
+	 * Ensure that only one instance is doing MM CID operations within
+	 * a MM. The common case is uncontended. The rare fixup case adds
+	 * some overhead.
+	 */
+	scoped_guard(mutex, &mm->mm_cid.mutex) {
+		/* mm_cid::mutex is sufficient to protect mm_cid::users */
+		if (likely(mm->mm_cid.users > 1)) {
+			scoped_guard(raw_spinlock_irq, &mm->mm_cid.lock) {
+				if (!__sched_mm_cid_exit(t))
+					return;
+				/* Mode change required. Transfer currents CID */
+				mm_cid_transit_to_task(current, this_cpu_ptr(mm->mm_cid.pcpu));
+			}
+			mm_cid_fixup_cpus_to_tasks(mm);
+			return;
+		}
+		/* Last user */
+		scoped_guard(raw_spinlock_irq, &mm->mm_cid.lock) {
+			/* Required across execve() */
+			if (t == current)
+				mm_cid_transit_to_task(t, this_cpu_ptr(mm->mm_cid.pcpu));
+			/* Ignore mode change. There is nothing to do. */
+			sched_mm_cid_remove_user(t);
+		}
+	}
 
-	guard(mutex)(&mm->mm_cid.mutex);
-	scoped_guard(raw_spinlock, &mm->mm_cid.lock)
-		sched_mm_cid_remove_user(t);
+	/*
+	 * As this is the last user (execve(), process exit or failed
+	 * fork(2)) there is no concurrency anymore.
+	 *
+	 * Synchronize eventally pending work to ensure that there are no
+	 * dangling references left. @t->mm_cid.users is zero so nothing
+	 * can queue this work anymore.
+	 */
+	irq_work_sync(&mm->mm_cid.irq_work);
+	cancel_work_sync(&mm->mm_cid.work);
 }
 
 /* Deactivate MM CID allocation across execve() */
@@ -10718,18 +10807,12 @@ void sched_mm_cid_before_execve(struct t
 void sched_mm_cid_after_execve(struct task_struct *t)
 {
 	sched_mm_cid_fork(t);
-	guard(preempt)();
-	mm_cid_select(t);
 }
 
 static void mm_cid_work_fn(struct work_struct *work)
 {
 	struct mm_struct *mm = container_of(work, struct mm_struct, mm_cid.work);
 
-	/* Make it compile, but not functional yet */
-	if (!IS_ENABLED(CONFIG_NEW_MM_CID))
-		return;
-
 	guard(mutex)(&mm->mm_cid.mutex);
 	/* Did the last user task exit already? */
 	if (!mm->mm_cid.users)
--- a/kernel/sched/sched.h
+++ b/kernel/sched/sched.h
@@ -3744,83 +3744,7 @@ static inline void mm_cid_switch_to(stru
 	mm_cid_schedin(next);
 }
 
-/* Active implementation */
-static inline void init_sched_mm_cid(struct task_struct *t)
-{
-	struct mm_struct *mm = t->mm;
-	unsigned int max_cid;
-
-	if (!mm)
-		return;
-
-	/* Preset last_mm_cid */
-	max_cid = min_t(int, READ_ONCE(mm->mm_cid.nr_cpus_allowed), atomic_read(&mm->mm_users));
-	t->mm_cid.last_cid = max_cid - 1;
-}
-
-static inline bool __mm_cid_get(struct task_struct *t, unsigned int cid, unsigned int max_cids)
-{
-	struct mm_struct *mm = t->mm;
-
-	if (cid >= max_cids)
-		return false;
-	if (test_and_set_bit(cid, mm_cidmask(mm)))
-		return false;
-	t->mm_cid.cid = t->mm_cid.last_cid = cid;
-	__this_cpu_write(mm->mm_cid.pcpu->cid, cid);
-	return true;
-}
-
-static inline bool mm_cid_get(struct task_struct *t)
-{
-	struct mm_struct *mm = t->mm;
-	unsigned int max_cids;
-
-	max_cids = READ_ONCE(mm->mm_cid.max_cids);
-
-	/* Try to reuse the last CID of this task */
-	if (__mm_cid_get(t, t->mm_cid.last_cid, max_cids))
-		return true;
-
-	/* Try to reuse the last CID of this mm on this CPU */
-	if (__mm_cid_get(t, __this_cpu_read(mm->mm_cid.pcpu->cid), max_cids))
-		return true;
-
-	/* Try the first zero bit in the cidmask. */
-	return __mm_cid_get(t, find_first_zero_bit(mm_cidmask(mm), num_possible_cpus()), max_cids);
-}
-
-static inline void mm_cid_select(struct task_struct *t)
-{
-	/*
-	 * mm_cid_get() can fail when the maximum CID, which is determined
-	 * by min(mm->nr_cpus_allowed, mm->mm_users) changes concurrently.
-	 * That's a transient failure as there cannot be more tasks
-	 * concurrently on a CPU (or about to be scheduled in) than that.
-	 */
-	for (;;) {
-		if (mm_cid_get(t))
-			break;
-	}
-}
-
-static inline void switch_mm_cid(struct task_struct *prev, struct task_struct *next)
-{
-	if (prev->mm_cid.active) {
-		if (prev->mm_cid.cid != MM_CID_UNSET)
-			clear_bit(prev->mm_cid.cid, mm_cidmask(prev->mm));
-		prev->mm_cid.cid = MM_CID_UNSET;
-	}
-
-	if (next->mm_cid.active) {
-		mm_cid_select(next);
-		rseq_sched_set_task_mm_cid(next, next->mm_cid.cid);
-	}
-}
-
 #else /* !CONFIG_SCHED_MM_CID: */
-static inline void mm_cid_select(struct task_struct *t) { }
-static inline void switch_mm_cid(struct task_struct *prev, struct task_struct *next) { }
 static inline void mm_cid_switch_to(struct task_struct *prev, struct task_struct *next) { }
 #endif /* !CONFIG_SCHED_MM_CID */
 


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

* Re: [patch V3 02/20] sched/mmcid: Use proper data structures
  2025-10-29 13:08 ` [patch V3 02/20] sched/mmcid: Use proper data structures Thomas Gleixner
@ 2025-10-29 15:31   ` Mathieu Desnoyers
  0 siblings, 0 replies; 65+ messages in thread
From: Mathieu Desnoyers @ 2025-10-29 15:31 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On 2025-10-29 09:08, Thomas Gleixner wrote:
> Having a lot of CID functionality specific members in struct task_struct
> and struct mm_struct is not really making the code easier to read.
> 
> Encapsulate the CID specific parts in data structures and keep them
> seperate from the stuff they are embedded in.

seperate -> separate

Other than this nit:

Reviewed-by: Mathieu Desnoyers <mathieu.desnoyers@efficios.com>

-- 
Mathieu Desnoyers
EfficiOS Inc.
https://www.efficios.com

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

* Re: [patch V3 03/20] sched/mmcid: Cacheline align MM CID storage
  2025-10-29 13:08 ` [patch V3 03/20] sched/mmcid: Cacheline align MM CID storage Thomas Gleixner
@ 2025-10-29 15:39   ` Mathieu Desnoyers
  2025-10-29 21:09     ` Thomas Gleixner
  0 siblings, 1 reply; 65+ messages in thread
From: Mathieu Desnoyers @ 2025-10-29 15:39 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On 2025-10-29 09:08, Thomas Gleixner wrote:
[...]
>   struct mm_cid_pcpu {
>   	unsigned int	cid;
> -};
> +}____cacheline_aligned_in_smp;

What's the point in cacheline aligning this per-CPU variable ?
Should we expect other accesses to per-CPU variables sharing the
same cache line to update them frequently from remote CPUs ?

I did not cacheline align it expecting that per-CPU variables are
typically updated from their respective CPUs. So perhaps reality
don't match my expectations, but that's news to me.

> @@ -126,7 +126,7 @@ struct mm_mm_cid {
[...]
> -};
> +}____cacheline_aligned_in_smp;

OK for this cacheline align.

Thanks,

Mathieu


-- 
Mathieu Desnoyers
EfficiOS Inc.
https://www.efficios.com

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

* Re: [patch V3 04/20] sched: Fixup whitespace damage
  2025-10-29 13:09 ` [patch V3 04/20] sched: Fixup whitespace damage Thomas Gleixner
@ 2025-10-29 15:42   ` Mathieu Desnoyers
  2025-10-29 21:11     ` Thomas Gleixner
  0 siblings, 1 reply; 65+ messages in thread
From: Mathieu Desnoyers @ 2025-10-29 15:42 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On 2025-10-29 09:09, Thomas Gleixner wrote:
> With whitespace checks enabled in the editor this makes eyes bleed.
> 
[...]
> -	 *
> -	 * switch_mm_cid() needs to be updated if the barriers provided
> -	 * by context_switch() are modified.

This is also removing a comment, but the patch subject and commit
message clearly state that the intent is to fix whitespaces only.

This change should probably be moved to a different patch.

Thanks,

Mathieu


-- 
Mathieu Desnoyers
EfficiOS Inc.
https://www.efficios.com

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

* Re: [patch V3 05/20] sched/mmcid: Move scheduler code out of global header
  2025-10-29 13:09 ` [patch V3 05/20] sched/mmcid: Move scheduler code out of global header Thomas Gleixner
@ 2025-10-29 15:43   ` Mathieu Desnoyers
  0 siblings, 0 replies; 65+ messages in thread
From: Mathieu Desnoyers @ 2025-10-29 15:43 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On 2025-10-29 09:09, Thomas Gleixner wrote:
> This is only used in the scheduler core code, so there is no point to have
> it in a global header.

Reviewed-by: Mathieu Desnoyers <mathieu.desnoyers@efficios.com>

-- 
Mathieu Desnoyers
EfficiOS Inc.
https://www.efficios.com

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

* Re: [patch V3 06/20] sched/mmcid: Prevent pointless work in mm_update_cpus_allowed()
  2025-10-29 13:09 ` [patch V3 06/20] sched/mmcid: Prevent pointless work in mm_update_cpus_allowed() Thomas Gleixner
@ 2025-10-29 15:45   ` Mathieu Desnoyers
  0 siblings, 0 replies; 65+ messages in thread
From: Mathieu Desnoyers @ 2025-10-29 15:45 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On 2025-10-29 09:09, Thomas Gleixner wrote:
> mm_update_cpus_allowed() is not required to be invoked for affinity changes
> due to migrate_disable() and migrate_enable().
> 
> migrate_disable() restricts the task temporarily to a CPU on which the task
> was already allowed to run, so nothing changes. migrate_enable() restores
> the actual task affinity mask.
> 
> If that mask changed between migrate_disable() and migrate_enable() then
> that change was already accounted for.
> 
> Move the invocation to the proper place to avoid that.

Reviewed-by: Mathieu Desnoyers <mathieu.desnoyers@efficios.com>

-- 
Mathieu Desnoyers
EfficiOS Inc.
https://www.efficios.com

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

* Re: [patch V3 07/20] cpumask: Introduce cpumask_weighted_or()
  2025-10-29 13:09 ` [patch V3 07/20] cpumask: Introduce cpumask_weighted_or() Thomas Gleixner
@ 2025-10-29 15:49   ` Mathieu Desnoyers
  2025-11-03  9:15   ` Shrikanth Hegde
  1 sibling, 0 replies; 65+ messages in thread
From: Mathieu Desnoyers @ 2025-10-29 15:49 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On 2025-10-29 09:09, Thomas Gleixner wrote:
[...]
> Provide cpumask_weighted_or() and the corresponding bitmap functions which
> return the weight of the OR result right away.

Reviewed-by: Mathieu Desnoyers <mathieu.desnoyers@efficios.com>

-- 
Mathieu Desnoyers
EfficiOS Inc.
https://www.efficios.com

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

* Re: [patch V3 08/20] sched/mmcid: Use cpumask_weighted_or()
  2025-10-29 13:09 ` [patch V3 08/20] sched/mmcid: Use cpumask_weighted_or() Thomas Gleixner
@ 2025-10-29 15:51   ` Mathieu Desnoyers
  0 siblings, 0 replies; 65+ messages in thread
From: Mathieu Desnoyers @ 2025-10-29 15:51 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On 2025-10-29 09:09, Thomas Gleixner wrote:
> Use cpumask_weighted_or() instead of cpumask_or() and cpumask_weight() on
> the result, which walks the same bitmap twice. Results in 10-20% less
> cycles, which reduces the runqueue lock hold time.
Reviewed-by: Mathieu Desnoyers <mathieu.desnoyers@efficios.com>

-- 
Mathieu Desnoyers
EfficiOS Inc.
https://www.efficios.com

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

* Re: [patch V3 09/20] cpumask: Cache num_possible_cpus()
  2025-10-29 13:09 ` [patch V3 09/20] cpumask: Cache num_possible_cpus() Thomas Gleixner
@ 2025-10-29 15:54   ` Mathieu Desnoyers
  2025-10-29 21:11     ` Thomas Gleixner
  2025-11-03 10:06   ` Shrikanth Hegde
  1 sibling, 1 reply; 65+ messages in thread
From: Mathieu Desnoyers @ 2025-10-29 15:54 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On 2025-10-29 09:09, Thomas Gleixner wrote:
[...]
>   
> +void set_cpu_possible(unsigned int cpu, bool possible)

Unless I'm missing something, I suspect that "set_cpu_possible()" should
be marked as __init.

Thanks,

Mathieu

-- 
Mathieu Desnoyers
EfficiOS Inc.
https://www.efficios.com

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

* Re: [patch V3 03/20] sched/mmcid: Cacheline align MM CID storage
  2025-10-29 15:39   ` Mathieu Desnoyers
@ 2025-10-29 21:09     ` Thomas Gleixner
  2025-10-30 14:15       ` Mathieu Desnoyers
  0 siblings, 1 reply; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-29 21:09 UTC (permalink / raw)
  To: Mathieu Desnoyers, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On Wed, Oct 29 2025 at 11:39, Mathieu Desnoyers wrote:
> On 2025-10-29 09:08, Thomas Gleixner wrote:
> [...]
>>   struct mm_cid_pcpu {
>>   	unsigned int	cid;
>> -};
>> +}____cacheline_aligned_in_smp;
>
> What's the point in cacheline aligning this per-CPU variable ?
> Should we expect other accesses to per-CPU variables sharing the
> same cache line to update them frequently from remote CPUs ?
>
> I did not cacheline align it expecting that per-CPU variables are
> typically updated from their respective CPUs. So perhaps reality
> don't match my expectations, but that's news to me.

It depends. While per CPU variables are typically updated only on the
local CPU there are situations where there is cross CPU access and it
really depends in which proximity it ends up. I made it that way because
I saw the accesses prominent in perf top, which means there is
contention on the cache line.

Thanks,

        tglx

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

* Re: [patch V3 04/20] sched: Fixup whitespace damage
  2025-10-29 15:42   ` Mathieu Desnoyers
@ 2025-10-29 21:11     ` Thomas Gleixner
  0 siblings, 0 replies; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-29 21:11 UTC (permalink / raw)
  To: Mathieu Desnoyers, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On Wed, Oct 29 2025 at 11:42, Mathieu Desnoyers wrote:
> On 2025-10-29 09:09, Thomas Gleixner wrote:
>> With whitespace checks enabled in the editor this makes eyes bleed.
>> 
> [...]
>> -	 *
>> -	 * switch_mm_cid() needs to be updated if the barriers provided
>> -	 * by context_switch() are modified.
>
> This is also removing a comment, but the patch subject and commit
> message clearly state that the intent is to fix whitespaces only.
>
> This change should probably be moved to a different patch.

It was in a different patch and ended up here when reshuffling the
queue. I'll move it to the patch which gets rid of this dependency,
which is actually patch 1/N.

Thanks,

        tglx



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

* Re: [patch V3 09/20] cpumask: Cache num_possible_cpus()
  2025-10-29 15:54   ` Mathieu Desnoyers
@ 2025-10-29 21:11     ` Thomas Gleixner
  2025-11-01 22:59       ` Thomas Gleixner
  0 siblings, 1 reply; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-29 21:11 UTC (permalink / raw)
  To: Mathieu Desnoyers, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On Wed, Oct 29 2025 at 11:54, Mathieu Desnoyers wrote:
> On 2025-10-29 09:09, Thomas Gleixner wrote:
> [...]
>>   
>> +void set_cpu_possible(unsigned int cpu, bool possible)
>
> Unless I'm missing something, I suspect that "set_cpu_possible()" should
> be marked as __init.

Good point!

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

* Re: [patch V3 00/20] sched: Rewrite MM CID management
  2025-10-29 13:08 [patch V3 00/20] sched: Rewrite MM CID management Thomas Gleixner
                   ` (19 preceding siblings ...)
  2025-10-29 13:09 ` [patch V3 20/20] sched/mmcid: Switch over to the new mechanism Thomas Gleixner
@ 2025-10-30  5:00 ` Shrikanth Hegde
  2025-10-30  6:40   ` Shrikanth Hegde
  2025-11-10 17:09 ` Gabriele Monaco
  21 siblings, 1 reply; 65+ messages in thread
From: Shrikanth Hegde @ 2025-10-30  5:00 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov

Hi Thomas.

On 10/29/25 6:38 PM, Thomas Gleixner wrote:
> This is a follow up on V2 series which can be found here:
> 
>      https://lore.kernel.org/20251022104005.907410538@linutronix.de
> 
> The V1 cover letter contains a detailed analyisis of the issues:
> 
>      https://lore.kernel.org/20251015164952.694882104@linutronix.de
> 
> TLDR: The CID management is way to complex and adds significant overhead
> into scheduler hotpaths.
> 
> The series rewrites MM CID management in a more simplistic way which
> focusses on low overhead in the scheduler while maintaining per task CIDs
> as long as the number of threads is not exceeding the number of possible
> CPUs.
> 
> The series is based on the V6 series of the rseq rewrite:
> 
>      https://lore.kernel.org/20251027084220.785525188@linutronix.de
> 
> which is also available from git:
> 
>      git://git.kernel.org/pub/scm/linux/kernel/git/tip/tip.git core/rseq
> 
> The series on top of the tip core/rseq branch is available from git as
> well:
> 
>      git://git.kernel.org/pub/scm/linux/kernel/git/tglx/devel.git rseq/cid
> 
> Changes vs. V2:
> 
>     - Rename to cpumask/bitmap_weighted_or() - Yury
> 
>     - Zero the bitmap with length of bitmap_size(nr_possible_cpus()) -
>       Shrikanth
>     
>     - Move cpu_relax() out of for() as that fails to build when cpu_relax()
>       is a macro. - Shrikanth
> 
>     - Picked up Reviewed/Acked-by tags where appropriate
> 
> Thanks,
> 
> 	tglx
> ---
> Thomas Gleixner (20):
>        sched/mmcid: Revert the complex CID management
>        sched/mmcid: Use proper data structures
>        sched/mmcid: Cacheline align MM CID storage
>        sched: Fixup whitespace damage
>        sched/mmcid: Move scheduler code out of global header
>        sched/mmcid: Prevent pointless work in mm_update_cpus_allowed()
>        cpumask: Introduce cpumask_weighted_or()
>        sched/mmcid: Use cpumask_weighted_or()
>        cpumask: Cache num_possible_cpus()
>        sched/mmcid: Convert mm CID mask to a bitmap
>        signal: Move MMCID exit out of sighand lock
>        sched/mmcid: Move initialization out of line
>        sched/mmcid: Provide precomputed maximal value
>        sched/mmcid: Serialize sched_mm_cid_fork()/exit() with a mutex
>        sched/mmcid: Introduce per task/CPU ownership infrastrcuture
>        sched/mmcid: Provide new scheduler CID mechanism
>        sched/mmcid: Provide CID ownership mode fixup functions
>        irqwork: Move data struct to a types header
>        sched/mmcid: Implement deferred mode change
>        sched/mmcid: Switch over to the new mechanism
> 
>   include/linux/bitmap.h         |   15
>   include/linux/cpumask.h        |   26 +
>   include/linux/irq_work.h       |    9
>   include/linux/irq_work_types.h |   14
>   include/linux/mm_types.h       |  125 ------
>   include/linux/rseq.h           |   27 -
>   include/linux/rseq_types.h     |   71 +++
>   include/linux/sched.h          |   19
>   init/init_task.c               |    3
>   kernel/cpu.c                   |   15
>   kernel/exit.c                  |    1
>   kernel/fork.c                  |    7
>   kernel/sched/core.c            |  815 +++++++++++++++++++----------------------
>   kernel/sched/sched.h           |  395 ++++++++-----------
>   kernel/signal.c                |    2
>   lib/bitmap.c                   |    6
>   16 files changed, 727 insertions(+), 823 deletions(-)
> 
> 

I am running into crash at boot on power10 pseries.
Thought of putting it here first. Me trying to figure out why.

I am using your tree.
git://git.kernel.org/pub/scm/linux/kernel/git/tglx/devel.git

commit 789ff6e7cc5aa423473eb135f94812fe77b8aeab (HEAD -> rseq/cid, origin/rseq/cid)
Author: Thomas Gleixner <tglx@linutronix.de>
Date:   Tue Oct 14 10:51:04 2025 +0200

     sched/mmcid: Switch over to the new mechanism


Oops: Kernel access of bad area, sig: 7 [#3]
LE PAGE_SIZE=64K MMU=Radix  SMP NR_CPUS=8192 NUMA pSeries
Modules linked in: drm drm_panel_orientation_quirks xfs sd_mod sg ibmvscsi ibmveth scsi_transport_srp pseries_wdt dm_mirror dm_region_hash dm_log dm_mod fuse
CPU: 96 UID: 0 PID: 0 Comm: swapper/96 Tainted: G      D W           6.18.0-rc3+ #4 PREEMPT(lazy)
Tainted: [D]=DIE, [W]=WARN
NIP [c0000000001b5c10] mm_cid_switch_to+0x58/0x52c
LR [c000000001117c84] __schedule+0x4bc/0x760
Call Trace:
[c00000668367fde0] [c0000000001b53c8] __pick_next_task+0x60/0x2ac (unreliable)
[c00000668367fe40] [c000000001117a14] __schedule+0x24c/0x760
[c00000668367fee0] [c0000000011183d0] schedule_idle+0x3c/0x64
[c00000668367ff10] [c0000000001f2470] do_idle+0x15c/0x1ac
[c00000668367ff60] [c0000000001f2788] cpu_startup_entry+0x4c/0x50
[c00000668367ff90] [c00000000005ef20] start_secondary+0x284/0x288
[c00000668367ffe0] [c00000000000e158] start_secondary_prolog+0x10/0x14


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

* Re: [patch V3 00/20] sched: Rewrite MM CID management
  2025-10-30  5:00 ` [patch V3 00/20] sched: Rewrite MM CID management Shrikanth Hegde
@ 2025-10-30  6:40   ` Shrikanth Hegde
  2025-10-31 19:36     ` Thomas Gleixner
  0 siblings, 1 reply; 65+ messages in thread
From: Shrikanth Hegde @ 2025-10-30  6:40 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov



On 10/30/25 10:30 AM, Shrikanth Hegde wrote:
> Hi Thomas.

>> +----------------------
>>   kernel/sched/sched.h           |  395 ++++++++-----------
>>   kernel/signal.c                |    2
>>   lib/bitmap.c                   |    6
>>   16 files changed, 727 insertions(+), 823 deletions(-)
>>
>>
> 
> I am running into crash at boot on power10 pseries.
> Thought of putting it here first. Me trying to figure out why.
> 
> I am using your tree.
> git://git.kernel.org/pub/scm/linux/kernel/git/tglx/devel.git
> 
> commit 789ff6e7cc5aa423473eb135f94812fe77b8aeab (HEAD -> rseq/cid, 
> origin/rseq/cid)
> Author: Thomas Gleixner <tglx@linutronix.de>
> Date:   Tue Oct 14 10:51:04 2025 +0200
> 
>      sched/mmcid: Switch over to the new mechanism
> 
> 
> Oops: Kernel access of bad area, sig: 7 [#3]
> LE PAGE_SIZE=64K MMU=Radix  SMP NR_CPUS=8192 NUMA pSeries
> Modules linked in: drm drm_panel_orientation_quirks xfs sd_mod sg 
> ibmvscsi ibmveth scsi_transport_srp pseries_wdt dm_mirror dm_region_hash 
> dm_log dm_mod fuse
> CPU: 96 UID: 0 PID: 0 Comm: swapper/96 Tainted: G      D W           
> 6.18.0-rc3+ #4 PREEMPT(lazy)
> Tainted: [D]=DIE, [W]=WARN
> NIP [c0000000001b5c10] mm_cid_switch_to+0x58/0x52c
> LR [c000000001117c84] __schedule+0x4bc/0x760
> Call Trace:
> [c00000668367fde0] [c0000000001b53c8] __pick_next_task+0x60/0x2ac 
> (unreliable)
> [c00000668367fe40] [c000000001117a14] __schedule+0x24c/0x760
> [c00000668367fee0] [c0000000011183d0] schedule_idle+0x3c/0x64
> [c00000668367ff10] [c0000000001f2470] do_idle+0x15c/0x1ac
> [c00000668367ff60] [c0000000001f2788] cpu_startup_entry+0x4c/0x50
> [c00000668367ff90] [c00000000005ef20] start_secondary+0x284/0x288
> [c00000668367ffe0] [c00000000000e158] start_secondary_prolog+0x10/0x14
> 

Issue happens with NR_CPUS=8192. System boots fine with NR_CPUS=2048

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

* Re: [patch V3 10/20] sched/mmcid: Convert mm CID mask to a bitmap
  2025-10-29 13:09 ` [patch V3 10/20] sched/mmcid: Convert mm CID mask to a bitmap Thomas Gleixner
@ 2025-10-30 13:59   ` Mathieu Desnoyers
  0 siblings, 0 replies; 65+ messages in thread
From: Mathieu Desnoyers @ 2025-10-30 13:59 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On 2025-10-29 09:09, Thomas Gleixner wrote:
[...]
> @@ -1363,7 +1363,7 @@ static inline void mm_init_cid(struct mm
>   	mm->mm_cid.nr_cpus_allowed = p->nr_cpus_allowed;
>   	raw_spin_lock_init(&mm->mm_cid.lock);
>   	cpumask_copy(mm_cpus_allowed(mm), &p->cpus_mask);
> -	cpumask_clear(mm_cidmask(mm));
> +	bitmap_zero(mm_cidmask(mm), bitmap_size(num_possible_cpus()));

Then perhaps you'll also want to update mm_cid_size() to:

     return cpumask_size() + bitmap_size(num_possible_cpus());      /* mm_cpus_allowed(), mm_cidmask(). */

Thanks,

Mathieu

-- 
Mathieu Desnoyers
EfficiOS Inc.
https://www.efficios.com

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

* Re: [patch V3 12/20] sched/mmcid: Move initialization out of line
  2025-10-29 13:09 ` [patch V3 12/20] sched/mmcid: Move initialization out of line Thomas Gleixner
@ 2025-10-30 14:02   ` Mathieu Desnoyers
  0 siblings, 0 replies; 65+ messages in thread
From: Mathieu Desnoyers @ 2025-10-30 14:02 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On 2025-10-29 09:09, Thomas Gleixner wrote:
> It's getting bigger soon, so just move it out of line to the rest of the
> code.

Reviewed-by: Mathieu Desnoyers <mathieu.desnoyers@efficios.com>

-- 
Mathieu Desnoyers
EfficiOS Inc.
https://www.efficios.com

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

* Re: [patch V3 03/20] sched/mmcid: Cacheline align MM CID storage
  2025-10-29 21:09     ` Thomas Gleixner
@ 2025-10-30 14:15       ` Mathieu Desnoyers
  0 siblings, 0 replies; 65+ messages in thread
From: Mathieu Desnoyers @ 2025-10-30 14:15 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On 2025-10-29 17:09, Thomas Gleixner wrote:
> On Wed, Oct 29 2025 at 11:39, Mathieu Desnoyers wrote:
>> On 2025-10-29 09:08, Thomas Gleixner wrote:
>> [...]
>>>    struct mm_cid_pcpu {
>>>    	unsigned int	cid;
>>> -};
>>> +}____cacheline_aligned_in_smp;
>>
>> What's the point in cacheline aligning this per-CPU variable ?
>> Should we expect other accesses to per-CPU variables sharing the
>> same cache line to update them frequently from remote CPUs ?
>>
>> I did not cacheline align it expecting that per-CPU variables are
>> typically updated from their respective CPUs. So perhaps reality
>> don't match my expectations, but that's news to me.
> 
> It depends. While per CPU variables are typically updated only on the
> local CPU there are situations where there is cross CPU access and it
> really depends in which proximity it ends up. I made it that way because
> I saw the accesses prominent in perf top, which means there is
> contention on the cache line.
I did notice false sharing in the past within the mm_struct between
the mm_count field and the mm_cid percpu _pointer load_:

commit c1753fd02a00 ("mm: move mm_count into its own cache line")

Before understanding that this was actually the pointer load
that was false-sharing with mm_count, I initially thought that the
per-cpu memory somehow had false sharing because I was reading
the perf profiles incorrectly.

I just want to make sure that what you have identified in the perf
profiles is indeed false sharing of the per-cpu memory and not false
sharing of the per-cpu pointer load. Otherwise we'd been adding entirely
useless padding to percpu structures.

Note that in the current layout, atomic_t mm_users is right besides
the pcpu_cid percpu pointer, which may cause false sharing if mm_users
is updated often. But if that's indeed the culprit, then just adding
the cacheline align on the new struct mm_mm_cid suffices.

Thanks,

Mathieu

-- 
Mathieu Desnoyers
EfficiOS Inc.
https://www.efficios.com

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

* Re: [patch V3 13/20] sched/mmcid: Provide precomputed maximal value
  2025-10-29 13:09 ` [patch V3 13/20] sched/mmcid: Provide precomputed maximal value Thomas Gleixner
@ 2025-10-30 14:23   ` Mathieu Desnoyers
  2025-10-31 15:06     ` Thomas Gleixner
  0 siblings, 1 reply; 65+ messages in thread
From: Mathieu Desnoyers @ 2025-10-30 14:23 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On 2025-10-29 09:09, Thomas Gleixner wrote:
> Reading mm::mm_users and mm:::mm_cid::nr_cpus_allowed everytime to compute

every time

> + * @max_cids:		The exclusive maximum CID value for allocation and convergance

convergence

> + * @users:		The number of tasks sharing this MM. Seperate from mm::mm_users

Separate

> + *			as that is modified by mmget()/mm_put() by other entities which
> + *			do not actually share the MM.
>    * @lock:		Spinlock to protect all fields except @pcpu. It also protects
>    *			the MM cid cpumask and the MM cidmask bitmap.
>    */
>   struct mm_mm_cid {
>   	struct mm_cid_pcpu	__percpu *pcpu;
> +	unsigned int		max_cids;
>   	unsigned int		nr_cpus_allowed;
> +	unsigned int		users;

I suspect this reintroduces false-sharing between the "users"
and "lock" fields (updated every time a thread is forked/exits)
and load of the pcpu pointer which is pretty much immutable.
This will slow down accesses to the percpu data in the scheduler
fast path.

>   	raw_spinlock_t		lock;
>   }____cacheline_aligned_in_smp;
>   #else /* CONFIG_SCHED_MM_CID */

Thanks,

Mathieu

-- 
Mathieu Desnoyers
EfficiOS Inc.
https://www.efficios.com

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

* Re: [patch V3 14/20] sched/mmcid: Serialize sched_mm_cid_fork()/exit() with a mutex
  2025-10-29 13:09 ` [patch V3 14/20] sched/mmcid: Serialize sched_mm_cid_fork()/exit() with a mutex Thomas Gleixner
@ 2025-10-30 14:25   ` Mathieu Desnoyers
  0 siblings, 0 replies; 65+ messages in thread
From: Mathieu Desnoyers @ 2025-10-30 14:25 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On 2025-10-29 09:09, Thomas Gleixner wrote:
> Prepare for the new CID management scheme which puts the CID ownership
> transition into the fork() and exit() slow path by serializing
> sched_mm_cid_fork()/exit() with it, so task list and cpu mask walks can be
> done in interruptible and preemptible code.

Reviewed-by: Mathieu Desnoyers <mathieu.desnoyers@efficios.com>

-- 
Mathieu Desnoyers
EfficiOS Inc.
https://www.efficios.com

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

* Re: [patch V3 15/20] sched/mmcid: Introduce per task/CPU ownership infrastrcuture
  2025-10-29 13:09 ` [patch V3 15/20] sched/mmcid: Introduce per task/CPU ownership infrastrcuture Thomas Gleixner
@ 2025-10-30 14:45   ` Mathieu Desnoyers
  0 siblings, 0 replies; 65+ messages in thread
From: Mathieu Desnoyers @ 2025-10-30 14:45 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On 2025-10-29 09:09, Thomas Gleixner wrote:

Subject typo:

infrastrcuture -> infrastructure

> The MM CID management has two fundamental requirements:
> 
>    1) It has to guarantee that at no given point in time the same CID is
>       used by concurrent tasks in userspace.
> 
>    2) The CID space must not exceed the number of possible CPUs in a
>       system. While most allocators (glibc, tcmalloc, jemalloc) do not
>       care about that, there seems to be at least some LTTng library
>       depending on it.

About (2), it's not at the moment specifically about LTTng. The librseq project
depends on this limit within its per-cpu allocator:

https://git.kernel.org/pub/scm/libs/librseq/librseq.git/tree/src/rseq-mempool.c#n970

librseq is not used by LTTng right now. It is a library that was initially sitting
in the rseq kernel selftests, and I'm planning to eventually remove the code
duplication and have the rseq selftests just depend on librseq.

We are discussing with the BIND9 name server developers and they are interested
to start using the librseq per-cpu allocator within their project as well.

The LTTng userspace tracer will only be able to use the concept of "concurrency ID"
when/if we implement support for a "per-ipc-namespace concurrency id domain". We
basically need to have per-CID indexing of a per-cpu data structure over a shared
memory mapping shared across many processes within a namespace (IPC namespace seems
like a plausible candidate for this). So we'd have two "concurrency id" fields in
struct rseq: mm_cid (per-process CID), and ipc_ns_cid (per-namespace CID).

> milliseconds later when the next task changes it's affinity.

it's -> its

> penalazing everyone else.

penalize

> That allows to check for ownership trivialy and provides a simple check for

trivially

> + * MM_CID_ONCPU bit set. During transitioning from CPU to task ownership

transitioning -> transition

> + * actualy handed over to user space in the RSEQ memory.

actually

Thanks,

Mathieu


-- 
Mathieu Desnoyers
EfficiOS Inc.
https://www.efficios.com

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

* Re: [patch V3 16/20] sched/mmcid: Provide new scheduler CID mechanism
  2025-10-29 13:09 ` [patch V3 16/20] sched/mmcid: Provide new scheduler CID mechanism Thomas Gleixner
@ 2025-10-30 15:09   ` Mathieu Desnoyers
  0 siblings, 0 replies; 65+ messages in thread
From: Mathieu Desnoyers @ 2025-10-30 15:09 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On 2025-10-29 09:09, Thomas Gleixner wrote:
> The MM CID management has two fundamental requirements:

This explanation is duplicated verbatim between the previous
infrastructure patch and this one. I recommend keeping only
one of the two copies and adding a cross-reference in case
people are looking for details.

> @@ -130,7 +132,10 @@ struct mm_cid_pcpu {
>    * @mutex:		Mutex to serialize forks and exits related to this mm
>    */
>   struct mm_mm_cid {
> +	/* Hotpath read mostly members */
>   	struct mm_cid_pcpu	__percpu *pcpu;
> +	unsigned int		percpu;
> +	unsigned int		transit;
>   	unsigned int		max_cids;
>   	unsigned int		nr_cpus_allowed;
>   	unsigned int		users;

I'll repeat my concern about false-sharing between frequent "users"
updates and surrounding read-mostly fields.

> +
> +static __always_inline void mm_cid_schedout(struct task_struct *prev)
> +{
> +	/* During mode transitions CIDs are temporary and need to be dropped */
> +	if (likely(!cid_in_transit(prev->mm_cid.cid)))
> +		return;
> +
> +	mm_drop_cid(prev->mm, cid_from_transit_cid(prev->mm_cid.cid));
> +	prev->mm_cid.cid = MM_CID_UNSET;

I wonder why the transition from per-task to per-cpu needs to drop the
per-task CIDs. Can't we just move them to the per-cpu CID tracker
instead ? This would save us a "thundering herd" effect on the
cid mask allocation when switching from per-task to per-cpu.

Thanks,

Mathieu

-- 
Mathieu Desnoyers
EfficiOS Inc.
https://www.efficios.com

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

* Re: [patch V3 17/20] sched/mmcid: Provide CID ownership mode fixup functions
  2025-10-29 13:09 ` [patch V3 17/20] sched/mmcid: Provide CID ownership mode fixup functions Thomas Gleixner
@ 2025-10-30 15:51   ` Mathieu Desnoyers
  2025-10-31 16:54     ` Thomas Gleixner
  0 siblings, 1 reply; 65+ messages in thread
From: Mathieu Desnoyers @ 2025-10-30 15:51 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On 2025-10-29 09:09, Thomas Gleixner wrote:
> 
> At the point of switching to per CPU mode the new user is not yet visible
> in the system, so the task which initiated the fork() runs the fixup
> function: mm_cid_fixup_tasks_to_cpu() walks the thread list and either
> transfers each tasks owned CID to the CPU the task runs on or drops it into
> the CID pool if a task is not on a CPU at that point in time. Tasks which
> schedule in before the task walk reaches them do the handover in
> mm_cid_schedin(). When mm_cid_fixup_tasks_to_cpus() completes it's
> guaranteed that no task related to that MM owns a CID anymore.
> 
> Switching back to task mode happens when the user count goes below the
> threshold which was recorded on the per CPU mode switch:
> 
> 	pcpu_thrs = min(opt_cids - (opt_cids / 4), nr_cpu_ids / 2);
> 

AFAIU this provides an hysteresis so we don't switch back and
forth between modes if a single thread is forked/exits repeatedly,
right ?


> did not cover yet do the handover themself.

themselves

> 
> This transition from CPU to per task ownership happens in two phases:
> 
>   1) mm:mm_cid.transit contains MM_CID_TRANSIT. This is OR'ed on the task
>      CID and denotes that the CID is only temporarily owned by the
>      task. When it schedules out the task drops the CID back into the
>      pool if this bit is set.

OK, so the mm_drop_cid() on sched out only happens due to a transition
from per-cpu back to per-task. This answers my question in the previous
patch.


> 
>   2) The initiating context walks the per CPU space and after completion
>      clears mm:mm_cid.transit. After that point the CIDs are strictly
>      task owned again.
> 
> This two phase transition is required to prevent CID space exhaustion
> during the transition as a direct transfer of ownership would fail if
> two tasks are scheduled in on the same CPU before the fixup freed per
> CPU CIDs.

Clever. :-)


> + * Switching to per CPU mode happens when the user count becomes greater
> + * than the maximum number of CIDs, which is calculated by:
> + *
> + *	opt_cids = min(mm_cid::nr_cpus_allowed, mm_cid::users);
> + *	max_cids = min(1.25 * opt_cids, num_possible_cpus());
[...]
> + * Switching back to task mode happens when the user count goes below the
> + * threshold which was recorded on the per CPU mode switch:
> + *
> + *	pcpu_thrs = min(opt_cids - (opt_cids / 4), num_possible_cpus() / 2);

I notice that mm_update_cpus_allowed() calls __mm_update_max_cids() 
before updating the pcpu_thrs threshold.

sched_mm_cid_{add,remove}_user() only invoke mm_update_max_cids(mm)
without updating pcpu_thrs first.

Are those done on purpose ?

Thanks,

Mathieu



-- 
Mathieu Desnoyers
EfficiOS Inc.
https://www.efficios.com

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

* Re: [patch V3 18/20] irqwork: Move data struct to a types header
  2025-10-29 13:09 ` [patch V3 18/20] irqwork: Move data struct to a types header Thomas Gleixner
@ 2025-10-30 15:52   ` Mathieu Desnoyers
  0 siblings, 0 replies; 65+ messages in thread
From: Mathieu Desnoyers @ 2025-10-30 15:52 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On 2025-10-29 09:09, Thomas Gleixner wrote:
> ... to avoid header recursion hell.
> 
> Signed-off-by: Thomas Gleixner <tglx@linutronix.de>

Reviewed-by: Mathieu Desnoyers <mathieu.desnoyers@efficios.com>

-- 
Mathieu Desnoyers
EfficiOS Inc.
https://www.efficios.com

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

* Re: [patch V3 19/20] sched/mmcid: Implement deferred mode change
  2025-10-29 13:09 ` [patch V3 19/20] sched/mmcid: Implement deferred mode change Thomas Gleixner
@ 2025-10-30 15:56   ` Mathieu Desnoyers
  0 siblings, 0 replies; 65+ messages in thread
From: Mathieu Desnoyers @ 2025-10-30 15:56 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On 2025-10-29 09:09, Thomas Gleixner wrote:
> When affinity changes cause an increase of the number of CPUs allowed for
> tasks which are related to a MM, that might results in a situation where
> the ownership mode can go back from per CPU mode to per task mode.
> 
> As affinity changes happen with runqueue lock held there is no way to do
> the actual mode change and required fixup right there.
> 
> Add the infrastructure to defer it to a workqueue. The scheduled work can
> race with a fork() or exit(). Whatever happens first takes care of it.

Reviewed-by: Mathieu Desnoyers <mathieu.desnoyers@efficios.com>


-- 
Mathieu Desnoyers
EfficiOS Inc.
https://www.efficios.com

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

* Re: [patch V3 20/20] sched/mmcid: Switch over to the new mechanism
  2025-10-29 13:09 ` [patch V3 20/20] sched/mmcid: Switch over to the new mechanism Thomas Gleixner
@ 2025-10-30 16:07   ` Mathieu Desnoyers
  2025-10-31 16:57     ` Thomas Gleixner
  2025-11-24 12:10   ` Mark Brown
  1 sibling, 1 reply; 65+ messages in thread
From: Mathieu Desnoyers @ 2025-10-30 16:07 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On 2025-10-29 09:09, Thomas Gleixner wrote:

> @@ -10702,10 +10758,43 @@ void sched_mm_cid_exit(struct task_struc
>   
>   	if (!mm || !t->mm_cid.active)
>   		return;
> +	/*
> +	 * Ensure that only one instance is doing MM CID operations within
> +	 * a MM. The common case is uncontended. The rare fixup case adds
> +	 * some overhead.
> +	 */
> +	scoped_guard(mutex, &mm->mm_cid.mutex) {

When exiting from a mm where mm->mm_cid.users == 1 (read with
READ_ONCE()), can we do this without holding the mutex as an
optimization ?

> +	 * Synchronize eventally pending work to ensure that there are no

eventually

Thanks,

Mathieu

-- 
Mathieu Desnoyers
EfficiOS Inc.
https://www.efficios.com

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

* Re: [patch V3 13/20] sched/mmcid: Provide precomputed maximal value
  2025-10-30 14:23   ` Mathieu Desnoyers
@ 2025-10-31 15:06     ` Thomas Gleixner
  2025-10-31 15:14       ` Mathieu Desnoyers
  0 siblings, 1 reply; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-31 15:06 UTC (permalink / raw)
  To: Mathieu Desnoyers, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On Thu, Oct 30 2025 at 10:23, Mathieu Desnoyers wrote:
> On 2025-10-29 09:09, Thomas Gleixner wrote:
>>   struct mm_mm_cid {
>>   	struct mm_cid_pcpu	__percpu *pcpu;
>> +	unsigned int		max_cids;
>>   	unsigned int		nr_cpus_allowed;
>> +	unsigned int		users;
>
> I suspect this reintroduces false-sharing between the "users"
> and "lock" fields (updated every time a thread is forked/exits)
> and load of the pcpu pointer which is pretty much immutable.
> This will slow down accesses to the percpu data in the scheduler
> fast path.

At this point yes, but when all bits are in place then the lock fields
end up in a different cache line.

The false sharing issue vs. *pcpu and max_cids is minor, but I can move
the low frequency modified members past the work, so it does not matter
at all. The work stuff is rarely used, so there is no point to worry
about the occasional cache line contention on that.

Thanks,

        tglx

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

* Re: [patch V3 13/20] sched/mmcid: Provide precomputed maximal value
  2025-10-31 15:06     ` Thomas Gleixner
@ 2025-10-31 15:14       ` Mathieu Desnoyers
  0 siblings, 0 replies; 65+ messages in thread
From: Mathieu Desnoyers @ 2025-10-31 15:14 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On 2025-10-31 11:06, Thomas Gleixner wrote:
> On Thu, Oct 30 2025 at 10:23, Mathieu Desnoyers wrote:
>> On 2025-10-29 09:09, Thomas Gleixner wrote:
>>>    struct mm_mm_cid {
>>>    	struct mm_cid_pcpu	__percpu *pcpu;
>>> +	unsigned int		max_cids;
>>>    	unsigned int		nr_cpus_allowed;
>>> +	unsigned int		users;
>>
>> I suspect this reintroduces false-sharing between the "users"
>> and "lock" fields (updated every time a thread is forked/exits)
>> and load of the pcpu pointer which is pretty much immutable.
>> This will slow down accesses to the percpu data in the scheduler
>> fast path.
> 
> At this point yes, but when all bits are in place then the lock fields
> end up in a different cache line.
> 
> The false sharing issue vs. *pcpu and max_cids is minor, but I can move
> the low frequency modified members past the work, so it does not matter
> at all. The work stuff is rarely used, so there is no point to worry
> about the occasional cache line contention on that.

If we have alignment requirements on fields that matter for performance,
I recommend using __cacheline_group_{begin,end}_aligned() to make this
explicit. See include/linux/cache.h.

Thanks,

Mathieu



-- 
Mathieu Desnoyers
EfficiOS Inc.
https://www.efficios.com

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

* Re: [patch V3 17/20] sched/mmcid: Provide CID ownership mode fixup functions
  2025-10-30 15:51   ` Mathieu Desnoyers
@ 2025-10-31 16:54     ` Thomas Gleixner
  2025-10-31 19:15       ` Mathieu Desnoyers
  0 siblings, 1 reply; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-31 16:54 UTC (permalink / raw)
  To: Mathieu Desnoyers, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On Thu, Oct 30 2025 at 11:51, Mathieu Desnoyers wrote:
> On 2025-10-29 09:09, Thomas Gleixner wrote:
>> At the point of switching to per CPU mode the new user is not yet visible
>> in the system, so the task which initiated the fork() runs the fixup
>> function: mm_cid_fixup_tasks_to_cpu() walks the thread list and either
>> transfers each tasks owned CID to the CPU the task runs on or drops it into
>> the CID pool if a task is not on a CPU at that point in time. Tasks which
>> schedule in before the task walk reaches them do the handover in
>> mm_cid_schedin(). When mm_cid_fixup_tasks_to_cpus() completes it's
>> guaranteed that no task related to that MM owns a CID anymore.
>> 
>> Switching back to task mode happens when the user count goes below the
>> threshold which was recorded on the per CPU mode switch:
>> 
>> 	pcpu_thrs = min(opt_cids - (opt_cids / 4), nr_cpu_ids / 2);
>> 
>
> AFAIU this provides an hysteresis so we don't switch back and
> forth between modes if a single thread is forked/exits repeatedly,
> right ?

Yes. We could do that with a timer too, but the hysteresis worked fine
so far.

>> This transition from CPU to per task ownership happens in two phases:
>> 
>>   1) mm:mm_cid.transit contains MM_CID_TRANSIT. This is OR'ed on the task
>>      CID and denotes that the CID is only temporarily owned by the
>>      task. When it schedules out the task drops the CID back into the
>>      pool if this bit is set.
>
> OK, so the mm_drop_cid() on sched out only happens due to a transition
> from per-cpu back to per-task. This answers my question in the previous
> patch.

:)

>> + * Switching back to task mode happens when the user count goes below the
>> + * threshold which was recorded on the per CPU mode switch:
>> + *
>> + *	pcpu_thrs = min(opt_cids - (opt_cids / 4), num_possible_cpus() / 2);
>
> I notice that mm_update_cpus_allowed() calls __mm_update_max_cids() 
> before updating the pcpu_thrs threshold.
>
> sched_mm_cid_{add,remove}_user() only invoke mm_update_max_cids(mm)
> without updating pcpu_thrs first.
>
> Are those done on purpose ?

Yes. Update of pcpu_thrs is only possible when a resulting transition
can be handled in the context. max_cids update is always possible.

That's why mm_update_cpus_allowed() only updates max_cids and then
schedules work to defer a potential transition to the worker thread
context.

sched_mm_cid_{add,remove}_user() does:

    mm_update_max_cids()
      __mm_update_max_cids()    <- Updates max_cids

      update threshold and potentially switch ownership mode

As this holds the mutex it prevents new tasks coming in or other tasks
exiting until it managed the transition.

mm_cid_work_fn() does the same thing unless a
sched_mm_cid_{add,remove}_user() did not already handle it.

Thanks,

        tglx

      

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

* Re: [patch V3 20/20] sched/mmcid: Switch over to the new mechanism
  2025-10-30 16:07   ` Mathieu Desnoyers
@ 2025-10-31 16:57     ` Thomas Gleixner
  2025-10-31 19:17       ` Mathieu Desnoyers
  0 siblings, 1 reply; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-31 16:57 UTC (permalink / raw)
  To: Mathieu Desnoyers, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On Thu, Oct 30 2025 at 12:07, Mathieu Desnoyers wrote:
> On 2025-10-29 09:09, Thomas Gleixner wrote:
>
>> @@ -10702,10 +10758,43 @@ void sched_mm_cid_exit(struct task_struc
>>   
>>   	if (!mm || !t->mm_cid.active)
>>   		return;
>> +	/*
>> +	 * Ensure that only one instance is doing MM CID operations within
>> +	 * a MM. The common case is uncontended. The rare fixup case adds
>> +	 * some overhead.
>> +	 */
>> +	scoped_guard(mutex, &mm->mm_cid.mutex) {
>
> When exiting from a mm where mm->mm_cid.users == 1 (read with
> READ_ONCE()), can we do this without holding the mutex as an
> optimization ?

What's the optimization in that case? The mutex is uncontended and the
extra instructions for taking and releasing it are so trivial that you
can't measure it at all.

But aside of that this might race against a scheduled work which was
initiated by mm_update_cpus_allowed(). So keeping it strictly serialized
makes the code simple and race free :)

Thanks,

        tglx

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

* Re: [patch V3 17/20] sched/mmcid: Provide CID ownership mode fixup functions
  2025-10-31 16:54     ` Thomas Gleixner
@ 2025-10-31 19:15       ` Mathieu Desnoyers
  0 siblings, 0 replies; 65+ messages in thread
From: Mathieu Desnoyers @ 2025-10-31 19:15 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On 2025-10-31 12:54, Thomas Gleixner wrote:
> On Thu, Oct 30 2025 at 11:51, Mathieu Desnoyers wrote:

Reviewed-by: Mathieu Desnoyers <mathieu.desnoyers@efficios.com>

-- 
Mathieu Desnoyers
EfficiOS Inc.
https://www.efficios.com

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

* Re: [patch V3 20/20] sched/mmcid: Switch over to the new mechanism
  2025-10-31 16:57     ` Thomas Gleixner
@ 2025-10-31 19:17       ` Mathieu Desnoyers
  0 siblings, 0 replies; 65+ messages in thread
From: Mathieu Desnoyers @ 2025-10-31 19:17 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On 2025-10-31 12:57, Thomas Gleixner wrote:
> On Thu, Oct 30 2025 at 12:07, Mathieu Desnoyers wrote:
>> On 2025-10-29 09:09, Thomas Gleixner wrote:
>>
>>> @@ -10702,10 +10758,43 @@ void sched_mm_cid_exit(struct task_struc
>>>    
>>>    	if (!mm || !t->mm_cid.active)
>>>    		return;
>>> +	/*
>>> +	 * Ensure that only one instance is doing MM CID operations within
>>> +	 * a MM. The common case is uncontended. The rare fixup case adds
>>> +	 * some overhead.
>>> +	 */
>>> +	scoped_guard(mutex, &mm->mm_cid.mutex) {
>>
>> When exiting from a mm where mm->mm_cid.users == 1 (read with
>> READ_ONCE()), can we do this without holding the mutex as an
>> optimization ?
> 
> What's the optimization in that case? The mutex is uncontended and the
> extra instructions for taking and releasing it are so trivial that you
> can't measure it at all.

Fair enough.

> But aside of that this might race against a scheduled work which was
> initiated by mm_update_cpus_allowed(). So keeping it strictly serialized
> makes the code simple and race free :)

OK!

With the "eventally" -> "eventually" nit fixed:

Reviewed-by: Mathieu Desnoyers <mathieu.desnoyers@efficios.com>


-- 
Mathieu Desnoyers
EfficiOS Inc.
https://www.efficios.com

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

* Re: [patch V3 00/20] sched: Rewrite MM CID management
  2025-10-30  6:40   ` Shrikanth Hegde
@ 2025-10-31 19:36     ` Thomas Gleixner
  2025-11-01  7:56       ` Shrikanth Hegde
  0 siblings, 1 reply; 65+ messages in thread
From: Thomas Gleixner @ 2025-10-31 19:36 UTC (permalink / raw)
  To: Shrikanth Hegde, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov

On Thu, Oct 30 2025 at 12:10, Shrikanth Hegde wrote:
>> I am running into crash at boot on power10 pseries.
>> Thought of putting it here first. Me trying to figure out why.
>> 
>> I am using your tree.
>> git://git.kernel.org/pub/scm/linux/kernel/git/tglx/devel.git

Can you update and revalidate? There are a couple of fixes there though
I don't know how they would be related.

>> Oops: Kernel access of bad area, sig: 7 [#3]
>> LE PAGE_SIZE=64K MMU=Radix  SMP NR_CPUS=8192 NUMA pSeries
>> Modules linked in: drm drm_panel_orientation_quirks xfs sd_mod sg 
>> ibmvscsi ibmveth scsi_transport_srp pseries_wdt dm_mirror dm_region_hash 
>> dm_log dm_mod fuse
>> CPU: 96 UID: 0 PID: 0 Comm: swapper/96 Tainted: G      D W           
>> 6.18.0-rc3+ #4 PREEMPT(lazy)
>> Tainted: [D]=DIE, [W]=WARN
>> NIP [c0000000001b5c10] mm_cid_switch_to+0x58/0x52c

If it happens again, can you decode the source line?

>> LR [c000000001117c84] __schedule+0x4bc/0x760
>> Call Trace:
>> [c00000668367fde0] [c0000000001b53c8] __pick_next_task+0x60/0x2ac 
>> (unreliable)
>> [c00000668367fe40] [c000000001117a14] __schedule+0x24c/0x760
>> [c00000668367fee0] [c0000000011183d0] schedule_idle+0x3c/0x64
>> [c00000668367ff10] [c0000000001f2470] do_idle+0x15c/0x1ac
>> [c00000668367ff60] [c0000000001f2788] cpu_startup_entry+0x4c/0x50
>> [c00000668367ff90] [c00000000005ef20] start_secondary+0x284/0x288
>> [c00000668367ffe0] [c00000000000e158] start_secondary_prolog+0x10/0x14
>> 
>
> Issue happens with NR_CPUS=8192. System boots fine with NR_CPUS=2048

Hmm. Let me build a kernel with 8K and throw it at a VM then.

Thanks,

        tglx

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

* Re: [patch V3 00/20] sched: Rewrite MM CID management
  2025-10-31 19:36     ` Thomas Gleixner
@ 2025-11-01  7:56       ` Shrikanth Hegde
  2025-11-01 12:27         ` Thomas Gleixner
  0 siblings, 1 reply; 65+ messages in thread
From: Shrikanth Hegde @ 2025-11-01  7:56 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov



On 11/1/25 1:06 AM, Thomas Gleixner wrote:
> On Thu, Oct 30 2025 at 12:10, Shrikanth Hegde wrote:
>>> I am running into crash at boot on power10 pseries.
>>> Thought of putting it here first. Me trying to figure out why.
>>>
>>> I am using your tree.
>>> git://git.kernel.org/pub/scm/linux/kernel/git/tglx/devel.git
> 
> Can you update and revalidate? There are a couple of fixes there though
> I don't know how they would be related.

Tried with latest. It boots fine with NR_CPUS=8192.

at commit:
commit 870c1793316eddb6f8c9814f830f237e6e1c40ee (origin/rseq/cid)
Author: Thomas Gleixner <tglx@linutronix.de>
Date:   Tue Oct 14 10:51:04 2025 +0200

     sched/mmcid: Switch over to the new mechanism

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

* Re: [patch V3 00/20] sched: Rewrite MM CID management
  2025-11-01  7:56       ` Shrikanth Hegde
@ 2025-11-01 12:27         ` Thomas Gleixner
  0 siblings, 0 replies; 65+ messages in thread
From: Thomas Gleixner @ 2025-11-01 12:27 UTC (permalink / raw)
  To: Shrikanth Hegde, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov

On Sat, Nov 01 2025 at 13:26, Shrikanth Hegde wrote:
> On 11/1/25 1:06 AM, Thomas Gleixner wrote:
>> On Thu, Oct 30 2025 at 12:10, Shrikanth Hegde wrote:
>>>> I am running into crash at boot on power10 pseries.
>>>> Thought of putting it here first. Me trying to figure out why.
>>>>
>>>> I am using your tree.
>>>> git://git.kernel.org/pub/scm/linux/kernel/git/tglx/devel.git
>> 
>> Can you update and revalidate? There are a couple of fixes there though
>> I don't know how they would be related.
>
> Tried with latest. It boots fine with NR_CPUS=8192.

Thanks a lot!

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

* Re: [patch V3 09/20] cpumask: Cache num_possible_cpus()
  2025-10-29 21:11     ` Thomas Gleixner
@ 2025-11-01 22:59       ` Thomas Gleixner
  0 siblings, 0 replies; 65+ messages in thread
From: Thomas Gleixner @ 2025-11-01 22:59 UTC (permalink / raw)
  To: Mathieu Desnoyers, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, Shrikanth Hegde

On Wed, Oct 29 2025 at 22:11, Thomas Gleixner wrote:

> On Wed, Oct 29 2025 at 11:54, Mathieu Desnoyers wrote:
>> On 2025-10-29 09:09, Thomas Gleixner wrote:
>> [...]
>>>   
>>> +void set_cpu_possible(unsigned int cpu, bool possible)
>>
>> Unless I'm missing something, I suspect that "set_cpu_possible()" should
>> be marked as __init.
>
> Good point!

And only wishful thinking as set_cpu_possible() is wrongly used in code
which is not marked __init all over the architecture zoo. Cleaning that
up is yet another massive patch series dealing with mostly unmaintained
code. A nice task for people who want to get started with kernel
development. :)

On anything contemporary invoking set_cpu_possible() after init is going
to crash and burn because the related cpumask and variables are marked
RO at that point.

I've just addded a comment to that effect above the function to prevent
people from trying casually.

Thanks,

        tglx

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

* Re: [patch V3 07/20] cpumask: Introduce cpumask_weighted_or()
  2025-10-29 13:09 ` [patch V3 07/20] cpumask: Introduce cpumask_weighted_or() Thomas Gleixner
  2025-10-29 15:49   ` Mathieu Desnoyers
@ 2025-11-03  9:15   ` Shrikanth Hegde
  2025-11-03 13:29     ` Thomas Gleixner
  1 sibling, 1 reply; 65+ messages in thread
From: Shrikanth Hegde @ 2025-11-03  9:15 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov, LKML



On 10/29/25 6:39 PM, Thomas Gleixner wrote:
> CID management OR's two cpumasks and then calculates the weight on the
> result. That's inefficient as that has to walk the same stuff twice. As
> this is done with runqueue lock held, there is a real benefit of speeding
> this up. Depending on the system this results in 10-20% less cycles spent
> with runqueue lock held for a 4K cpumask.
> 
> Provide cpumask_weighted_or() and the corresponding bitmap functions which
> return the weight of the OR result right away.
> 
> Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
> Reviewed-by: Yury Norov (NVIDIA) <yury.norov@gmail.com>
> ---
> V3: Rename again - Yury
> V2: Rename and use the BITMAP_WEIGHT() macro - Yury
> ---
>   include/linux/bitmap.h  |   15 +++++++++++++++
>   include/linux/cpumask.h |   16 ++++++++++++++++
>   lib/bitmap.c            |    6 ++++++
>   3 files changed, 37 insertions(+)
> 
   
>   /**
> + * cpumask_weighted_or - *dstp = *src1p | *src2p and return the weight of the result
> + * @dstp: the cpumask result
> + * @src1p: the first input
> + * @src2p: the second input
> + *
> + * Return: The number of bits set in the resulting cpumask @dstp
> + */
> +static __always_inline
> +unsigned int cpumask_weighted_or(struct cpumask *dstp, const struct cpumask *src1p,
> +				 const struct cpumask *src2p)
> +{
> +	return bitmap_weighted_or(cpumask_bits(dstp), cpumask_bits(src1p),
> +				  cpumask_bits(src2p), small_cpumask_bits);
> +}

nit:

We have currently cpumask_weight_and & variants.
Wouldn't it be better to name it cpumask_weight_or ?

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

* Re: [patch V3 09/20] cpumask: Cache num_possible_cpus()
  2025-10-29 13:09 ` [patch V3 09/20] cpumask: Cache num_possible_cpus() Thomas Gleixner
  2025-10-29 15:54   ` Mathieu Desnoyers
@ 2025-11-03 10:06   ` Shrikanth Hegde
  2025-11-03 13:28     ` Thomas Gleixner
  1 sibling, 1 reply; 65+ messages in thread
From: Shrikanth Hegde @ 2025-11-03 10:06 UTC (permalink / raw)
  To: Thomas Gleixner, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov



On 10/29/25 6:39 PM, Thomas Gleixner wrote:
> Reevaluating num_possible_cpus() over and over does not make sense. That
> becomes a constant after init as cpu_possible_mask is marked ro_after_init.
> 
> Cache the value during initialization and provide that for consumption.
> 

Reviewed-by: Shrikanth Hegde <sshegde@linux.ibm.com>


Observation:
I see below usage which could be improved too?
kernel/irq/affinity.c:		set_vecs = cpumask_weight(cpu_possible_mask);
lib/tests/cpumask_kunit.c:	KUNIT_EXPECT_EQ_MSG(test, nr_cpu_ids, cpumask_weight(cpu_possible_mask),

Specially irq_calc_affinity_vectors, it seems to take cpus_read_lock, but I don't think
that lock is protecting possible cpus. possible cpus can't change after boot. No?

> Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
> Reviewed-by: Yury Norov <yury.norov@gmail.com>
> ---

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

* Re: [patch V3 09/20] cpumask: Cache num_possible_cpus()
  2025-11-03 10:06   ` Shrikanth Hegde
@ 2025-11-03 13:28     ` Thomas Gleixner
  0 siblings, 0 replies; 65+ messages in thread
From: Thomas Gleixner @ 2025-11-03 13:28 UTC (permalink / raw)
  To: Shrikanth Hegde, LKML
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov

On Mon, Nov 03 2025 at 15:36, Shrikanth Hegde wrote:
> On 10/29/25 6:39 PM, Thomas Gleixner wrote:
>> Reevaluating num_possible_cpus() over and over does not make sense. That
>> becomes a constant after init as cpu_possible_mask is marked ro_after_init.
>> 
>> Cache the value during initialization and provide that for consumption.
>> 
>
> Reviewed-by: Shrikanth Hegde <sshegde@linux.ibm.com>
>
>
> Observation:
> I see below usage which could be improved too?
> kernel/irq/affinity.c:		set_vecs = cpumask_weight(cpu_possible_mask);
> lib/tests/cpumask_kunit.c:	KUNIT_EXPECT_EQ_MSG(test, nr_cpu_ids, cpumask_weight(cpu_possible_mask),
>
> Specially irq_calc_affinity_vectors, it seems to take cpus_read_lock, but I don't think
> that lock is protecting possible cpus. possible cpus can't change after boot. No?

It can't. So yes the cpus_read_lock() is pointless.

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

* Re: [patch V3 07/20] cpumask: Introduce cpumask_weighted_or()
  2025-11-03  9:15   ` Shrikanth Hegde
@ 2025-11-03 13:29     ` Thomas Gleixner
  2025-11-10 16:11       ` Peter Zijlstra
  0 siblings, 1 reply; 65+ messages in thread
From: Thomas Gleixner @ 2025-11-03 13:29 UTC (permalink / raw)
  To: Shrikanth Hegde
  Cc: Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov, LKML

On Mon, Nov 03 2025 at 14:45, Shrikanth Hegde wrote:
> On 10/29/25 6:39 PM, Thomas Gleixner wrote:
>> +static __always_inline
>> +unsigned int cpumask_weighted_or(struct cpumask *dstp, const struct cpumask *src1p,
>> +				 const struct cpumask *src2p)
>> +{
>> +	return bitmap_weighted_or(cpumask_bits(dstp), cpumask_bits(src1p),
>> +				  cpumask_bits(src2p), small_cpumask_bits);
>> +}
>
> nit:
>
> We have currently cpumask_weight_and & variants.
> Wouldn't it be better to name it cpumask_weight_or ?

No. cpumask_weight_and() does weight(mask1 & mask2) but this does

    mask3 = mask1 | mask2;
    weight(mask3);

That's two very different things.

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

* Re: [patch V3 07/20] cpumask: Introduce cpumask_weighted_or()
  2025-11-03 13:29     ` Thomas Gleixner
@ 2025-11-10 16:11       ` Peter Zijlstra
  2025-11-10 16:42         ` Mathieu Desnoyers
  0 siblings, 1 reply; 65+ messages in thread
From: Peter Zijlstra @ 2025-11-10 16:11 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: Shrikanth Hegde, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov, LKML

On Mon, Nov 03, 2025 at 02:29:59PM +0100, Thomas Gleixner wrote:
> On Mon, Nov 03 2025 at 14:45, Shrikanth Hegde wrote:
> > On 10/29/25 6:39 PM, Thomas Gleixner wrote:
> >> +static __always_inline
> >> +unsigned int cpumask_weighted_or(struct cpumask *dstp, const struct cpumask *src1p,
> >> +				 const struct cpumask *src2p)
> >> +{
> >> +	return bitmap_weighted_or(cpumask_bits(dstp), cpumask_bits(src1p),
> >> +				  cpumask_bits(src2p), small_cpumask_bits);
> >> +}
> >
> > nit:
> >
> > We have currently cpumask_weight_and & variants.
> > Wouldn't it be better to name it cpumask_weight_or ?
> 
> No. cpumask_weight_and() does weight(mask1 & mask2) but this does

The comment was about naming, notable: s/_weighted_or/_weight_or/g to
better match the existing _weight_and().

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

* Re: [patch V3 07/20] cpumask: Introduce cpumask_weighted_or()
  2025-11-10 16:11       ` Peter Zijlstra
@ 2025-11-10 16:42         ` Mathieu Desnoyers
  0 siblings, 0 replies; 65+ messages in thread
From: Mathieu Desnoyers @ 2025-11-10 16:42 UTC (permalink / raw)
  To: Peter Zijlstra, Thomas Gleixner
  Cc: Shrikanth Hegde, Gabriele Monaco, Michael Jeanson, Jens Axboe,
	Paul E. McKenney, Gautham R. Shenoy, Florian Weimer, Tim Chen,
	Yury Norov, LKML

On 2025-11-10 11:11, Peter Zijlstra wrote:
> On Mon, Nov 03, 2025 at 02:29:59PM +0100, Thomas Gleixner wrote:
>> On Mon, Nov 03 2025 at 14:45, Shrikanth Hegde wrote:
>>> On 10/29/25 6:39 PM, Thomas Gleixner wrote:
>>>> +static __always_inline
>>>> +unsigned int cpumask_weighted_or(struct cpumask *dstp, const struct cpumask *src1p,
>>>> +				 const struct cpumask *src2p)
>>>> +{
>>>> +	return bitmap_weighted_or(cpumask_bits(dstp), cpumask_bits(src1p),
>>>> +				  cpumask_bits(src2p), small_cpumask_bits);
>>>> +}
>>>
>>> nit:
>>>
>>> We have currently cpumask_weight_and & variants.
>>> Wouldn't it be better to name it cpumask_weight_or ?
>>
>> No. cpumask_weight_and() does weight(mask1 & mask2) but this does
> 
> The comment was about naming, notable: s/_weighted_or/_weight_or/g to
> better match the existing _weight_and().

But if we go for "_weight_or" to match "_weight_and", we end up with
the following different semantics between "or" and "and":

cpumask_weight_and():
     inputs: mask1, mask2
     outputs: none

     return weight(mask1 & mask2);

cpumask_weight_or():
     inputs: mask1, mask2
     outputs: mask3

     mask3 = mask1 | mask2;
     return weight(mask3);

What we are trying to do here is apply a bitwise operation on two
inputs, write the resulting mask into mask3, *and* calculate the weight
as well, which is different from just calculating the weight.

Naming things is hard. I agree that the distinction between "weight" and
"weighted" is subtle.

Perhaps something along the lines of cpumask_eval_weight_or()
which would state the two operations performed (evaluate and calculate
the weight) could work ?

Thanks,

Mathieu

-- 
Mathieu Desnoyers
EfficiOS Inc.
https://www.efficios.com

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

* Re: [patch V3 00/20] sched: Rewrite MM CID management
  2025-10-29 13:08 [patch V3 00/20] sched: Rewrite MM CID management Thomas Gleixner
                   ` (20 preceding siblings ...)
  2025-10-30  5:00 ` [patch V3 00/20] sched: Rewrite MM CID management Shrikanth Hegde
@ 2025-11-10 17:09 ` Gabriele Monaco
  21 siblings, 0 replies; 65+ messages in thread
From: Gabriele Monaco @ 2025-11-10 17:09 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Peter Zijlstra, Mathieu Desnoyers, Michael Jeanson,
	Jens Axboe, Paul E. McKenney, Gautham R. Shenoy, Florian Weimer,
	Tim Chen, Yury Norov, Shrikanth Hegde

2025-10-29T13:09:04Z Thomas Gleixner <tglx@linutronix.de>:

> This is a follow up on V2 series which can be found here:
>

I confirm this series passes the selftest in [1] consistently and the observed latency spikes caused by task_mm_cid_work are gone.

Tested-by: Gabriele Monaco <gmonaco@redhat.com>

Thanks,
Gabriele

[1] - https://lore.kernel.org/lkml/20250929114225.36172-5-gmonaco@redhat.com

>     https://lore.kernel.org/20251022104005.907410538@linutronix.de
>
> The V1 cover letter contains a detailed analyisis of the issues:
>
>     https://lore.kernel.org/20251015164952.694882104@linutronix.de
>
> TLDR: The CID management is way to complex and adds significant overhead
> into scheduler hotpaths.
>
> The series rewrites MM CID management in a more simplistic way which
> focusses on low overhead in the scheduler while maintaining per task CIDs
> as long as the number of threads is not exceeding the number of possible
> CPUs.
>
> The series is based on the V6 series of the rseq rewrite:
>
>     https://lore.kernel.org/20251027084220.785525188@linutronix.de
>
> which is also available from git:
>
>     git://git.kernel.org/pub/scm/linux/kernel/git/tip/tip.git core/rseq
>
> The series on top of the tip core/rseq branch is available from git as
> well:
>
>     git://git.kernel.org/pub/scm/linux/kernel/git/tglx/devel.git rseq/cid
>
> Changes vs. V2:
>
>    - Rename to cpumask/bitmap_weighted_or() - Yury
>
>    - Zero the bitmap with length of bitmap_size(nr_possible_cpus()) -
>      Shrikanth
>   
>    - Move cpu_relax() out of for() as that fails to build when cpu_relax()
>      is a macro. - Shrikanth
>
>    - Picked up Reviewed/Acked-by tags where appropriate
>
> Thanks,
>
>     tglx
> ---
> Thomas Gleixner (20):
>       sched/mmcid: Revert the complex CID management
>       sched/mmcid: Use proper data structures
>       sched/mmcid: Cacheline align MM CID storage
>       sched: Fixup whitespace damage
>       sched/mmcid: Move scheduler code out of global header
>       sched/mmcid: Prevent pointless work in mm_update_cpus_allowed()
>       cpumask: Introduce cpumask_weighted_or()
>       sched/mmcid: Use cpumask_weighted_or()
>       cpumask: Cache num_possible_cpus()
>       sched/mmcid: Convert mm CID mask to a bitmap
>       signal: Move MMCID exit out of sighand lock
>       sched/mmcid: Move initialization out of line
>       sched/mmcid: Provide precomputed maximal value
>       sched/mmcid: Serialize sched_mm_cid_fork()/exit() with a mutex
>       sched/mmcid: Introduce per task/CPU ownership infrastrcuture
>       sched/mmcid: Provide new scheduler CID mechanism
>       sched/mmcid: Provide CID ownership mode fixup functions
>       irqwork: Move data struct to a types header
>       sched/mmcid: Implement deferred mode change
>       sched/mmcid: Switch over to the new mechanism
>
> include/linux/bitmap.h         |   15
> include/linux/cpumask.h        |   26 +
> include/linux/irq_work.h       |    9
> include/linux/irq_work_types.h |   14
> include/linux/mm_types.h       |  125 ------
> include/linux/rseq.h           |   27 -
> include/linux/rseq_types.h     |   71 +++
> include/linux/sched.h          |   19
> init/init_task.c               |    3
> kernel/cpu.c                   |   15
> kernel/exit.c                  |    1
> kernel/fork.c                  |    7
> kernel/sched/core.c            |  815 +++++++++++++++++++----------------------
> kernel/sched/sched.h           |  395 ++++++++-----------
> kernel/signal.c                |    2
> lib/bitmap.c                   |    6
> 16 files changed, 727 insertions(+), 823 deletions(-)


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

* Re: [patch V3 20/20] sched/mmcid: Switch over to the new mechanism
  2025-10-29 13:09 ` [patch V3 20/20] sched/mmcid: Switch over to the new mechanism Thomas Gleixner
  2025-10-30 16:07   ` Mathieu Desnoyers
@ 2025-11-24 12:10   ` Mark Brown
  2025-11-24 12:27     ` Florian Weimer
  1 sibling, 1 reply; 65+ messages in thread
From: Mark Brown @ 2025-11-24 12:10 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Peter Zijlstra, Gabriele Monaco, Mathieu Desnoyers,
	Michael Jeanson, Jens Axboe, Paul E. McKenney, Gautham R. Shenoy,
	Florian Weimer, Tim Chen, Yury Norov, Shrikanth Hegde

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

On Wed, Oct 29, 2025 at 02:09:32PM +0100, Thomas Gleixner wrote:
> Now that all pieces are in place, change the implementations of
> sched_mm_cid_fork() and sched_mm_cid_exit() to adhere to the new strict
> ownership scheme and switch context_switch() over to use the new
> mm_cid_schedin() functionality.
> 
> The common case is that there is no mode change required, which makes
> fork() and exit() just update the user count and the constraints.

I'm seeing boot failures on x86 qemu in -next which bisect to this
patch.  The boot gets to userspace but then grinds to a halt:

    https://lava.sirena.org.uk/scheduler/job/2118455#L542

which isn't terribly informative unfortunately.  I don't have any x86
hardware, and nothing else seems affected.

Bisect log:

# bad: [d724c6f85e80a23ed46b7ebc6e38b527c09d64f5] Add linux-next specific files for 20251121
# good: [abc1c7acb7daa90afde43c24e73844cf054cd11d] Merge branch 'for-linux-next-fixes' of https://gitlab.freedesktop.org/drm/misc/kernel.git
# good: [041e502d4ca66341696fcf54a76f3724b80f0b1f] ASoC: codecs: arizona: convert to snd_soc_dapm_xxx()
# good: [d5089fffe1db04a802b028c2ef4875be1ed452a3] ASoC: tas2781: Add tas2568/2574/5806m/5806md/5830 support
# good: [be2b723d94f2fa3f8ca028a55d313ba0b3007d44] gpio: improve support for shared GPIOs
# good: [bb52dc1d0342a4d2dccbfb1aedda019b8415cce1] ASoC: dt-bindings: ti,tas2781: Add TAS2568/2574/5806M/5806MD/5830 support
# good: [902f497a1ff5d275d2f48c2422d8a9eaa2174dee] ASoC: codecs: lpass-wsa-macro: remove useless gain read/write sequence
# good: [bdf96e9135a0cf53a853a19c30fa11131a744062] ASoC: codecs: lpass-rx-macro: fix mute_stream affecting all paths
# good: [c47f28ef62cb03de1e5a97844a90eda0415da233] ASoC: codecs: lpass-wsa-macro: remove mix path event
# good: [50c28498e9fd6784dea82378d509572d118111f9] ASoC: codecs: lpass-wsa-macro: remove unused WSA_MACRO_RX_MIX enum
# good: [c22f7a5cd205492c48c6deb72f5efe2ae63c931e] gpio: improve support for shared GPIOs
# good: [7ec95f46759ba0843a2695decba3cea028cb84ea] ASoC: codecs: lpass-wsa-macro: remove main path event
# good: [3ea1b0dbc684191b2e0b5697356a74571ab28155] ASoC: codecs: lpass-wsa-macro: fix path clock dependencies
# good: [38fc5addd2a0e51ba750c1a401efe7e3c84a2916] ASoC: codecs: lpass-wsa-macro: remove unnecessary bounds check
# good: [da49a21b3fe9fbee7be0418916c43f27ed91ad0a] ASoC: codecs: lpass-wsa-macro: add volume controls for mix path
# good: [b871d9adffe5a64a1fd9edcb1aebbcc995b17901] regulator: make the subsystem aware of shared GPIOs
# good: [7a0a87712120329c034b0aae88bdaa05bd046f10] ASoC: wsa883x: drop GPIOD_FLAGS_BIT_NONEXCLUSIVE flag from GPIO lookup
# good: [d01fbee5c0d3d3061fb16235b71f5a117128e2c1] ASoC: wsa881x: drop GPIOD_FLAGS_BIT_NONEXCLUSIVE flag from GPIO lookup
# good: [e2c48498a93404743e0565dcac29450fec02e6a3] ASoC: soc-core: Pre-check zero CPU/codec DAIs, handle early rtd->dais alloc failure
# good: [d9813cd23d5a7b254cc1b1c1ea042634d8da62e6] spi: sophgo: Fix incorrect use of bus width value macros
# good: [683c03b95c2ae95ff05ebf0dcc040bf3db633135] ASoC: mediatek: Add support for MT8189 SoC
# good: [7eb1535855983f67b95c2ba777d686cc17188285] ASoC: mediatek: mt8189: add platform driver
# good: [4980df101676f598ad515725a94424d244a5c5a3] ASoC: dt-bindings: mediatek,mt8189-nau8825: add mt8189-nau8825 document
# good: [dc637ffeed6c32c951520774897601ebd9ffc3d5] ASoC: mediatek: mt8189: support audio clock control
# good: [21e68bcb1b0c688c2d9ca0d457922febac650ac1] regulator: renesas-usb-vbus-regulator: Remove unused headers
# good: [0140fc11893bf22928a6f6ebcea96315671d75dc] ASoC: convert to snd_soc_dapm_xxx()
# good: [bd79452b39c21599e2cff42e9fbeb182656b6f6a] MAINTAINERS: adjust file entry in RISC-V MICROCHIP SUPPORT
# good: [402ff043395fd9444810c723056fe741c77dbc21] ASoC: mediatek: mt8189: support PCM in platform driver
# good: [d218ea171430e49412804efb794942dd121a8032] ASoC: mediatek: mt8189: add machine driver with nau8825
# good: [4dbb5f6e1b4eed64037d4462977c196acab2af16] spi: imx: add 16/32 bits per word support for target PIO mode
# good: [e45979641a9a9dbb48fc77c5b36a5091a92e7227] ASoC: SOF: sof-client-probes: Replace snprintf() with scnprintf()
# good: [f5cb3ee251b4f9db2761aced191f10579bd7e64e] ASoC: SDCA: Add companion amp Function
# good: [93218e3f2cfeb1e7992768dd3e64c9012e705992] regulator: dt-bindings: pca9540: add debounce timer configuration
# good: [e3acef6ef89ffbd6c80950d5fa0d024a8d11c1c4] ASoC: mediatek: mt8189: support ADDA in platform driver
# good: [22e9bd51e518e8fcad269c0c1f4bd84467db093e] ASoC: dt-bindings: mediatek,mt8189-afe-pcm: add audio afe document
# good: [34e437097247f92fba6fac3d6e40e33af5f32e3d] ASoC: mediatek: mt8189: support I2S in platform driver
# good: [81f8f29a48defd572ec34d2b9c84374fd05158cc] ASoC: mediatek: mt8189: add common header
# good: [9f202872ba04d71c96908c56abcc6e3f4a629a40] ASoC: mediatek: mt8189: support TDM in platform driver
# good: [94000534e0883b4f4ba9882e4630cfcdf2af539d] ASoC: SOF: imx9: use SCMI API for LM management
# good: [d9d0be59be2580f2c5e4b7217aafb980e8c371cf] regulator: pca9450: Add support for setting debounce settings
# good: [f2fb3f7bdf8b996ef7bd1ecc13c93c4dfc9b0d2a] spi: aspeed: Add AST2700 SoC support and Quad SPI
# good: [20772c4e0f0b58211ebdddfb8606694677c4c4c8] ASoC: Intel: avs: Allow for NHLT configuration
# good: [96498e804cb6629e02747336a0a33e4955449732] spi: davinci: remove platform data header
# good: [a060b8c511abb0997381b397e52149a5e3e5259a] gpiolib: implement low-level, shared GPIO support
# good: [1e4f6db614a310cc34d07ffbf031c76ea9581bcf] gpiolib: support shared GPIOs in core subsystem code
# good: [197b3f3c70d61ff1c7ca24f66d567e06fe8ed3d9] string: provide strends()
# good: [d4340ff75eaa083f261e16d49f13191236bfad06] gpiolib: define GPIOD_FLAG_SHARED
# good: [e511d484cbe44fe48a1b9f621f6a947c72503f9e] arm64: select HAVE_SHARED_GPIOS for ARCH_QCOM
# good: [eb374f764a7012eda28019266a6d9191670c4fa5] gpio: provide gpiod_is_shared()
# good: [e992d54c6f970b382ffeacd7c88f68b94a3c6caf] gpio: shared-proxy: implement the shared GPIO proxy driver
# good: [716d0a0a2ab00c601120c19bb357f4373f4722d1] spi: aspeed: Enable Quad SPI mode for page program
# good: [4422df6782eb7aa9725a3c09d9ba3c38ecc85df4] ASoC: ux500: mop500_ab8500: convert to snd_soc_dapm_xxx()
# good: [c69951603150c2b08f69feb8eb2475d58c1ef7cd] ASoC: ti: omap3pandora: convert to snd_soc_dapm_xxx()
# good: [508f3d3b688e1650ed383fe208b323aa6c164420] spi: aspeed: Use phys_addr_t for bus addresses to support 64-bit platforms
# good: [fefd4e0b26392ab7676b13b597166b6478eec4c6] ASoC: ti: rx51: convert to snd_soc_dapm_xxx()
# good: [be6671d3908e97a2128f5327610a1dcb4d420cfa] spi: dt-bindings: aspeed,ast2600-fmc: Add AST2700 SoC support
# good: [9e510e677090bb794b46348b10e1c8038286e00a] spi: aspeed: Add support for the AST2700 SPI controller
# good: [d10755f895eb295738c4db23ab970f8acaebd7ce] ASoC: sdw_utils: soc_sdw_rt_mf_sdca: convert to snd_soc_dapm_xxx()
# good: [575eb51c72251f73d21408829ed39b8b03e67894] ASoC: intel: boards: bytcr_rt5651: convert to snd_soc_dapm_xxx()
# good: [8686dd09358b92c2d3dc7b6d0cc893d1d72ef628] ASoC: qcom: q6routing: convert to snd_soc_dapm_xxx()
# good: [c2bdc0301db723b67b9c6cbd572f8df4b473cc4a] ASoC: codecs: wm9712: convert to snd_soc_dapm_xxx()
# good: [74c772546632500f64a4fc4b318cde9f87a29cf6] ASoC: meson: g12a-tohdmitx: convert to snd_soc_dapm_xxx()
# good: [9b5a1d32da7e343e8fbdfbc536770544551ef59b] ASoC: ti: n810: convert to snd_soc_dapm_xxx()
# good: [9c082ab3ebb8baa1f9f17a5c4a85d2345f5b305c] ASoC: intel: boards: bytcht_cx2072x: convert to snd_soc_dapm_xxx()
# good: [4cc345c56b2458d449bf2479cb72afbb8e9dfc1c] ASoC: intel: avs: rt5640: convert to snd_soc_dapm_xxx()
# good: [42e142c842cb9c7dbde569ff137974e76d1a0e67] ASoC: ti: omap-twl4030: convert to snd_soc_dapm_xxx()
# good: [d1cca4a46f6dfad9a383e5aa30dd8a63833d7a82] ASoC: samsung: speyside: convert to snd_soc_dapm_xxx()
# good: [771f522d64303751f976d888a9d6160a345042e2] ASoC: mediatek: mt8186-mt6366: convert to snd_soc_dapm_xxx()
# good: [2f8d6a86936e10bb65042f2c4da8c84e164b3693] ASoC: intel: avs: rt274: convert to snd_soc_dapm_xxx()
# good: [b10489e9da2850bcaa53117df1331dc371df8d35] ASoC: soc-dapm: convert to snd_soc_dapm_xxx()
# good: [b63f223ce0d402237ef339b8482ecc627320df3d] ASoC: mediatek: mtk-dsp-sof-common: convert to snd_soc_dapm_xxx()
# good: [1c4143ea606a60a11d623a0bd751c8b0af0119e0] ASoC: samsung: tobermory: convert to snd_soc_dapm_xxx()
# good: [2cc325972373bf86f561eaec45680d4ab957e4eb] ASoC: codecs: wm9090: convert to snd_soc_dapm_xxx()
# good: [aa851ed8746e20d4123efad82fa4ea3dd1f2b14e] ASoC: mediatek: mt8186-mt6366-common: convert to snd_soc_dapm_xxx()
# good: [cf3c9aa075313898ac1669d0214946f0af84d18b] ASoC: ti: ams-delta: convert to snd_soc_dapm_xxx()
# good: [5b54c4ae6c880024699a7c0391991e45ca053477] ASoC: sdw_utils: soc_sdw_utils: convert to snd_soc_dapm_xxx()
# good: [8c827c1c78b7260bfdd33cd8ddf82e3c19c24bb2] ASoC: intel: avs: es8336: convert to snd_soc_dapm_xxx()
# good: [d0764e1476da99ad781745b5b6a7d583cbf1d7e5] ASoC: sdw_utils: soc_sdw_cs_amp: convert to snd_soc_dapm_xxx()
# good: [7fb390727decac201fc9370bb14b17d91c334da6] ASoC: fsl: fsl-asoc-card: convert to snd_soc_dapm_xxx()
# good: [f74aa1e909e7ea960effb5d054cb9ff7e39d84b8] ASoC: qcom: q6usb: convert to snd_soc_dapm_xxx()
# good: [063e91a9141e708fe07a2a06d50fb0b3ed1ecbea] ASoC: mediatek: mtk-afe-platform-driver: convert to snd_soc_dapm_xxx()
# good: [be0c2c4010c2e641772408c2bce1649e5db20f2f] ASoC: intel: boards: cht_bsw_rt5672: convert to snd_soc_dapm_xxx()
# good: [6a995aaaa4b63c324116f03adb94985654a6cb6e] ASoC: meson: g12a-toacodec: convert to snd_soc_dapm_xxx()
# good: [a0593f6d1200bb17b69f85e89a9c6c6bcb2966b2] ASoC: meson: t9015: convert to snd_soc_dapm_xxx()
# good: [2bcf4235819c22b8a6dcbae9d9101d6202ea949c] ASoC: sdw_utils: soc_sdw_rt_sdca_jack_common: convert to snd_soc_dapm_xxx()
# good: [49b476efceb52a8cefd4e1c74672b55826680c4e] ASoC: sof-client-probes: convert to snd_soc_dapm_xxx()
# good: [a3fc90c52ad0d525918aa5039cb07d937dc1d841] ASoC: ti: j721e-evm: convert to snd_soc_dapm_xxx()
# good: [e70ef8a90d431157ad1b314a78125aab50eeda3e] ASoC: codecs: wm_hubs: convert to snd_soc_dapm_xxx()
# good: [26de56c382450acd5d397631f490dc2d50e7a9c4] ASoC: rockchip: rk3288_hdmi_analog: convert to snd_soc_dapm_xxx()
# good: [85216ea34568352c7c06ba7152ab1e9ccc73eff7] ASoC: intel: boards: sof_pcm512x: convert to snd_soc_dapm_xxx()
# good: [6603016167aace5d4c78bdfa5eb1f928a42d7e47] ASoC: intel: avs: control: convert to snd_soc_dapm_xxx()
# good: [84496142859a4c66aab3d82ce110952541614ef3] ASoC: intel: avs: nau8825: convert to snd_soc_dapm_xxx()
# good: [cacd21b61750c998f74e3d6aabfca1315298e699] ASoC: qcom: sc7180: convert to snd_soc_dapm_xxx()
# good: [bc8ec019290f00d605423db6ba5fd9db81e83bac] ASoC: sunxi: sun4i-codec: convert to snd_soc_dapm_xxx()
# good: [32fc5bf2ea987504d10afe329727ab8c6fd8c836] ASoC: meson: axg-spdifout: convert to snd_soc_dapm_xxx()
# good: [02697e7049ddca90e128a89ca276e3ccaeff447a] ASoC: samsung: aries_wm8994: convert to snd_soc_dapm_xxx()
# good: [fbada860c48112083c6212d2a0df5e4d919998c9] ASoC: sdw_utils: soc_sdw_dmic: convert to snd_soc_dapm_xxx()
# good: [6353596bb18588c5d24c96501081c1ef9cce5754] ASoC: sdw_utils: soc_sdw_rt700: convert to snd_soc_dapm_xxx()
# good: [2f558a84ced9e7ad6d5e53617d6848df9696ecd9] ASoC: codecs: wm9713: convert to snd_soc_dapm_xxx()
# good: [256cfc79911658a90caa7655e88c255255068ad1] ASoC: rockchip: rockchip_max98090: convert to snd_soc_dapm_xxx()
# good: [857c31f15cef81745855a53262b021bec95ca7a8] ASoC: samsung: bells: convert to snd_soc_dapm_xxx()
# good: [0e435dd54f5e3f4077ae3c8db1d1c3c2e5823a8a] ASoC: codecs: wm_adsp: convert to snd_soc_dapm_xxx()
# good: [d02bbabec7e330346b85a43c0f89ee61144985a8] ASoC: intel: boards: bytcht_es8316: convert to snd_soc_dapm_xxx()
# good: [ab947ebd0e06b1bba68702cadec93359371f0c16] ASoC: mediatek: mt8365-afe-pcm: convert to snd_soc_dapm_xxx()
# good: [b6ba1a11ea626dd82672e06ffa7809beb671277a] ASoC: samsung: tm2_wm5110: convert to snd_soc_dapm_xxx()
# good: [8d5a933224b59fa5a4d540f37b77fee5d22b3eb9] ASoC: mediatek: mt8195-mt6359: convert to snd_soc_dapm_xxx()
# good: [638d14a0a0658f7fd8b868ebb93289d21fb64006] ASoC: intel: boards: sof_da7219: convert to snd_soc_dapm_xxx()
# good: [a6aa027dffef1b6d6ebc9ef87210493bcf7abf22] ASoC: tegra: tegra_wm8903: convert to snd_soc_dapm_xxx()
# good: [0019f120bfe862a583aa1a4347c1761e947e0054] ASoC: tegra: tegra210_ahub: convert to snd_soc_dapm_xxx()
# good: [8c62a765be9ece320dd4da00dcd038f6b83ea2f2] ASoC: intel: boards: cht_bsw_max98090_ti: convert to snd_soc_dapm_xxx()
# good: [e758b08d7170ee96e3461cec5c5b33e9aff5a75e] ASoC: soc-core: convert to snd_soc_dapm_xxx()
# good: [6c65a8c3ad82616e6654c82ec0da33233e6f4e02] ASoC: meson: aiu-codec-ctrl: convert to snd_soc_dapm_xxx()
# good: [088fa2cc7c4c4ec80a293692456f48c43b4da0e7] ASoC: soc-topology: convert to snd_soc_dapm_xxx()
# good: [f0762675a23396e80a30fdb6d71a89dcfb972154] ASoC: sunxi: sun8i-codec: convert to snd_soc_dapm_xxx()
# good: [2990b658ed446abd3b509854e2b266eaef666ac5] ASoC: intel: boards: bytcr_wm5102: convert to snd_soc_dapm_xxx()
# good: [9c95183d4e0cd557bcae21608b03ae9cf0eb87cb] ASoC: intel: boards: sof_nuvoton_common: convert to snd_soc_dapm_xxx()
# good: [beb865ab8e5a2d92ae11daca23bdfa51aea587cb] ASoC: intel: boards: sof_es8336: convert to snd_soc_dapm_xxx()
# good: [e6995aa8165571c0f85ca259de15edde713bfa7c] ASoC: intel: boards: bytcr_rt5640: convert to snd_soc_dapm_xxx()
# good: [fd73d62f5f8b61bd396e4e4185ba2f8ad3a712db] ASoC: sunxi: sun50i-codec-analog: convert to snd_soc_dapm_xxx()
# good: [9abf16750a7bb052799229b5cff5b14459d3bb21] ASoC: sdw_utils: soc_sdw_rt5682: convert to snd_soc_dapm_xxx()
# good: [9defa941d828ad48bb24249363b0f956b3dd3770] ASoC: intel: boards: sof_nau8825: convert to snd_soc_dapm_xxx()
# good: [620fd65ec496b4fe0f49f3b6bc1fd1f890432fd6] ASoC: samsung: smdk_wm8994: convert to snd_soc_dapm_xxx()
# good: [5fca4f6a958c51f2c82d0838735b1cc6ba4da3dd] ASoC: sdw_utils: soc_sdw_rt711: convert to snd_soc_dapm_xxx()
# good: [047191ae16d336b5343b3eec1ad3532dedacee2d] ASoC: meson: aiu-acodec-ctrl: convert to snd_soc_dapm_xxx()
# good: [805759446d9e8f278881f03d5b4b6bcacf9f0ce3] ASoC: sdw_utils: soc_sdw_rt_amp: convert to snd_soc_dapm_xxx()
# good: [9188f03aa5157abbe8eba3130df4381245fe4cd0] ASoC: ti: davinci-evm: convert to snd_soc_dapm_xxx()
# good: [c9d3bcafc711794efe493b7ad26ff704d560b725] ASoC: samsung: littlemill: convert to snd_soc_dapm_xxx()
# good: [21838f53f47e2fa4dfdda71a9bc895f599e7cf9f] ASoC: fsl: imx-rpmsg: convert to snd_soc_dapm_xxx()
# good: [738a0014640cc1a2ee75d66e13327db87fca8a58] ASoC: intel: avs: pcm: convert to snd_soc_dapm_xxx()
# good: [f1ead097d7ddb7014215518f290e49e7fc374266] ASoC: intel: boards: sof_maxim_common: convert to snd_soc_dapm_xxx()
# good: [f39ee5784bb848d320e7e387ef2c11ef21c97e83] ASoC: sdw_utils: soc_sdw_maxim: convert to snd_soc_dapm_xxx()
# good: [5a3c54a94c9101ce126cae1fe21df0cab5d80c63] ASoC: intel: boards: sof_rt5682: convert to snd_soc_dapm_xxx()
# good: [d4dfae0a980c30ecb4c9c3bd90d36171735dcc79] ASoC: sunxi: sun8i-codec-analog: convert to snd_soc_dapm_xxx()
# good: [af535c76426ebbd3df12a804fa6da5b9e8c0339e] ASoC: mediatek: mt8188-mt6359: convert to snd_soc_dapm_xxx()
# good: [5a2b4ddc2cf7f150328f4d1ae617b62a3a3df45c] ASoC: sdca: sdca_asoc: convert to snd_soc_dapm_xxx()
# good: [1014ec36b43a997a8d8fcf68728d1fd6c891b004] ASoC: generic: audio-graph-card: convert to snd_soc_dapm_xxx()
# good: [311c12f7031d845c27e383f2987e53a35b684e5c] ASoC: samsung: midas_wm1811: convert to snd_soc_dapm_xxx()
# good: [5969275804b8f033618ff8f7309b016deb5e5117] ASoC: intel: boards: sof_cirrus_common: convert to snd_soc_dapm_xxx()
# good: [03258f7765e8c0210ea35b865f8e5c03fd9ccab9] ASoC: tegra: tegra_asoc_machine: convert to snd_soc_dapm_xxx()
# good: [eb802ba07bcc1fbcf637b9ccf1ffcdc74277cca4] ASoC: soc-pcm: convert to snd_soc_dapm_xxx()
# good: [97a57e6ea996a773b278106755368755a9c2dc18] ASoC: sdw_utils: soc_sdw_cs42l43: convert to snd_soc_dapm_xxx()
# good: [4fb45e5d4abda09bfd805b08fa3446686aa3c6bb] ASoC: ti: omap-abe-twl6040: convert to snd_soc_dapm_xxx()
# good: [e3c9bc1f31214f33e24fae70ade0d748ed5cba95] ASoC: intel: boards: bdw-rt5677: convert to snd_soc_dapm_xxx()
# good: [0c02abd9f443565b00f79f5d84166d610200b1a5] ASoC: meson: axg-tdm-interface: convert to snd_soc_dapm_xxx()
# good: [22219a4af85457a7ab8fdfa925a1fa2bffe66dd2] ASoC: pxa: spitz: convert to snd_soc_dapm_xxx()
# good: [eac123525d4214d4ef4d751cb48649130b4cf51f] ASoC: samsung: lowland: convert to snd_soc_dapm_xxx()
# good: [8124fe3d5b254a0c17da853591616c8d8ee602e7] ASoC: intel: boards: sof_realtek_common: convert to snd_soc_dapm_xxx()
# good: [61907c9a0d25200ea4681a0ae6b2f7c7673ea5bb] ASoC: intel: boards: cht_bsw_rt5645: convert to snd_soc_dapm_xxx()
# good: [a30381e9cc2d32d8e0d4077cfa6aab0c70d34105] ASoC: intel: boards: sof_board_helpers: convert to snd_soc_dapm_xxx()
# good: [0813e864070dd6fca79619bb4f99e38c97422383] ASoC: soc-jack: convert to snd_soc_dapm_xxx()
# good: [574a59412d32e8e094e636feeaa4d29b55d35410] ASoC: sdw_utils: soc_sdw_cs42l42: convert to snd_soc_dapm_xxx()
# good: [cb5aaf1bab5f7ef98f04eea303eb39ed05325dfe] ASoC: sdw_utils: soc_sdw_bridge_cs35l56: convert to snd_soc_dapm_xxx()
# good: [194c43841d9e5695837789f288889ca02c023074] ASoC: intel: atom: sst-atom-controls: convert to snd_soc_dapm_xxx()
# good: [dd929092b3122a9cfd1f7115c4eecdb7944c45a8] ASoC: intel: avs: rt5514: convert to snd_soc_dapm_xxx()
# good: [a4230eea80ddb9d08e4098f980ddf1d22bbc18ef] ASoC: qcom: topology: convert to snd_soc_dapm_xxx()
# good: [2988abc11bb7c4cf756b4197de0879ed92bb39bf] ASoC: intel: avs: da7219: convert to snd_soc_dapm_xxx()
# good: [fdc5ce7f7272e59a1c32919b1ba8ab73d1bc28ab] ASoC: sdw_utils: soc_sdw_ti_amp: convert to snd_soc_dapm_xxx()
# good: [1b30118264fc75f164ebf8a9888a16ddd2ebb3d4] ASoC: codecs: rt700: convert to snd_soc_dapm_xxx()
# good: [ca7f864f91bff158a697349dae2c81afae2b3073] ASoC: codecs: sta529: convert to snd_soc_dapm_xxx()
# good: [0da5d1fb00c1108e543fa94c92419006a1008d84] ASoC: codecs: tlv320aic32x4: convert to snd_soc_dapm_xxx()
# good: [fef5517fff15652903f56f153938eaf93c2c53a0] ASoC: codecs: wm9081: convert to snd_soc_dapm_xxx()
# good: [9e1666651224267179d1e4f1e366cd526c384ae8] ASoC: codecs: wm8711: convert to snd_soc_dapm_xxx()
# good: [7d6ca6db9b6ee95adfdfabe84819b3b1900dfc05] ASoC: codecs: rt274: convert to snd_soc_dapm_xxx()
# good: [1a8870e2e2eeb7819ad540f0027faec500833351] ASoC: codecs: wm8997: convert to snd_soc_dapm_xxx()
# good: [9bcc76f58217c1701deeb4b4092e54dc64ce38bc] ASoC: codecs: rt5670: convert to snd_soc_dapm_xxx()
# good: [11c256c49d75c3d795339c93f52a19a202db5a15] ASoC: codecs: rt298: convert to snd_soc_dapm_xxx()
# good: [7881b033afa1a2f6ec587dabfe2e12d0bfdef022] ASoC: codecs: wm8988: convert to snd_soc_dapm_xxx()
# good: [636819c3962b9a2377bfdd3baa11e187cba39916] ASoC: codecs: sma1303: convert to snd_soc_dapm_xxx()
# good: [aa57703fd738d0d9f1328d44bc6b9ee82daf0326] ASoC: codecs: wm8960: convert to snd_soc_dapm_xxx()
# good: [c13e3d24dbe84b398bc9e5ea767e4d3515b06771] ASoC: codecs: wm0010: convert to snd_soc_dapm_xxx()
# good: [128cd2206689ebddb6f7a9682f5671190a570a56] ASoC: codecs: simple-mux: convert to snd_soc_dapm_xxx()
# good: [46f360c51cfeb7b29340815885e4a7e82b411bc8] ASoC: codecs: wcd9335: convert to snd_soc_dapm_xxx()
# good: [da9ad75afba487013bd4345d4b77477f35ed25ec] ASoC: codecs: wm8731: convert to snd_soc_dapm_xxx()
# good: [bdc210a4485b640b65f3cf9cfd6fbe1e564e7645] ASoC: codecs: rt715: convert to snd_soc_dapm_xxx()
# good: [58fc7681c470d39332f01aae88b2259e613e53e3] ASoC: codecs: wm8962: convert to snd_soc_dapm_xxx()
# good: [a5a6abd02d100913327a150e764dbc29045970a1] ASoC: codecs: wm8940: convert to snd_soc_dapm_xxx()
# good: [2049f715599eb677722a2e91d3616c4feff0e1fc] ASoC: codecs: rt5659: convert to snd_soc_dapm_xxx()
# good: [8b1c56ae39d3bf304ab0c8de456e8f02081be9ed] ASoC: codecs: rt1015: convert to snd_soc_dapm_xxx()
# good: [8df50a4f2f9faaf336d3640174a828e9b979544b] ASoC: codecs: wm8523: convert to snd_soc_dapm_xxx()
# good: [24bca18098187d56257a938dfcae2673f8fa847a] ASoC: codecs: rt712-sdca: convert to snd_soc_dapm_xxx()
# good: [3affdae134871f7be7d28fe5a763f2a17d1c18ab] ASoC: codecs: tlv320dac33: convert to snd_soc_dapm_xxx()
# good: [8efb94a90e6558a2aeffac518e7b7479fdaa14d9] ASoC: codecs: wm8971: convert to snd_soc_dapm_xxx()
# good: [92a3e4bb1813cd5eaba1050b9d82356b17daca2e] ASoC: codecs: wm8770: convert to snd_soc_dapm_xxx()
# good: [a9b239d098dbbfa8dd0cc56001cfc80053da1317] ASoC: codecs: tas6424: convert to snd_soc_dapm_xxx()
# good: [2416360daf6e17c10af12f40139bfa84c99f7f2f] ASoC: codecs: wcd937x: convert to snd_soc_dapm_xxx()
# good: [12cdef76abbbd5209829e73cc58d44a0921deeda] ASoC: codecs: ssm2518: convert to snd_soc_dapm_xxx()
# good: [5b35bb517f27fc2401ec3cfd8c02a127627a0188] ASoC: codecs: rt1011: convert to snd_soc_dapm_xxx()
# good: [88539900888437c446f44b8248797b958e10d14f] ASoC: codecs: tas571x: convert to snd_soc_dapm_xxx()
# good: [10d5d21a3379b56a0655609918fcc4b4ccb4909d] ASoC: codecs: wm8994: convert to snd_soc_dapm_xxx()
# good: [8a19eabf16304642a1c281f6676b7248b234ca1d] ASoC: codecs: wm8998: convert to snd_soc_dapm_xxx()
# good: [0c8392fd64e8be4fab28d8f568582f4cd7092196] ASoC: codecs: rt5665: convert to snd_soc_dapm_xxx()
# good: [5d9958b2eff8ef283bb70fd0ae82cc6e65372876] ASoC: codecs: sma1307: convert to snd_soc_dapm_xxx()
# good: [21bb118ea37adc40027771b1f45a246eb3956265] ASoC: codecs: wm8978: convert to snd_soc_dapm_xxx()
# good: [453dfd1c9b486f2289d60871ee11fa05809303ab] ASoC: codecs: rt5660: convert to snd_soc_dapm_xxx()
# good: [1ddafb83a431b2d4c847b2640b352cfca0d1b077] ASoC: codecs: tlv320aic31xx: convert to snd_soc_dapm_xxx()
# good: [a3de4a9c3e0627989efe88ea31f422f1da896dcc] ASoC: codecs: wm5110: convert to snd_soc_dapm_xxx()
# good: [697f92e637d1c0de9ec9d1dc31a6fa3c921be831] ASoC: codecs: uda1380: convert to snd_soc_dapm_xxx()
# good: [6254032a219c7581df8203b31480f06a2babd46d] ASoC: codecs: rt715-sdca: convert to snd_soc_dapm_xxx()
# good: [cffb8638191684198b4e696393c678498791c590] ASoC: codecs: rt5516: convert to snd_soc_dapm_xxx()
# good: [4f9d291e6828aa92503988e319b175c0fa3303af] ASoC: codecs: wm8900: convert to snd_soc_dapm_xxx()
# good: [5b08edab386559a7d8549fa94688ea3878e9574f] ASoC: codecs: wm8903: convert to snd_soc_dapm_xxx()
# good: [46db12032942a47d6c2b1158633514a739f780eb] ASoC: codecs: rt721-sdca: convert to snd_soc_dapm_xxx()
# good: [588dc19b1246fc05bd62b3471a01b19cafdc3809] ASoC: codecs: nau8810: convert to snd_soc_dapm_xxx()
# good: [c153d6da9cde3efc5fbfcf39515d224c2b6ad48f] ASoC: codecs: wm8750: convert to snd_soc_dapm_xxx()
# good: [cf3937987c18980c3dffc456c91fc1f7b1cec4ca] ASoC: codecs: rt5682: convert to snd_soc_dapm_xxx()
# good: [dde23aa355aa0fc1fbafd40dc53ffb2d67a7e8b9] ASoC: codecs: wm8580: convert to snd_soc_dapm_xxx()
# good: [3eca632e86ac79012fbbeb3b7753d345f0baad04] ASoC: codecs: nau8824: convert to snd_soc_dapm_xxx()
# good: [3f553ac8005e07e7c3d84edfdc08727439e4f5c7] ASoC: codecs: wm8728: convert to snd_soc_dapm_xxx()
# good: [60b6a547a88313022ca10af6a3029891bc638b1d] ASoC: codecs: rt5668: convert to snd_soc_dapm_xxx()
# good: [5d1a88f50569d1c4637fe29620e6d194f7e50014] ASoC: codecs: wm8955: convert to snd_soc_dapm_xxx()
# good: [2d8a091725e6f71c87a7ae35e51550693afdef77] ASoC: codecs: rk3308: convert to snd_soc_dapm_xxx()
# good: [7b10c284495c7f9a666f7057351e67d6bc9c54bd] ASoC: codecs: wcd934x: convert to snd_soc_dapm_xxx()
# good: [9050b9a872a9bb78874ff6b7ba933ca0a90dcecd] ASoC: codecs: wm8996: convert to snd_soc_dapm_xxx()
# good: [f72c526d2e7a512ab969c3492a365633f5ff2f33] ASoC: codecs: mt6359: convert to snd_soc_dapm_xxx()
# good: [52239c555cbd7369b4009dbf72791f0bb56469b6] ASoC: codecs: rt712-sdca-dmic: convert to snd_soc_dapm_xxx()
# good: [664ae39314732223786b1c94ac6031b01886fc84] ASoC: codecs: ssm4567: convert to snd_soc_dapm_xxx()
# good: [0e8d0789ded09e681cb3ba48beaceca125d441f5] ASoC: codecs: wm8995: convert to snd_soc_dapm_xxx()
# good: [04b80a9aa21f2d02ac6ccbfacc33eb0827eb9899] ASoC: codecs: wm8737: convert to snd_soc_dapm_xxx()
# good: [c3282edee7bd18cde6cfdf31a918dbd112dd794b] ASoC: codecs: rt5640: convert to snd_soc_dapm_xxx()
# good: [00246ae603ef2ee4765fd71923519849a1690286] ASoC: codecs: twl4030: convert to snd_soc_dapm_xxx()
# good: [0ba0819b8fb4ea27d700e5784e150217e2d9f295] ASoC: codecs: wm5100: convert to snd_soc_dapm_xxx()
# good: [82d779d150ed27232c940a7908c0f42c248a981f] ASoC: codecs: rt5663: convert to snd_soc_dapm_xxx()
# good: [6f917e47db4984e784a5245b4e3f35ca16b01ccf] ASoC: codecs: wm8991: convert to snd_soc_dapm_xxx()
# good: [098c486f07000c70fe18abd879a0abdb96ccfc23] ASoC: codecs: wm8510: convert to snd_soc_dapm_xxx()
# good: [02dbbb7e982a6873f81e69e4fef59a42decb7b1a] ASoC: codecs: pcm512x: convert to snd_soc_dapm_xxx()
# good: [8d27adc88a5461b08990d33a16d215e3e7e890dc] ASoC: codecs: wm8961: convert to snd_soc_dapm_xxx()
# good: [ca6cd3d3f1d7f98908dbc8342a28663346067b9a] ASoC: codecs: rt5514: convert to snd_soc_dapm_xxx()
# good: [4e111f49809959bd568cf68bea83155908872164] ASoC: codecs: wm8990: convert to snd_soc_dapm_xxx()
# good: [ab7963a3adc1dec9dd6efb9f4755fedb9b56d35d] ASoC: codecs: rt5631: convert to snd_soc_dapm_xxx()
# good: [e727c8d01a5d6049e756fc15dfec08fbf4f7b0cf] ASoC: codecs: rt5682s: convert to snd_soc_dapm_xxx()
# good: [acee6bda8872f715dcc6d3ff1fe387606d752661] ASoC: codecs: wm8776: convert to snd_soc_dapm_xxx()
# good: [d1b06d4a37c301a09403746f1d12bdcaeed40830] ASoC: codecs: pcm186x: convert to snd_soc_dapm_xxx()
# good: [1b6934ca2a68bca2d6a776497f47ba319419d7dd] ASoC: codecs: wm8804: convert to snd_soc_dapm_xxx()
# good: [cf61841233bd0c905b3493f879b45fa9f7d44ff4] ASoC: codecs: tlv320aic3x: convert to snd_soc_dapm_xxx()
# good: [863a396f09f559344e41ccaf2cab6aea12bcc3bc] ASoC: codecs: wm8400: convert to snd_soc_dapm_xxx()
# good: [23805d059a597b001bd5e3655ee0bae652142944] ASoC: codecs: wm8753: convert to snd_soc_dapm_xxx()
# good: [8000e233e920062dc08222fc6911e13c193d7719] ASoC: codecs: nau8822: convert to snd_soc_dapm_xxx()
# good: [d6abe1e31fef77aab57f252c842acb92517ba9d3] ASoC: codecs: rt5651: convert to snd_soc_dapm_xxx()
# good: [f8b494b28806efc8e6dece1f66f30c371376788c] ASoC: codecs: sta350: convert to snd_soc_dapm_xxx()
# good: [6ad9058c495691f004e8e697e1acbb2b4775c7e2] ASoC: codecs: rt5677: convert to snd_soc_dapm_xxx()
# good: [c5b5f6d75ecc87eb543ca8e867ca1585c03b242a] ASoC: codecs: twl6040: convert to snd_soc_dapm_xxx()
# good: [b637319d1fdc3686b669ab9c0ec6faee5e8954c2] ASoC: codecs: ssm2602: convert to snd_soc_dapm_xxx()
# good: [3f2848403a817345c2d0984c49f128fc9dc28b3b] ASoC: codecs: wm8904: convert to snd_soc_dapm_xxx()
# good: [eba22be8d911dea8132ae6bf8e83e26740e068b2] ASoC: codecs: wm5102: convert to snd_soc_dapm_xxx()
# good: [d4ed7da08c767437400b2550092b7ea67cfac294] ASoC: codecs: wm8985: convert to snd_soc_dapm_xxx()
# good: [69532b4263a264c5bf9ad11046ad1acb2ab3534e] ASoC: codecs: wm8993: convert to snd_soc_dapm_xxx()
# good: [42f4ff0c0b21ac72ae3af23c67ce51209809e2a4] ASoC: codecs: nau8825: convert to snd_soc_dapm_xxx()
# good: [cd0cd7e57d2b300d3d63cb9befa9edcf3e05ad68] ASoC: codecs: rt5645: convert to snd_soc_dapm_xxx()
# good: [49a22e9811e1524cf4ee6c84e795526104f4fc3c] ASoC: codecs: rt711-sdca: convert to snd_soc_dapm_xxx()
# good: [c2b942fbf952d734646352ff56f04bea6c151d3a] ASoC: codecs: rt286: convert to snd_soc_dapm_xxx()
# good: [0314d78729a936de02f97daabd8756eb361922d4] ASoC: codecs: nau8821: convert to snd_soc_dapm_xxx()
# good: [62d12a8e945605468a8d119960c988014c44f6c0] ASoC: codecs: wm8350: convert to snd_soc_dapm_xxx()
# good: [6db8ba3439c995a08def399f9eda4e1d2b95afa8] ASoC: codecs: rt711: convert to snd_soc_dapm_xxx()
# good: [8268a73d6c93b9cadf323bb9ef92fd01d4eb6529] ASoC: codecs: wm8974: convert to snd_soc_dapm_xxx()
# good: [8454435315e5a6e2f8e531b5ec038c6db4d6ebfe] ASoC: codecs: sta32x: convert to snd_soc_dapm_xxx()
# good: [19bf3fb9baf235cca9f9bb52cee488662e5bba0c] ASoC: codecs: wm8983: convert to snd_soc_dapm_xxx()
# good: [eeb152d9636f3a1355ce1d2bdc481cb70e0ca2bb] ASoC: codecs: tlv320adc3xxx: convert to snd_soc_dapm_xxx()
# good: [78ad27bc554d33963fa829632c789ab8fb191a09] ASoC: codecs: cx20442: convert to snd_soc_dapm_xxx()
# good: [87bb65464be16f83e5a7a6642f33084b6ce22d2d] ASoC: codecs: adav80x: convert to snd_soc_dapm_xxx()
# good: [e765581ceba401ef62329a4699403d00cb17ce2b] ASoC: codecs: ad1836: convert to snd_soc_dapm_xxx()
# good: [2bed4e2a81bcc1e59eb747d0a51e46a24bf3a1b3] ASoC: codecs: cs42l73: convert to snd_soc_dapm_xxx()
# good: [f7e9d040d2b3243f9682c21ac7dda77a54b03c62] ASoC: codecs: jz4760: convert to snd_soc_dapm_xxx()
# good: [c475f3468361bb37815aa5beaac056568d18fb56] ASoC: codecs: es8311: convert to snd_soc_dapm_xxx()
# good: [5328d3d6b68d1819c199555aff88edad3ccd8bf3] ASoC: codecs: lpass-va-macro: convert to snd_soc_dapm_xxx()
# good: [daaf102aee4069d2fcaa4da60949b6e02f845c37] ASoC: codecs: jz4770: convert to snd_soc_dapm_xxx()
# good: [f3e9bca581321403c9614a075002e29d8e29c60d] ASoC: atmel: sam9g20_wm8731: convert to snd_soc_dapm_xxx()
# good: [bbbd507ab3c76f0ed262a29af89f36177a43d65b] ASoC: codecs: max98396: convert to snd_soc_dapm_xxx()
# good: [64f90b329ffadae00b187ac797837ba598289707] ASoC: audio_codec: convert to snd_soc_dapm_xxx()
# good: [bdd43845598cae0591f0203d6e70f510c4af852d] ASoC: codecs: adau1977: convert to snd_soc_dapm_xxx()
# good: [7a7ecd8d0a428e96f9271ea59b067b2a5c318de9] ASoC: codecs: ak4641: convert to snd_soc_dapm_xxx()
# good: [b42c7f40d2aa8be24601eaa80c672e2c34742c18] ASoC: amd: acp5x-mach: convert to snd_soc_dapm_xxx()
# good: [696713199ea5128a3d031a45f275bbfd275b87db] ASoC: codecs: ad193x: convert to snd_soc_dapm_xxx()
# good: [0ad0505f61402ddcd08d3a6c8f741f0e1487bdc0] ASoC: codecs: audio-iio-aux: convert to snd_soc_dapm_xxx()
# good: [cdc8feb84f7f22a9b21483b706382cea3cd248cd] ASoC: codecs: ab8500: convert to snd_soc_dapm_xxx()
# good: [464ac2a1085749dbfeee899ac06d51bea7ef1041] ASoC: codecs: da9055: convert to snd_soc_dapm_xxx()
# good: [8855eb7d29400fb7b2882da33725db2801c410e4] ASoC: asoc.h: convert to snd_soc_dapm_xxx()
# good: [2dcb4f1d27e9c7a610ca40ac37d2dd177db35a52] ASoC: codecs: max98373: convert to snd_soc_dapm_xxx()
# good: [d6b8ebab363bf3dd46cf4d64727910d648bd35cf] ASoC: arizona-haptics: convert to snd_soc_dapm_xxx()
# good: [efdc8c78a5191c85096ee772a36330ca280873ce] ASoC: codecs: cs4234: convert to snd_soc_dapm_xxx()
# good: [4d5c668c268b7812ff15452d303974ce247ad378] ASoC: soc.h: convert to snd_soc_dapm_xxx()
# good: [31f28cda703d442b75c4ae9f3d0855d787c6e909] ASoC: codecs: adau7118: convert to snd_soc_dapm_xxx()
# good: [638d7077e67f03926f94201f5508c886d0683adc] ASoC: codecs: cs47l90: convert to snd_soc_dapm_xxx()
# good: [46fb094b54c89ddf314f7e807e5457be9b185bbc] ASoC: codecs: max9867: convert to snd_soc_dapm_xxx()
# good: [476fb171ed43ce544d54cd082ec56df2e8f8aa5d] ASoC: codecs: da732x: convert to snd_soc_dapm_xxx()
# good: [aaf1f90c99d920f8ab95413cab7abc627316fdf6] ASoC: codecs: da7219-aad: convert to snd_soc_dapm_xxx()
# good: [eec3b674f2a5c2334d2f1a2200b231975277e2c9] ASoC: codecs: es8326: convert to snd_soc_dapm_xxx()
# good: [3df33a9e0faae857dc4c594a915404afa6a7a8f8] ASoC: codecs: lpass-tx-macro: convert to snd_soc_dapm_xxx()
# good: [f7a7e796a36f487758a9444408cef41f94460158] ASoC: codecs: adau1781: convert to snd_soc_dapm_xxx()
# good: [098931e46c0ed4e0203bcdc626ff94bcd58dbf36] ASoC: codecs: cs42l43-jack: convert to snd_soc_dapm_xxx()
# good: [1505741d727e210650c7f2cbe16ab450f0b1b0bf] ASoC: codecs: es8316: convert to snd_soc_dapm_xxx()
# good: [770f17143988f1804d24cb5ab5a4bb9087a715b2] ASoC: codecs: cs42l56: convert to snd_soc_dapm_xxx()
# good: [2e20e32c2e3ac17aae4f53ce6205a5d8bf34e895] ASoC: codecs: es8389: convert to snd_soc_dapm_xxx()
# good: [cc1bb5d435d601035be92d98295cea0b973ae5c2] ASoC: codecs: cpcap: convert to snd_soc_dapm_xxx()
# good: [729a995620ca34f589135f9852d2ef1c9ee7ebb2] ASoC: codecs: cs47l24: convert to snd_soc_dapm_xxx()
# good: [af9a1da6c3ae130fc44f218293410859f39bbd80] ASoC: codecs: lpass-wsa-macro: convert to snd_soc_dapm_xxx()
# good: [16d4e8616e78e5edbcb99268050b431afcfe4610] ASoC: codecs: lpass-rx-macro: convert to snd_soc_dapm_xxx()
# good: [45da4a452aeb4f5f44bcbf50079291b70bec9dc5] ASoC: codecs: lm49453: convert to snd_soc_dapm_xxx()
# good: [f97289522b7910921d771af00b7c0ec915f4e21f] ASoC: codecs: max98390: convert to snd_soc_dapm_xxx()
# good: [5dc2dcbb10614ca77b32340d88d24cef47cd8d58] ASoC: codecs: cs42l43: convert to snd_soc_dapm_xxx()
# good: [e195aad3cd473030ff696fdecb76a04d11f1d3b4] ASoC: codecs: ml26124: convert to snd_soc_dapm_xxx()
# good: [f2d57e22b51c0aa1b2d222b30c07cdf80e4a6ca5] ASoC: codecs: hdac_hdmi: convert to snd_soc_dapm_xxx()
# good: [9c3c02de4883862fdccbc38943ade051fc0de0e2] ASoC: codecs: max98088: convert to snd_soc_dapm_xxx()
# good: [75dc6bf1d4dd08d78dd9568eeb041a3e0787084d] ASoC: codecs: cs42xx8: convert to snd_soc_dapm_xxx()
# good: [96b0a24773849ca2d051a552a19eee262f1f7444] ASoC: codecs: da7219: convert to snd_soc_dapm_xxx()
# good: [6ef8e2f7e5eb6933e0897575e7869741ac34e302] ASoC: codecs: max98090: convert to snd_soc_dapm_xxx()
# good: [09ff5df2edbea67256209eb814bca4570436b7c6] ASoC: codecs: adau17x1: convert to snd_soc_dapm_xxx()
# good: [a7ab96cf1366f2e35d05b1e773db2a4c1b5e4a45] ASoC: codecs: max9850: convert to snd_soc_dapm_xxx()
# good: [bccf1a2e2a7819f2f9caebd5a135fe119f09dbd1] ASoC: codecs: cs35l56: convert to snd_soc_dapm_xxx()
# good: [f0962b1a35043b8d0e3384b7004f3785b7c7ffd0] ASoC: codecs: cs35l41: convert to snd_soc_dapm_xxx()
# good: [4b9ac2be5fc62ea65c032c3e6e554e60757e5944] ASoC: codecs: hda: convert to snd_soc_dapm_xxx()
# good: [6b1b50ed3c9409a85ff28335ca9b471ed399e652] ASoC: amd: acp-mach-common: convert to snd_soc_dapm_xxx()
# good: [57bea9f5f02e76cb81d84de17576faf336526167] ASoC: codecs: aw88395: convert to snd_soc_dapm_xxx()
# good: [797e4164fe55a03db19930ae94d021173c8e726b] ASoC: codecs: alc5623: convert to snd_soc_dapm_xxx()
# good: [c9ba54d36ac20fd94c9479e8b82b32610a4cb979] ASoC: codecs: cs42l51: convert to snd_soc_dapm_xxx()
# good: [6d2188983a12e036531ecc67d89c6e1388855ea1] ASoC: atmel: tse850-pcm5142: convert to snd_soc_dapm_xxx()
# good: [4db61b8cd50441c846ac8e6afb53781f62f92029] ASoC: codecs: cs47l15: convert to snd_soc_dapm_xxx()
# good: [4ddf34b148dfb08d47b3e920fdf831eabd973130] ASoC: codecs: mt6357: convert to snd_soc_dapm_xxx()
# good: [91f20e5167d38eb943a087e3087927d1ce7a5c78] ASoC: arizona-micsupp: convert to snd_soc_dapm_xxx()
# good: [80ac220583627912ef478a25e22b02bd813d5268] ASoC: codecs: cs47l35: convert to snd_soc_dapm_xxx()
# good: [f4435734559b8b4bd2dd6c457705b8a22baca0a9] ASoC: codecs: jz4740: convert to snd_soc_dapm_xxx()
# good: [946d58ae54f8cd9ead30571a71c75a3981d7862a] ASoC: codecs: 88pm860x: convert to snd_soc_dapm_xxx()
# good: [dd9896d41fdf1050934d6a46a1c5ca2164284e72] ASoC: Intel: avs: Allow the topology to carry NHLT data
# good: [82e5de78dbdeec04257d8bdba8465217ad9447d9] ASoC: codecs: es7134: convert to snd_soc_dapm_xxx()
# good: [436a4d82d2460926aa2a772acf5399accedfbc00] ASoC: codecs: cs35l33: convert to snd_soc_dapm_xxx()
# good: [12223b4534dd70bc13d792a836f53369db89caa1] ASoC: codecs: da7218: convert to snd_soc_dapm_xxx()
# good: [4a9eb06f9b36ca43106e6e12231b5171a660e4ce] ASoC: codecs: adau1761: convert to snd_soc_dapm_xxx()
# good: [7bb1edac96067d523a30b4ef5808d21c86725f64] ASoC: codecs: aw88261: convert to snd_soc_dapm_xxx()
# good: [7fdcd1d1add9c3abc1379563c98bcfcc5ce26343] ASoC: codecs: cs530x: convert to snd_soc_dapm_xxx()
# good: [4d01421647562d3f16319d79197d2afa8f3f1c84] ASoC: codecs: cs42l52: convert to snd_soc_dapm_xxx()
# good: [ea9d164c67245f5d94da8d46eac1de25fd1059d6] ASoC: codecs: mt6358: convert to snd_soc_dapm_xxx()
# good: [4c0f28830a4f96487cccfb0e8b78efd628766d91] ASoC: codecs: cs47l92: convert to snd_soc_dapm_xxx()
# good: [d5c8b7902a41625ea328b52c78ebe750fbf6fef7] ASoC: Intel: avs: Honor NHLT override when setting up a path
# good: [97062ef891393c82107cf44a0eb50c6e1a399e18] ASoC: codecs: cs48l32: convert to snd_soc_dapm_xxx()
# good: [0682c592ec98ba18ac65d1cd4b9c345e2995c121] ASoC: codecs: madera: convert to snd_soc_dapm_xxx()
# good: [a5511a6e10f1e1462d61aec8420d6ac4cd9e3efe] ASoC: codecs: es8328: convert to snd_soc_dapm_xxx()
# good: [ce99b1dd71340e78d5f8f94af1fdf929bc386b61] ASoC: codecs: cs47l85: convert to snd_soc_dapm_xxx()
# good: [ee0f171f4b9d21af0b202cec35c44c1eaf6e0beb] ASoC: audio_topology: convert to snd_soc_dapm_xxx()
# good: [170bc7a0e26ca3d36f543db4111cb7195137c145] ASoC: amd: acp3x-es83xx: convert to snd_soc_dapm_xxx()
# good: [118eb2cb97b8fc0d515bb0449495959247db58f0] spi: bcm63xx: drop wrong casts in probe()
# good: [335f0d947f52113110e2ed2ad2c030050d220ad8] ASoC: codecs: arizona-jack: convert to snd_soc_dapm_xxx()
# good: [a3151663d0274f79c52bb7ae11e69c0dad1f2267] ASoC: codecs: hdac_hda: convert to snd_soc_dapm_xxx()
# good: [fa2defb800d7c7b67fa7d9fa95824b7780a90575] ASoC: codecs: cs53l30: convert to snd_soc_dapm_xxx()
# good: [2e20be4b48c1d2886ff11147b302caab76f3c4b7] ASoC: codecs: max98095: convert to snd_soc_dapm_xxx()
# good: [ed71deee195d5d2478363d606b0fbe5665e99d7f] ASoC: audio_helper: convert to snd_soc_dapm_xxx()
# good: [4bac87b4742d479384ee133c0b591db9234acf0e] ASoC: codecs: cs35l45: convert to snd_soc_dapm_xxx()
# good: [e6f48607ac801864b99ff93e5193f3f422d55f7b] ASoC: codecs: cs2072x: convert to snd_soc_dapm_xxx()
# good: [b4403cf77e39c2d0e2694b1cf5311683cbfac789] ASoC: codecs: hdmi-codec: convert to snd_soc_dapm_xxx()
# good: [0a87517ae3e7af2019f80e32df01252dd8604080] ASoC: codecs: da7213: convert to snd_soc_dapm_xxx()
# good: [c94f134729491ab60fc68fcd919821014334db97] Add support for Microchip CoreSPI Controller
# good: [c2561572031a1a0ac94d1112fd9b768f65d9bdd5] ASoC: codecs: lpass-macro: complete sm6115 support
# good: [05b0e94153e313132b594783fa6a3fe9e0100595] ASoC: codecs: pm4125: Two minor fixes for
# good: [af330925a18ff483adace1c4ed2dcc975a31d5c0] spi-cadence: support transmission with
# good: [8ce9a2ed153bcaa750aa494e91ce2e70c3b0cdc5] spi: dt-binding: document Microchip CoreSPI
# good: [059f545832be85d29ac9ccc416a16f647aa78485] spi: add support for microchip "soft" spi controller
# good: [71c814e98696f2cd53e9e6cef7501c2d667d4c5a] spi: microchip: rename driver file and internal identifiers
# good: [6402ddf3027d8975f135cf2b2014d6bbeb2d3436] MAINTAINERS: refer to trivial-codec.yaml in relevant sections
# good: [e65b871c9b5af9265aefc5b8cd34993586d93aab] ASoC: codecs: pm4125: Remove irq_chip on component unbind
# good: [8d63e85c5b50f1dbfa0ccb214bd91fe5d7e2e860] firmware: cs_dsp: fix kernel-doc warnings in a header file
# good: [55b5d192bab5e152bda8f8cefe837c4ed0ec60c5] dt-bindings: spi: spi-cadence: update DT binding docs to support cix sky1 SoC
# good: [fd94857a934cbe613353810a024c84d54826ead3] ASoC: codecs: pm4125: Fix potential conflict when probing two devices
# good: [7c63b5a8ed972a2c8c03d984f6a43349007cea93] ASoC: codecs: lpass-tx-macro: fix SM6115 support
# good: [675f41b8d1675d9a51a6dcb978ff76b95cbb4b92] ASoC: dt-bindings: qcom,lpass-va-macro: Add sm6115 LPASS VA
# good: [8ff3dcb0e8a8bf6c41f23ed4aa62d066d3948a10] ASoC: codecs: lpass-rx-macro: add SM6115 compatible
# good: [5a0438622b49df1e723960ac9c0bf75e04370fdc] ASoC: dt-bindings: qcom,lpass-va-macro: re-arrange clock-names
# good: [65d03e84d8b8e42cf4636fcabb81c9b130cec710] ASoC: dt-bindings: qcom,lpass-rx-macro: Add sm6115 LPASS RX
# good: [893e2fd509e968cc1d76caadee0f5d2f2c72f137] ASoC: codecs: lpass-va-macro: add SM6115 compatible
# good: [4e00135b2dd1d7924a58bffa551b6ceb3bd836f2] spi: spi-cadence: supports transmission with bits_per_word of 16 and 32
# good: [123cd174a3782307787268adf45f22de4d290128] ASoC: Intel: atom: Replace strcpy() with strscpy()
# good: [1d562ba0aa7df81335bf96c02be77efe8d5bab87] spi: dt-bindings: nuvoton,npcm-pspi: Convert to DT schema
# good: [4d6e2211aeb932e096f673c88475016b1cc0f8ab] ASoC: Intel: boards: fix HDMI playback lookup when HDMI-In capture used
# good: [23523e9398efeee6a5741cf97052babf6fb2a808] ASoC: Intel: machine driver updates for 6.19
# good: [2b0d5d9b39489b9d8171896e8e24d8632c3dc807] ASoC: cs35l56: Support for restoring calibration on
# good: [660d946ce6fd64678b4ed6b083dfc24c0f4d5a69] ASoC: Intel: sof_sdw: Add quirk to exclude RT722 speaker
# good: [32172cf3cb543a04c41a1677c97a38e60cad05b6] ASoC: cs35l56: Allow restoring factory calibration through ALSA control
# good: [f78d96c289da743d517f700d3fe7c6c43df667b5] ASoC: Intel: sof_sdw: create BT dai link if bt_link_mask is set
# good: [69f3474a01e9867dd99fc4b703fa834ea1835c7d] ASoC: cs35l56: Add control to read CAL_SET_STATUS
# good: [38ecd1384079d5f7b6942b68496ee7e85aa9a883] ASoC: Intel: soc-acpi-intel-ptl-match: Add support for rt722_l0_rt1320_l23
# good: [b3a5302484033331af37569f7277d00131694b57] ASoC: Intel: sof_rt5682: Add quirk override support
# good: [1d5bf23aa1e8aea43beca78dc5d69c68955e8871] ASOC: Intel: sof_sdw: add quirk for Avell B.ON (OEM rebranded NUC15)
# good: [873bc94689d832878befbcadc10b6ad5bb4e0027] ASoC: Intel: sof_sdw: add codec speaker support for the SKU
# good: [dfbb57c2767dc128f6360c3e41136bff8fa6f924] ASoC: Intel: soc-acpi-ptl-match: add cs42l43_agg_l3_cs35l56_2 support
# good: [4acbfcf11cbe6c46c42091b49875c7002b0bff3d] ASoC: dt-bindings: consolidate simple audio codec to trivial-codec.yaml
# good: [772ada50282b0c80343c8989147db816961f571d] ASoC: cs35l56: Alter error codes for calibration routine
# good: [e32c402533cc68abe20fcf3246b9ff53e1f96021] ASoC: spacemit: fix incorrect error check for sspa clock
# good: [6985defd1d832f1dd9d1977a6a2cc2cef7632704] regmap: sdw-mbq: Reorder regmap_mbq_context struct for better packing
# good: [fb1ebb10468da414d57153ddebaab29c38ef1a78] regulator: core: disable supply if enabling main regulator fails
# good: [2089f086303b773e181567fd8d5df3038bd85937] regulator: mt6363: Remove unneeded semicolon
# good: [4db4ce15706d6423cc4cac4b05114b0469507bad] ASoC: stm32: dfsdm: don't use %pK through printk
# good: [4e92abd0a11b91af3742197a9ca962c3c00d0948] spi: imx: add i.MX51 ECSPI target mode support
# good: [b6f4bd64f453183954184ffbc2b89d73ed8fb135] regulator: irq_helper: replace use of system_wq with system_dfl_wq
# good: [6951be397ca8b8b167c9f99b5a11c541148c38cb] ASoC: codecs: pm4125: remove duplicate code
# good: [4a58f60df578bb55a1d0adca0c356e03b2818de1] spi: enable the SpacemiT K1 SoC QSPI
# good: [bd352547df647be8a1e6c9d4ca2b54b459f3abc1] spi: dt-bindings: fsl-qspi: support SpacemiT K1
# good: [1797d254f5c4b46b295527a635af7321a3fe1318] spi: fsl-qspi: add a clock disable quirk
# good: [106d7641e55a472e7523c1f525c77fb6d420064d] spi: fsl-qspi: add optional reset support
# good: [873a46141460d209bb62eaa0dc9e7b67bff924a6] spi: dt-bindings: fsl-qspi: add optional resets
# good: [47d98d432f19030faf06d4b2ca7fc7493057cae1] ASoC: Merge up fixes
# good: [6b398c1d3da7a673b13b1857f9fff4c15ee20cef] spi: fsl-qspi: switch predicates to bool
# good: [56931105074fe7e5fc9d54e3163df3b95075643c] spi: fsl-qspi: introduce sfa_size devtype data
# good: [abc9a349b87ac0fd3ba8787ca00971b59c2e1257] spi: fsl-qspi: support the SpacemiT K1 SoC
# good: [1044821176a64a8dc7f861e878506cde34c8b658] Add support MT6316/6363/MT6373 PMICs regulators
# good: [6d34b66fb726a613b98c936adee70d42aa5e4aa7] ASoC: soc-pcm: Preserve hw parameters from components in dpcm_runtime_setup_fe
# good: [55d03b5b5bdd04daf9a35ce49db18d8bb488dffb] spi: imx: remove CLK calculation and check for target mode
# good: [1b0f3f9ee41ee2bdd206667f85ea2aa36dfe6e69] ASoC: SDCA: support Q7.8 volume format
# good: [6bd1ad97eb790570c167d4de4ca59fbc9c33722a] regulator: pf9453: Fix kernel doc for mux_poll()
# good: [7a381e373a4243926a41b8e6ebbdeb90fe9afda3] ASoC: qcom: q6dsp: fixes and updates
# good: [81c53b52de21b8d5a3de55ebd06b6bf188bf7efd] ASoC: qcom: qdsp6: q6asm-dai: set 10 ms period and buffer alignment.
# good: [bcb3f6f5286b96ff0a7106d2ca7208744b20b37e] ASoc: qcom: audioreach: remove unused variables
# good: [3c36965df80801344850388592e95033eceea05b] regulator: Add support for MediaTek MT6363 SPMI PMIC Regulators
# good: [6e00112d31c86029ad0440f8c29ee0d131984cda] ASoc: qcom: q6asm: Use automatic cleanup of kfree()
# good: [bfbb12dfa144d45575bcfe139a71360b3ce80237] ASoC: qcom: q6asm-dai: perform correct state check before closing
# good: [b828059f86f67729aae3934650ae9e44a59ff9d8] ASoC: qcom: q6apm-dai: Use guard() for spin locks
# good: [89cf2223ee7bd83d45c6bef3ac52bedd018f77dd] ASoc: qcom: q6apm: Use automatic cleanup of kfree()
# good: [655079ac8a7721ac215a0596e3f33b740e01144a] ASoC: qcom: q6asm: Use guard() for spin locks
# good: [a354f030dbce17ab3cf299660a38b162d97f7151] ASoC: qcom: q6asm: handle the responses after closing
# good: [4b1edbb028fb95b546893068b321a983b1ef50bc] ASoC: qcom: q6afe: Use guard() for spin locks
# good: [680c683c8f49455bcfa1604eac4e508ba96ccbfa] ASoC: qcom: q6asm-dai: Use guard() for spin locks
# good: [b6a1d7243ec3a3fe1a3c2969fcd32c802fb5d2a9] ASoC: qcom: q6asm: add q6asm_get_hw_pointer
# good: [950a4e5788fc7dc6e8e93614a7d4d0449c39fb8d] ASoC: qcom: q6apm-dai: set flags to reflect correct operation of appl_ptr
# good: [896f1e4e106449ac2b39f0ca62b6afaed540f758] ASoc: qcom: q6adm: Use automatic cleanup of kfree()
# good: [74cc4f3ea4e99262ba0d619c6a4ee33e2cd47f65] ASoC: qcom: q6adm: the the copp device only during last instance
# good: [a87a7b3530728fb7477a74c27fc27e060bf5684f] regulator: Add support for MediaTek MT6316 SPMI PMIC Regulators
# good: [d0f9f5b7a3356b43f78b37b9cc3671ecc7469356] dt-bindings: regulator: Document MediaTek MT6363 PMIC Regulators
# good: [f678ea19e8f60b05b8d3a5bcda9a238d5355bd50] ASoC: qcom: q6asm: set runtime correctly for each stream
# good: [de8e95773c48bad9d7339ccb376ad22d93932468] ASoc: qcom: q6prm: Use automatic cleanup of kfree()
# good: [88a5f8e628ef745bb94bdf2fa019d9f8e069ccea] ASoc: qcom: audioreach: Use automatic cleanup of kfree()
# good: [e386c2cf3df2b62a6cc78d2f7984102b64753bee] ASoC: qcom: q6asm-dai: schedule all available frames to avoid dsp under-runs
# good: [44432315444cbbedadbb9d808c9fc8c8b21cb4bf] ASoC: qcom: q6asm-dai: use q6asm_get_hw_pointer
# good: [55094e55ae36c7566e29ae0473d205dbc9d2f4a8] ASoc: qcom: q6afe: Use automatic cleanup of kfree()
# good: [40a7c5db9020079547358f486ef12d57c1a7aa1f] dt-bindings: regulator: Document MediaTek MT6316 PMIC Regulators
# good: [8b6faa7fddf0ae69c5f1a9315a64edee6f022037] spi: tegra210-quad: Improve timeout handling under
# good: [aa897ffc396b48cc39eee133b6b43175d0df9eb5] ASoC: dt-bindings: ti,pcm1862: convert to dtschema
# good: [2f538ef9f6f7c3d700c68536f21447dfc598f8c8] spi: aspeed: Use devm_iounmap() to unmap devm_ioremap() memory
# good: [380fd29d57abe6679d87ec56babe65ddc5873a37] spi: tegra210-quad: Check hardware status on timeout
# good: [af9c8092d84244ca54ffb590435735f788e7a170] regmap: i3c: Use ARRAY_SIZE()
# good: [b4e002d8a7cee3b1d70efad0e222567f92a73000] spi: tegra210-quad: Fix timeout handling
# good: [c4e68959af66df525d71db619ffe44af9178bb22] ASoC: dt-bindings: ti,tas2781: Add TAS5822 support
# good: [ecaba8b7990d8c6d8ba097cd4499b3b92d9df6ea] ASoC: tas2781: Add tas5822 support
# good: [9de2057bbdfb58f4d9bb1476135317cd3fe6aa52] regulator: pf9453: optimize PMIC PF9453 driver
# good: [3b071bdd26849172101081573a18022af108fb21] ASoC: tas2781: Replace deprecated strcpy() with strscpy()
# good: [6022eacdda8b0b06a2e1d4122e5268099b62ff5d] spi: tegra210-quad: Refactor error handling into helper functions
# good: [84194c66aaf78fed150edb217b9f341518b1cba2] ASoC: codecs: aw88261: pass pointer directly instead of passing the address
# good: [1a88479436a52452429f6d50ec5bd0a546406485] ASoC: fsl_spdif: Constify some structures
# good: [2ecc8c089802e033d2e5204d21a9f467e2517df9] regulator: pf9453: remove unused I2C_LT register
# good: [252abf2d07d33b1c70a59ba1c9395ba42bbd793e] regulator: Small cleanup in of_get_regulation_constraints()
# good: [a2d4691b3fec6a2360e4ec953d06819ea055c3e7] regulator: pf9453: change the device ID register address
# good: [164d1037c4f86828b77a15c9071f051c5acddc93] ASoC: codecs: simplify aw87390_init() argument a bit
# good: [f47e782c45f8f0c3d7b84edd7e94bfce9eb00b64] ASoC: max98090/91: adding the two virtual Mux widgets in the routes
# good: [947eaef21577f52db046992c887ed9698f9d883c] ASoC: max98090/91: adding two virtual Mux widgets for digital mics
# good: [0144a2b29d95af8523c308116de65d398d6e935b] regulator: pf9453: remove low power mode
# good: [44c1a444b030647803d900e60f5a8af31a782f0e] regcache: flat: Split ->populate() from ->init()
# good: [27fef3048fe95934f6f2f87341eb33ef6581a075] regcache: flat: Remove unneeded check and error message for -ENOMEM
# good: [94a3a95f03154d8d4c6206950a7f6ef9a30baec6] regcache: Add ->populate() callback to separate from ->init()
# good: [ed5d499b5c9cc11dd3edae1a7a55db7dfa4f1bdc] regcache: maple: Split ->populate() from ->init()
# good: [bda6f8749c8e0b10f083dc7a1edf169f349fb776] regcache: rbtree: Split ->populate() from ->init()
# good: [ecd0de438c1f0ee86cf8f6d5047965a2a181444b] spi: tle62x0: Add newline to sysfs attribute output
# good: [3d8096fd378b0b4dae356e6f037d03da83f559d2] ASoC: qcom: sdw: fix memory leak
# good: [f1dfbc1b5cf8650ae9a0d543e5f5335fc0f478ce] ASoC: max98090/91: fixing the stream index
# good: [db66398fb03ee3599ec7387ceb5121b95e648eb3] Add support for an external Master Clock in the
# good: [ef37146360385282b5f6a5b4bf695db30d609887] ASoC: SOF: Intel: remove hyphen from AMP<index> name_prexix
# good: [6ef8e042cdcaabe3e3c68592ba8bfbaee2fa10a3] ASoC: codec: wm8400: replace printk() calls with dev_*() device aware logging
# good: [e73b743bfe8a6ff4e05b5657d3f7586a17ac3ba0] ASoC: soc-core: check ops & auto_selectable_formats in snd_soc_dai_get_fmt() to prevent dereference error
# good: [e2ff7154813a3834692703852604b2099ecf043a] ASoC: tas2783: Fix build for SoundWire API update
# good: [5fa671232f703a404caa05c581411ea858c4cf16] ASoC: qcom: sdm845: make use of common helpers
# good: [28039efa4d8e8bbf98b066133a906bd4e307d496] MAINTAINERS: remove obsolete file entry in DIALOG SEMICONDUCTOR DRIVERS
# good: [541aecd34383a85eaf7c8556779466e394554fed] ASoC: SOF: pcm: Set the PCM device name for HDMI
# good: [20bcda681f8597e86070a4b3b12d1e4f541865d3] ASoC: codecs: va-macro: fix revision checking
# good: [1e5351ba60f5355809f30c61bbd27e97611d2be9] ASoC: cs4271: Disable regulators in component_probe() error path
# good: [cf6bf51b53252284bafc7377a4d8dbf10f048b4d] ASoC: cs4271: Add support for the external mclk
# good: [9c7f7262bc1affb9b9acd2ec2fb1f6314d5d474c] regmap: add flat cache with sparse validity
# good: [d02460317ed9c95aa2c5f6ff1c70e22e1857d95d] ASoC: qcom: sdw: remove redundant code
# good: [2b4d53eb5cf32c5b7b7616f23f08471fd8b3708e] ASoC: SOF: Intel: select SND_SOC_SDW_UTILS in SND_SOC_SOF_HDA_GENERIC
# good: [e062bdfdd6adbb2dee7751d054c1d8df63ddb8b8] regmap: warn users about uninitialized flat cache
# good: [8fdb030fe283c84fd8d378c97ad0f32d6cdec6ce] ASoC: qcom: sc7280: make use of common helpers
# good: [3cd523ba270665861647304aa30500f238ebf26e] ASoC: dt-bindings: cirrus,cs4271: Document mclk clock
# good: [bcba17279327c6e85dee6a97014dc642e2dc93cc] ASoC: qcom: sdw: fix memory leak for sdw_stream_runtime
# good: [eda4a53f8fe021328c468175a02da8d7ad155494] Add SDCA UMP/FDL support
# good: [ac479277c24c859553a6db340aa1518d320bc5e2] Add support for Cirrus Logic CS530x DAC and CODEC
# good: [a4619aadb308db0418b56b237019dc5ab928aa0b] ALSA: cs35l56: Add support for factory calibration
# good: [060028c7fcdd7bc1c6ed61fb25380c0d6c36475b] ASoC: spacemit: fix build warning and error
# good: [6e8146909eae3a42807c750ad84e3fa29c192822] Sndcard compatible for qrb2210/qcm2290
# good: [250eddd69da21999007fd09e17a7c43b92cc6220] allwinner: a523: Enable I2S and SPDIF TX
# good: [f034c16a4663eaf3198dc18b201ba50533fb5b81] ASoC: spacemit: add failure check for spacemit_i2s_init_dai()
# good: [7a78e387654f20e798ceab5bae9c1f5557416431] ASoC: spacemit: add failure check for spacemit_i2s_init_dai()
# good: [66fecfa91deb536a12ddf3d878a99590d7900277] ASoC: spacemit: use `depends on` instead of `select`
# good: [dc74a00c7661a14a672ea7660caca5c4aa661a79] regulator: pca9450: add input supply links
# good: [67e4b0dfcc6702a31fbb6a3015c0dc867e295eb4] ASoC: dt-bindings: allwinner,sun4i-a10-i2s: Add compatible for A523
# good: [4a5ac6cd05a7e54f1585d7779464d6ed6272c134] ASoC: sun4i-spdif: Support SPDIF output on A523 family
# good: [6ddcd78aa7f85e1d94ab7f90c72d1ad0c0c7b6ea] ASoC: dt-bindings: allwinner,sun4i-a10-spdif: Add compatible for A523
# good: [e92e25f777483b7cc3e170214cc84337d7a415cf] ASoC: SDCA: Add UMP timeout handling for FDL
# good: [12aa3160c10a3179c73c4f99a2d5aec0fd907d0c] ASoC: SDCA: Add early IRQ handling
# good: [ef042df96d0e1089764f39ede61bc8f140a4be00] ASoC: SDCA: Add HID button IRQ
# good: [0723affa1bee50c3bd7ca00e00dee07fcef224b8] ASoC: SDCA: Add completion for FDL start and stop
# good: [aeaf27ec6571527e750eed84bb3865a0664ae316] ASoC: SDCA: Add FDL-specific IRQ processing
# good: [4c33cef58965eb655a0ac8e243aa323581ec025f] regulator: pca9450: link regulator inputs to supply groups
# good: [390c05f47d0749b24db65586482308c5fd680fe5] ASoC: SDCA: Pass device register map from IRQ alloc to handlers
# good: [c7b6c6b60594fd1efe35c61bc6a2176b25263ccc] ASoC: SDCA: Force some SDCA Controls to be volatile
# good: [8d557cc4867f2008f440c54b4423464301a1ef4b] ASoC: SDCA: Factor out a helper to find SDCA IRQ data
# good: [dfe7c3401ed3d3bd8e61be8d6d452896513eb52e] ASoC: SDCA: Rely less on the ASoC component in IRQ handling
# good: [7b6be935e7eff06025e18cea4c6620194450abe2] ASoC: SDCA: Parse Function Reset max delay
# good: [cdd27fa3298ad2f39788804f7d09ab31af2b416c] ASoC: cs-amp-lib: Add helpers for factory calibration
# good: [64670a6c062c4d183c366d46e71ee76395af6a15] ALSA: hda/cs35l56: Set cal_index to the amp index
# good: [86df0030b71d7172317d957df17524a7fd6232d4] regulator: dt-bindings: nxp,pca9450: document input supplies
# good: [1afc05996299b4546e8be9b13c89f78e19912c7d] ASoC: cs35l56: Read silicon ID during initialization and save it
# good: [56bbda23d4bece7ce998666118a068e4f71d59fb] ASoC: SDCA: Update externally_requested flag to cover all requests
# good: [191a27faf53edf9e9101901e402bfee49c44073c] ASoC: cs35l56: Create debugfs files for factory calibration
# good: [7159816707dc7040fe3ac4fa3d7ac3d173bd772a] ASoC: SDCA: Pass SoundWire slave to HID
# good: [f7097161e94cd39df7a8848ad0de5f394124ed69] ASoC: cs35l56: Add common code for factory calibration
# good: [01313661b248c5ba586acae09bff57077dbec0a5] regulator: Let raspberrypi drivers depend on ARM
# good: [907364ea3db47530751add6d2d62122ca17329cb] ASoC: SDCA: Add manual PM runtime gets to IRQ handlers
# good: [013a3a66f25af3fb614f45df43983657514944c4] regmap: sdw-mbq: Don't assume the regmap device is the SoundWire slave
# good: [715159314dfafee66e6deb50b4e3431539a919d8] ASoC: SDCA: Rename SoundWire struct device variables
# good: [46a3df50b0cab466099515f2375b01c5be4fb95c] ALSA: hda/cs35l56: Create debugfs files for factory calibration
# good: [daab108504be73182c16a72b9cfe47ac3b1928ca] ASoC: SDCA: Add UMP buffer helper functions
# good: [0a5e9769d088bd1d8faf01207210911b9341b62c] ASoC: SDCA: Parse XU Entity properties
# good: [cf6290eebe3cc4eb677d11aa061d10cb1df12ab9] ASoC: cs-amp-lib-test: Add cases for factory calibration helpers
# good: [c4d096c3ca425562192a3626c30e82651d0f2c1c] ASoC: SDCA: Add SDCA FDL data parsing
# good: [4795375d8aa072e9aacb0b278e6203c6ca41816a] ASoC: cs-amp-lib-test: Add test cases for cs_amp_set_efi_calibration_data()
# good: [71f7990a34cdb11f82d3cbbcddaca77a55635466] ASoC: SDCA: Add FDL library for XU entities
# good: [2b62e66626f05e277c8fdeb50d4c1e0cbab2fe0e] ASoC: cs-amp-lib: Add function to write calibration to UEFI
# good: [ef24466ee1912997c2bd526194006bbca424c24f] ASoC: cs35l56: Add calibration command to store into UEFI
# good: [959400caf51eb31f95d1ab754a285b5546ebd3e4] ASoC: cs-amp-lib: Return attributes from cs_amp_get_efi_variable()
# good: [ddbcd2f396116581ad035fb76a99fc2ed865a85f] ASoC: cs530x: Correct constant naming
# good: [77a58ba7c64ccca20616aa03599766ccb0d1a330] spi: spi-mem: Trace exec_op
# good: [9957614d2b79578b6f9a2512bfbb2bc7bbdc43ce] ASoC: dt-bindings: sound: cirrus: cs530x: Add cs530x
# good: [3941abb26ff327e53e1e8b873cab3ed3d5103eab] ASoC: cs530x: Rename bitfield to reflect common use for ADC and DAC
# good: [e7434adf0c53a84d548226304cdb41c8818da1cb] ASoC: cs530x: Add SPI bus support for cs530x parts
# good: [c63b2315b9cc6b705205c73dcf4591cfeb9a25ae] ASoC: cs530x: Add CODEC and DAC support
# good: [ca4d49f8a21c37be7e5aed80100ca6b13ac3cf9d] ASoC: cs530x: Update the copyright headers
# good: [e973dfe9259095fb509ab12658c68d46f0e439d7] ASoC: qcom: sm8250: add qrb2210-sndcard compatible string
# good: [d29479abaded34b2b1dab2e17efe96a65eba3d61] ASoC: renesas: fsi: Constify struct fsi_stream_handler
# good: [bb65cb96f64e9b4ea2bbd41e4591f3da91414fdb] ASoC: dt-bindings: sound: cirrus: cs530x: Add SPI bus support
# good: [38ff69586bbb3a823dd501972e17075374b685a1] ASoC: cs530x: Rename i2c related structures
# good: [e7ab858390f24a23ba0827066382ba0e6a8e4379] ASoC: cs530x: Correct MCLK reference frequency values
# good: [ee4407e1288ab85be16bacc45195b8bb23d44760] ASoC: dt-bindings: qcom,sm8250: add QRB2210 soundcard
# good: [c37c3e5e390dcd52cbe6178ac53f5a6131ef6f8c] ASoC: cs530x: Check the DEVID matches the devtype
# good: [1e0722a77b4e263854a812c9c106ddef8fd56720] ASoC: cs530x: Sort #include directives and tydy up whitespaces
# good: [f97ebfda8da28a77a0218a448829451ba7e30d5d] ASoC: cs530x: Remove unused struct members and constants
# good: [c17fa4cbc546c431ccf13e9354d5d9c1cd247b7c] ASoC: sdw_utils: add name_prefix for rt1321 part id
# good: [2914f6ea90772ce4a8311a6d5b3ab94e3cd31b12] ASoC: soc_sdw_utils: add cs35l57 support
# good: [f76dbe127f1b5910e37dfe307d2de5c13d61ed89] regulator: dt-bindings: qcom,rpmh: Add support for PMR735D
# good: [310bf433c01f78e0756fd5056a43118a2f77318c] ASoC: max98090/91: fixing a space
# good: [fd5ef3d69f8975bad16c437a337b5cb04c8217a2] spi: spi-qpic-snand: make qcom_spi_ecc_engine_ops_pipelined const
# good: [edf5c8920240dcafe830908549a0edee4ba3b4b0] ASoC: SOF: Fix function topology name check in profile info output
# good: [d054cc3a2ccfb19484f3b54d69b6e416832dc8f4] regulator: rpmh-regulator: Add RPMH regulator support for PMR735D
# good: [2528c15f314ece50218d1273654f630d74109583] ASoC: max98090/91: adding DAPM routing for digital output for max98091
# good: [7c69694cec869e3bf7c810fd94f860253aeb8053] spi: dt-bindings: don't check node names
# good: [108fba26698a75bb8e1acc21553a6860313aebd2] ASoC: Merge up fixes
# good: [638bae3fb225a708dc67db613af62f6d14c4eff4] ASoC: max98090/91: added DAPM widget for digital output for max98091
# good: [8c465b1669bfeaaf0ebd504b96d689e2a2810874] ASoC: spacemit: add i2s support to K1 SoC
# good: [2880c42a0de68e9eb979e5331fbd5b2c79e093ad] ASoC: amd: ps: Propagate the PCI subsystem Vendor and
# good: [e0fb9464295bca2aa92835d02147d3265a83689a] ASoC: use snd_kcontrol_chip() instead of
# good: [5cd5f8fc29fa1b6d7c0a8f2b0a95b896ecadfa42] ASoC: SOF: Intel: add hyphen between name and index to amp name_prefix
# good: [ecba655bf54a661ffe078856cd8dbc898270e4b5] ASoC: fsl_aud2htx: add IEC958_SUBFRAME_LE format in supported list
# good: [6a4f29bc66294d44d61a294e5bdc623eae74587b] ASoC: dt-bindings: don't check node names
# good: [900da53226121c1e710ca95857806a136ab281a2] ASoC: rockchip: i2s-tdm: Omit a variable reassignment in rockchip_i2s_tdm_probe()
# good: [7e1906643a7374529af74b013bba35e4fa4e6ffc] ASoC: codecs: va-macro: Clean up on error path in probe()
# good: [b573898b82abf8d2736c90b14d76f65f9bc8d1f1] ASoC: codecs: twl4030: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [c5c6d972d6450350ecf6b1c6dc7ec7e1462bd221] ASoC: codecs: wcd934x: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [9a1e055bb51d17a44e6e46e2af922dd0d9a1fe2a] ASoC: codecs: tscs454: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [94cd5e54fe56165a2ac7a1ac7b3599a14b9497c0] ASoC: mediatek: mt8192: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [ca6731ad2a4fd1e5fc549d53d4f99445fd552f97] ASoC: rockchip: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [c387ff80f77e6832952fb92d511bfcfda6766a54] ASoC: codecs: wm8991: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [64d1d87d185e0cccdaff573e16af074193045167] ASoC: codecs: tas5805m: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [f72e15146ea024f849143a1d4c65f351cdd12488] ASoC: codecs: wm8731: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [6c8f8e79a3f796106b6de386b2055441b8256efb] ASoC: codecs: wm2000: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [37d9425fc9f19eb92abdcf96189e74e163b94619] ASoC: codecs: tas5086: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [0d58897657a9a31f4817a9596a200815f8c8e602] ASoC: codecs: wm8994: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [308eee447306e885254bca4bc23b9f90534feb1a] ASoC: tegra: tegra210: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [9846cb5a9d06c476c503abfa840251ad3c06414e] ASoC: codecs: wm8400: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [161e3c824e63eec6515f2c7d36fba391647ccf89] ASoC: codecs: wm8904: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [c293d36d1037fc63430f9c4ec7e3fd1e42a50c0e] ASoC: codecs: wm_adsp: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [6e3e296c564399f17e8c99623f8ba608b8fe1b7d] ASoC: codecs: tfa989x: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [3cba794c8fec8e1d5da6c3c91fafdf490de4b24f] ASoC: codecs: sma1307: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [a233cb3fe736647ab2a43aeb3ef1fb9a2a0744cf] ASoC: codecs: sta350: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [9bce11a336f0d5694340f85bdca81179a02eaa81] ASoC: intel: catpt: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [5190935a1b93940c28cc4f484e17662246648f66] ASoC: codecs: wsa883x: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [d742ebcfe524dc54023f7c520d2ed2e4b7203c19] ASoC: soc.h: remove snd_soc_kcontrol_component()
# good: [74fdbce5fe88f9204634e3923c86a84c3a505ecd] ASoC: codecs: wm8983: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [7e1393622591631673c419197dda2d5ff14aacc1] ASoC: codecs: tas5720: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [c59fe12fa2acb1613eb77fe2d690ab76107bd4a4] ASoC: codecs: twl6040: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [6bc24a7d35ceaffa0f7551b7bd13b495a27e203c] ASoC: codecs: wcd9335: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [3314ee8db5411e1f9adc0172ebe425c8a8066cab] ASoC: codecs: wm8962: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [626cf62777735ca51a8d88d8dc2e234b56d4f3a7] ASoC: tegra: tegra186: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [d98fdbdf8bc654242d94616a7bfc233c67b43866] ASoC: codecs: tlv320adc3xxx: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [95972dd17b189647e96605f4edf2177a8876eb4e] ASoC: codecs: wm8350: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [b5e0fc69e42b23e86b32f4810a41ff6b842d6426] ASoC: mediatek: mt8186: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [c8e43faeb4a608f45e362c55a1687f3249479a04] ASoC: codecs: wsa881x: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [7a0fefe28cc5a127cb63aafb5fe85852d198ee7a] ASoC: codecs: wm8753: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [01511c18f713b7579133f451332a1ccbb634e389] ASoC: codecs: tas571x: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [b56580c7ba8ae7ef585a7deefbb137212be6f1f0] ASoC: codecs: tlv320adcx140: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [844db7d7e7200e303fa3974e8dae5350646813d1] ASoC: codecs: wm9081: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [73f64a4803f1e092ed83f4e96225f35c42fe4be1] ASoC: fsl: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [18096d761f6be5a981f3480f7907859f3a431a0b] ASoC: codecs: wm8903: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [96b67f9204fb63168076142777e3cfc2dd236d8c] ASoC: mediatek: common: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [2d54738a39b49fc46b4b22472388d74fc698b87f] ASoC: intel: atom: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [bc78514e84035d1a596eb558ce9da9dd5269a85e] ASoC: codecs: wcd937x: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [7e4cdef1ef8f8a3baa6806d42f6ee2f64d75cc17] ASoC: uniphier: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [f793a6e08f5487b4f1e1207cc4952adc753c1f4b] ASoC: mediatek: mt8188: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [8b412d3233c69a8545ca771c2b4e25774a4a90b0] ASoC: codecs: wm8958: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [386db878776142f262c5997424832090ebe5c196] ASoC: codecs: wm5110: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [661584c295d75ff72fae3569e4f0439325b9835e] ASoC: codecs: wcd938x: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [3576a8d214386e314e83c36c13eb098433896fbc] ASoC: codecs: wm8996: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [9e851421266baaedf86f8a31b62aeda0c72a88d8] ASoC: codecs: wm8990: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [a7aa34438d6367a34555e8cda99ba379ef52938b] ASoC: codecs: sgtl5000: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [1a1a66c2cee604dedbd4de3ca7d8e3c38d7de05c] ASoC: codecs: wm8955: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [504219ea359c4545176f76bab77e0ba38a8629d7] ASoC: codecs: sta32x: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [ba93cda8cc9eb426c801aa8dca8a0e3874de958f] ASoC: codecs: tas2781: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [ecc35b2977af71d5fceef68ca723409cd9592bc3] ASoC: codecs: tlv320aic23: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [a31e9992873ebf3ff66699ee13fdf847891746c6] ASoC: codecs: tscs42xx: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [601e7b30670a25b4cadfb05bded9345126d82b58] ASoC: mediatek: mt8195: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [db1bcf18fe320ff4988a196f969ca739f9c3da95] ASoC: codecs: tas2562: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [b9b68f3a262bce63d4d363f08b345481e4f9d23b] ASoC: mediatek: mt8183: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [024530ea1a3b6e8c64baa76f046345800b99a1bd] ASoC: codecs: wm5102: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [801955fd9248ea5659189b0464e9e4ff0952a11a] ASoC: codecs: wsa884x: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [ae7ac6bbf0fa4475cd169b9ad2e7f751a7cb73f3] ASoC: codecs: uda1334: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [a6531a0414ed50cbbe6244e82fec4d432a207842] ASoC: codecs: tlv320dac33: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [fcc6ae5198bc13860053ea6e1fb7ca85415c77d0] ASoC: codecs: wm8580: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [41a6e1032c799184586d3c5ecb594cc05b844dc7] ASoC: codecs: wm8960: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [88b0ee610790877bd9e3efcf8f29efd53eac894e] ASoC: codecs: wm_hubs: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [ccf4bead907d8be7212e69adeca64a078712d8a6] ASoC: codecs: wm8985: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [c7ade18aad3e63e732a93efb5580a4b2e8563d0c] ASoC: codecs: sma1303: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [f4ae2720f353fe58b02f49cc935933dd9ccc8ce4] ASoC: codecs: wcd939x: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [be9342c57e28d8e42638e4f234d8727da9ee609e] ASoC: codecs: max98925: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [6950709b6658fadc7db860a4ec33de8c5226c728] ASoC: codecs: rt1011: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [2c688df0fd19d02146e031502c701c184b86cd0e] ASoC: codecs: rt5665: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [e7ed084f4cd49806b9ff91d800c6422f96f6f5ea] ASoC: codecs: lpass-tx-macro: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [a5e7aaee483c7022e96181ce899944b8c7b1f1e0] ASoC: codecs: fs210x: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [11c13a65e82afb830fa0f7df2fcf3ed415e6fa1b] ASoC: codecs: lpass-rx-macro: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [52a87e0b5ec1f7316293773f7859996d123b3ea7] ASoC: codecs: lpass-va-macro: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [3a071bb34cea8e97e403c7946a1fc03de164f70c] ASoC: codecs: ntp8835: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [ddd1705b62c6920b77d8cfcccc695e99dde25ad7] ASoC: codecs: da9055: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [ccbfc923fe935b4f1e3b1e51df828e07473faff9] ASoC: codecs: cs42l43: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [cf9d07738fd94e1c3cf0c3ffb61f4d2d1e4d0c57] ASoC: greybus: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [afcf45b7555fcc8e2fc177e8d4e7a2d9d79b13eb] ASoC: codecs: aw88261: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [776b160f3234f85899eb3c8d454c37176484facb] ASoC: codecs: pcm512x: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [e621116150c5983dde75195f0eb2ea22baf4e87c] ASoC: codecs: cs42l51: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [d7e1399884a65e868f7b121b8ca5d613cfac6538] ASoC: codecs: rt711-sdca: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [92ba8c4c78810b991b697af1087ba31226aa5871] ASoC: codecs: ak4458: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [211367ef1d4031a29fa8e07f430155d68802e8ec] ASoC: codecs: cs43130: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [6dcc15ed9e1b0ce6b1f3f53af33e2afaf0a53fa0] ASoC: codecs: pm4125: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [73e86fbf251708bb2766fff746eea0ebb6fe0747] ASoC: codecs: da732x: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [fe0b3f564f9b1ecd74180c296129486d840bc3f9] ASoC: codecs: lpass-wsa-macro: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [d9f08106c16a32cb9514986351754d36f3b3c7df] ASoC: codecs: pcm1681: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [c4ec2c9d7329b509e10c8980e4077f823ce68df6] ASoC: codecs: pcm6240: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [2191593d061cfee48763d67e69ccb1d27655ee62] ASoC: codecs: rt5631: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [eb52e20b15eac1c230fb81f7481f6f1299c2a9b8] ASoC: codecs: cs48l32: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [5366dcebe65bf1dc82a7362e505b69d441b5f755] ASoC: codecs: idt821034: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [e548389df744bd28db1ba3923b6ee74ba6f73926] ASoC: codecs: max98095: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [404e8bfcb9d6a8b6af4745f23906a7382b7eb4c4] ASoC: codecs: aw88166: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [fd27a636d3746ae20e0853f4fa48b93080a2a43d] ASoC: codecs: bd28623: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [a2aa8c0a2f1cbc5be1cb6195038e4484d2af38d9] ASoC: codecs: cs4271: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [14af3d3c4d2d8d01d91049576a59590d9c586f3a] ASoC: codecs: nau8822: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [09b29035fb5c93fa0db24df165081cc27239b1f0] ASoC: codecs: rt5659: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [7e90cf82a33827e174d3955d3ede1b3e7bb56325] ASoC: codecs: nau8810: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [455f2f9509b0034f32979bcdbc5441579c3a1d0a] ASoC: codecs: 88pm860x: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [3c9e6dccede1146d53f940ff8b25ceee275c0686] ASoC: codecs: msm8916-wcd-digital: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [13c4c81b7bc819d533c6e0aaa1bf495cd0dfd8f6] ASoC: codecs: es8328: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [0bfe0c0a9aa7edb88f571bfbfc8f4546d6b311cb] ASoC: codecs: cs4234: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [c0a750ce364b248efc0f4b993904dcef194612cf] ASoC: codecs: cs42l42: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [205a7f5c6ec0051248384e7440fb17d3f3aee372] ASoC: codecs: aw88081: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [46c4e08bb11aeae95c364fa610ac6348896b881a] ASoC: codecs: cs530x: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [f7fbe0ea5f03d918418e6ad28d61bcae52bb2d22] ASoC: codecs: cs35l45: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [5ab215f2b60ec988e63ddde421eecc6ced137064] ASoC: codecs: rt9123: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [2e6db9d197ed161702434c6c14b1c65bdef51a2c] ASoC: codecs: ak4619: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [8f1d72f84e85a8757a0981040654be1fd18ea6e3] ASoC: codecs: cs42l84: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [6047387ab92acfc77dd981df27adca36b0c2e9ea] ASoC: codecs: rt5670: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [8b5cc56389d08debb225d94a35e62e11a80a9b4f] ASoC: codecs: da7210: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [2673034a1911c215f67c64d0b844563f7903a677] ASoC: codecs: max9867: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [c34209ba43623c2ea2593ee332f4e4f6b47fb921] ASoC: codecs: max98088: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [1ad2f1c5662fabafc8df405a2e480ba45cdaf054] ASoC: codecs: rt1318: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [2f0a334df0fd8e0793a7dbea40fca0d4a48dc927] ASoC: codecs: da7218: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [fce217449075d59b29052b8cdac567f0f3e22641] ASoC: spacemit: add i2s support for K1 SoC
# good: [2041666b8f4320da1f7b8efacaafab40a745e126] ASoC: codecs: mt6359: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [c5f73c6679ef675fdb4e35dbc8ae0ec59eb0526c] ASoC: atmel: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [3827b34564951febc0ffa7ce93c1d9c05888e53b] ASoC: codecs: ab8500: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [ac7db529b6c2d748102cef60da29f5ea281a757e] ASoC: codecs: mt6660: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [a43676fa0585a9988e44ec000649d20c2649a3e4] ASoC: codecs: aw88395: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [800c666497e045c3152da516fdd6bc2392571a90] ASoC: codecs: peb2466: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [db25c438f9e02c0f1566d54840640d23f59a3d4f] ASoC: codecs: aw87390: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [40c729f0a3fa14845d733f5572c283728bea0c32] ASoC: codecs: aw88399: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [dc5aa86741789d6dcc0c24dcff5f3ba8fbecccf9] ASoC: codecs: max98090: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [13b2c42b0dcf539a1ddf18374792a7a49a7afa4d] ASoC: codecs: max98390: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [0c933edebd96961529e01686ce6930be5decf400] ASoC: codecs: max9768: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [6658472a3e2de08197acfe099ba71ee0e2505ecf] ASoC: amd: amd_sdw: Propagate the PCI subsystem Vendor and Device IDs
# good: [f2b55e567172ccb248e66e8a0f1c7380279127f1] ASoC: codecs: cros_ec: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [73978d274eba0d9081bc9b5aedebb0bc6abb832c] ASoC: dt-bindings: Add bindings for SpacemiT K1
# good: [344af572f088e2bb1248bb752ea3a532f7fa4208] ASoC: codecs: ak4641: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [4b5e66fc0315b9a85ce817b97a1f33829eeadd29] ASoC: codecs: da7219: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [b1daca0396b91a9b436b0654b8aae6a8220aa51b] ASoC: codecs: madera: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [ef464d9c418741ac83a33c7662db3cffda655b4f] ASoC: codecs: mt6358: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [3c000883d110b381fca21572147a8c0463ba4a25] ASoC: codecs: adav80x: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [aab1301cfde344f966bbc442a4e655867ab56768] ASoC: amd: ps: Propagate the PCI subsystem Vendor and Device IDs
# good: [3a3271a52075d4dc34f403c0ed850801cdc4bd4d] ASoC: codecs: da7213: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [9549a29371c5548a4eb0d43622a25d6bc989337f] ASoC: codecs: cs47l15: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [b805b4e062fa216302f86bd66485734683a51236] ASoC: codecs: rt1015: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [3815962969c811bbfa32c7d503246f9abace38cb] ASoC: codecs: cs35l36: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [62b175d6ff19de91544c9047dcaec8a33a5759a1] ASoC: codecs: arizona: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [ee4ac04e3ccd7f4d54e767b7ab5a4067f9737fff] ASoC: codecs: max9759: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [a014442252380abd19a81ae27fdfc29b988203c6] ASoC: codecs: cs4270: use snd_kcontrol_chip() instead of snd_soc_kcontrol_component()
# good: [0cc08c8130ac8f74419f99fe707dc193b7f79d86] spi: aspeed: Fix an IS_ERR() vs NULL bug in probe()
# good: [0743acf746a81e0460a56fd5ff847d97fa7eb370] spi: airoha: buffer must be 0xff-ed before writing
# good: [902c0ea18a97b1a6eeee5799cb1fd9a79ef9208e] spi: airoha: avoid reading flash page settings from SNFI registers during driver startup
# good: [0cdb2b1b7edaefb54773d790c7b5c2e4ac7db60d] spi: airoha: driver fixes & improvements
# good: [233a22687411ea053a4b169c07324ee6aa33bf38] spi: airoha: unify dirmap read/write code
# good: [661856ca131c8bf6724905966e02149805660abe] spi: airoha: remove unnecessary restriction length
# good: [93b2838c6e79bc263e6129d88c5ab043dd793d28] Add Audio Support for Kaanapali MTP Boards
# good: [7350f8dc15bfbb7abf1ce4babea6fcace1c574c5] spi: airoha: remove unnecessary switch to non-dma mode
# good: [80b09137aeab27e59004383058f8cc696a9ee048] spi: airoha: support of dualio/quadio flash reading commands
# good: [d1ff30df1d9a4eb4c067795abb5e2a66910fd108] spi: airoha: reduce the number of modification of REG_SPI_NFI_CNFG and REG_SPI_NFI_SECCUS_SIZE registers
# good: [5263cd81578f99a00b2dd7de1da2b570b96a1b7c] rpmh-regulators: Update rpmh-regulator driver and
# good: [70eec454f2d6cdfab547c262781acd38328e11a1] spi: airoha: avoid setting of page/oob sizes in REG_SPI_NFI_PAGEFMT
# good: [fb81b5cecb8553e3ca2b45288cf340d43c9c2991] spi: airoha: set custom sector size equal to flash page size
# good: [15afe57a874eaf104bfbb61ec598fa31627f7b19] ASoC: dt-bindings: qcom: Add Kaanapali LPASS macro codecs
# good: [281c97376cfcfc8cef4f5ed5dd961a1b39f5a25e] ASoC: codecs: va-macro: Rework version checking
# good: [d77daa49085b067137d0adbe3263f75a7ee13a1b] spi: aspeed: fix spelling mistake "triming" -> "trimming"
# good: [367ca0688e4218e51c3d4dfdf3ef5657a62cf88d] ASoC: dt-bindings: qcom,sm8250: Add kaanapali sound card
# good: [4673dbe9837e3eb2fecdd12f0953006c31636aac] ASoC: qcom: sc8280xp: Add support for Kaanapali
# good: [1e570e77392f43a3cdab2849d1f81535f8a033e2] ASoC: mxs-saif: support usage with simple-audio-card
# good: [cee2c8396d9c8e834fe28929bc1d8153d7e9897f] ASoC: use sof_sdw as default Intel SOF SDW machine
# good: [1356c98ef911e14ccfaf374800840ce5bdcb3bbd] regulator: dt-bindings: rpmh-regulator: Update pmic-id DT prop info for new CMD-DB
# good: [fb25114cd760c13cf177d9ac37837fafcc9657b5] regulator: sy7636a: add gpios and input regulator
# good: [835dfb12fc389f36eb007657f163bd1c539dcd45] regulator: dt-bindings: rpmh-regulator : Add compatibles for PMH01XX & PMCX0102
# good: [79c36ecfc8994011ab0a973d3b4148aa5d9e0c91] ASoC: use sof_sdw as default Intel SOF SDW machine
# good: [c1afb0350069c3be137b5692923ad13d69648970] Add tegra264 audio device tree support
# good: [65efe5404d151767653c7b7dd39bd2e7ad532c2d] regulator: rpmh-regulator: Add RPMH regulator support for Glymur
# good: [6621b0f118d500092f5f3d72ddddb22aeeb3c3a0] ASoC: codecs: rt5670: use SOC_VALUE_ENUM_SINGLE_DECL for DAC2 L/R MX-1B
# good: [6a8cdef7dc2a4c0dbde3f7d7100b3d99712a766b] regulator: rpmh-regulator: Add support for new resource name format
# good: [433e294c3c5b5d2020085a0e36c1cb47b694690a] regulator: core: forward undervoltage events downstream by default
# good: [a758314f71ba90cca2a5813bbf96c4954a15b613] spi: offload: Add offset parameter
# good: [7d9c2924f61dcabcbc5868bec6054ab4f4de01d1] spi: aspeed: Improve clock, timing and address
# good: [92a42edd347c3b5a9045bb137a33204c6ddc0803] Add target mode support for the DesignWare SPI
# good: [0b0eb7702a9fa410755e86124b4b7cd36e7d1cb4] ASoC: replace use of system_wq with system_dfl_wq
# good: [0d202ae0256e8e7dcea862ead5904fa27cf4ce6a] ASoC: SOF: add platform name into sof_intel_dsp_desc
# good: [7f47685b150dbc20f881d029a7366a81b1d66322] ASoC: SOF: Intel: use sof_sdw as default SDW machine driver
# good: [7e7e2c6e2a1cb250f8d03bb99eed01f6d982d5dd] ASoC: sof-function-topology-lib: escalate the log when missing function topoplogy
# good: [55f8b5a96597a7b88c323a7de7228f9eae8c9943] ASoC: Intel: export sof_sdw_get_tplg_files
# good: [bb940b13998c40d55e186f0cf5d65c592ea1677a] ASoC: SOF: Don't print the monolithic topology name if function topology may be used
# good: [225d70b8074502acee3943bf0c2e839e867cd38c] ASoC: SOF: don't check the existence of dummy topology
# good: [aa1ee85ce3576defd29f2a389d7508d2036af977] ASoC: soc_sdw_utils: add name_prefix to asoc_sdw_codec_info struct
# good: [3180c7b1575d635851f0ceab6bdb176bb15e69dd] ASoC: soc-acpi: make some variables of acpi adr and link adr non-const
# good: [506cbe36a2ac7b504a2362476dc53cd548b7a29e] ASoC: soc_sdw_utils: export asoc_sdw_get_dai_type
# good: [b926b15547d29a88932de3c24a05c12826fc1dbc] spi: dw: rename the spi controller to ctlr
# good: [454cd43a283f7697297c52981c7a499a16725656] spi: dt-bindings: spi-qpic-snand: Add IPQ5424 compatible
# good: [d25de16477657f9eddd4be9abd409515edcc3b9e] ASoC: soc-acpi: make some variables of acpi adr and link adr non-const
# good: [6277a486a7faaa6c87f4bf1d59a2de233a093248] regulator: dt-bindings: Convert Dialog DA9211 Regulators to DT schema
# good: [31dcc7e1f8a9377d8fd9f967f84c121c5ba8f89c] spi: aspeed: Update clock selection strategy
# good: [4d410ba9aa275e7990a270f63ce436990ace1bea] dt-bindings: sound: Update ADMAIF bindings for tegra264
# good: [9797329220a2c6622411eb9ecf6a35b24ce09d04] ASoC: sof-function-topology-lib: escalate the log when missing function topoplogy
# good: [efb79de36e947d136517bac14c139d494fcc72fa] spi: aspeed: Improve timing calibration algorithm for AST2600 platform
# good: [4412ab501677606436e5c49e41151a1e6eac7ac0] spi: dt-bindings: spi-qpic-snand: Add IPQ5332 compatible
# good: [5e537031f322d55315cd384398b726a9a0748d47] ASoC: codecs: Fix the error of excessive semicolons
# good: [630a185fd06109193574d10f38b29812986c21de] spi: aspeed: Force default address decoding range assignment for each CS
# good: [64d87ccfae3326a9561fe41dc6073064a083e0df] spi: aspeed: Only map necessary address window region
# good: [fe8cc44dd173cde5788ab4e3730ac61f3d316d9c] spi: dw: add target mode support
# good: [b546e0023a203e7edf9377ac8f4f490a6965afd6] spi: aspeed: Centralize address decoding region management
# good: [7196fc4e482928a276da853e2687f31cd8ea2611] ASoC: Intel: export sof_sdw_get_tplg_files
# good: [6937ff42f28a13ffdbe2d1f5b9a51a35f626e93a] ASoC: SOF: add platform name into sof_intel_dsp_desc
# good: [2b92b98cc4765fbb0748742e7e0dd94d15d6f178] ASoC: SOF: Don't print the monolithic topology name if function topology may be used
# good: [5226d19d4cae5398caeb93a6052bfb614e0099c7] ASoC: SOF: Intel: use sof_sdw as default SDW machine driver
# good: [5ed60e45c59d66e61586a10433e2b5527d4d72b5] ASoC: soc_sdw_utils: export asoc_sdw_get_dai_type
# good: [0586b53d4a0c7c5a132629f99da934cc674ea4cd] spi: aspeed: Add per-platform adjust_window callback for decoding range
# good: [99c159279c6dfa2c4867c7f76875f58263f8f43b] ASoC: SOF: don't check the existence of dummy topology
# good: [ea97713903784286ef1ce45456f404ed288f19b1] ASoC: soc_sdw_utils: add name_prefix to asoc_sdw_codec_info struct
# good: [29fa213c6ab00c6749db47b47e384cab760c109e] ASoC: dt-bindings: ti,tas2781: Add TAS5802, TAS5815, and TAS5828
# good: [4ebe64f507ca921c5109eb37eae6058b77413d93] ASoC: tas2781: Add TAS5802, TAS5815, and TAS5828
# good: [3c89238ca35bfe176ba34bc688541f90f6fa7bdb] ASoc: tas2783A: Remove unneeded variable assignment
# good: [b83fb1b14c06bdd765903ac852ba20a14e24f227] spi: offload: Add offset parameter
git bisect start 'd724c6f85e80a23ed46b7ebc6e38b527c09d64f5' 'abc1c7acb7daa90afde43c24e73844cf054cd11d' '041e502d4ca66341696fcf54a76f3724b80f0b1f' 'd5089fffe1db04a802b028c2ef4875be1ed452a3' 'be2b723d94f2fa3f8ca028a55d313ba0b3007d44' 'bb52dc1d0342a4d2dccbfb1aedda019b8415cce1' '902f497a1ff5d275d2f48c2422d8a9eaa2174dee' 'bdf96e9135a0cf53a853a19c30fa11131a744062' 'c47f28ef62cb03de1e5a97844a90eda0415da233' '50c28498e9fd6784dea82378d509572d118111f9' 'c22f7a5cd205492c48c6deb72f5efe2ae63c931e' '7ec95f46759ba0843a2695decba3cea028cb84ea' '3ea1b0dbc684191b2e0b5697356a74571ab28155' '38fc5addd2a0e51ba750c1a401efe7e3c84a2916' 'da49a21b3fe9fbee7be0418916c43f27ed91ad0a' 'b871d9adffe5a64a1fd9edcb1aebbcc995b17901' '7a0a87712120329c034b0aae88bdaa05bd046f10' 'd01fbee5c0d3d3061fb16235b71f5a117128e2c1' 'e2c48498a93404743e0565dcac29450fec02e6a3' 'd9813cd23d5a7b254cc1b1c1ea042634d8da62e6' '683c03b95c2ae95ff05ebf0dcc040bf3db633135' '7eb1535855983f67b95c2ba777d686cc17188285' '4980df101676f598ad515725a94424d244a5c5a3' 'dc637ffeed6c32c951520774897601ebd9ffc3d5' '21e68bcb1b0c688c2d9ca0d457922febac650ac1' '0140fc11893bf22928a6f6ebcea96315671d75dc' 'bd79452b39c21599e2cff42e9fbeb182656b6f6a' '402ff043395fd9444810c723056fe741c77dbc21' 'd218ea171430e49412804efb794942dd121a8032' '4dbb5f6e1b4eed64037d4462977c196acab2af16' 'e45979641a9a9dbb48fc77c5b36a5091a92e7227' 'f5cb3ee251b4f9db2761aced191f10579bd7e64e' '93218e3f2cfeb1e7992768dd3e64c9012e705992' 'e3acef6ef89ffbd6c80950d5fa0d024a8d11c1c4' '22e9bd51e518e8fcad269c0c1f4bd84467db093e' '34e437097247f92fba6fac3d6e40e33af5f32e3d' '81f8f29a48defd572ec34d2b9c84374fd05158cc' '9f202872ba04d71c96908c56abcc6e3f4a629a40' '94000534e0883b4f4ba9882e4630cfcdf2af539d' 'd9d0be59be2580f2c5e4b7217aafb980e8c371cf' 'f2fb3f7bdf8b996ef7bd1ecc13c93c4dfc9b0d2a' '20772c4e0f0b58211ebdddfb8606694677c4c4c8' '96498e804cb6629e02747336a0a33e4955449732' 'a060b8c511abb0997381b397e52149a5e3e5259a' '1e4f6db614a310cc34d07ffbf031c76ea9581bcf' '197b3f3c70d61ff1c7ca24f66d567e06fe8ed3d9' 'd4340ff75eaa083f261e16d49f13191236bfad06' 'e511d484cbe44fe48a1b9f621f6a947c72503f9e' 'eb374f764a7012eda28019266a6d9191670c4fa5' 'e992d54c6f970b382ffeacd7c88f68b94a3c6caf' '716d0a0a2ab00c601120c19bb357f4373f4722d1' '4422df6782eb7aa9725a3c09d9ba3c38ecc85df4' 'c69951603150c2b08f69feb8eb2475d58c1ef7cd' '508f3d3b688e1650ed383fe208b323aa6c164420' 'fefd4e0b26392ab7676b13b597166b6478eec4c6' 'be6671d3908e97a2128f5327610a1dcb4d420cfa' '9e510e677090bb794b46348b10e1c8038286e00a' 'd10755f895eb295738c4db23ab970f8acaebd7ce' '575eb51c72251f73d21408829ed39b8b03e67894' '8686dd09358b92c2d3dc7b6d0cc893d1d72ef628' 'c2bdc0301db723b67b9c6cbd572f8df4b473cc4a' '74c772546632500f64a4fc4b318cde9f87a29cf6' '9b5a1d32da7e343e8fbdfbc536770544551ef59b' '9c082ab3ebb8baa1f9f17a5c4a85d2345f5b305c' '4cc345c56b2458d449bf2479cb72afbb8e9dfc1c' '42e142c842cb9c7dbde569ff137974e76d1a0e67' 'd1cca4a46f6dfad9a383e5aa30dd8a63833d7a82' '771f522d64303751f976d888a9d6160a345042e2' '2f8d6a86936e10bb65042f2c4da8c84e164b3693' 'b10489e9da2850bcaa53117df1331dc371df8d35' 'b63f223ce0d402237ef339b8482ecc627320df3d' '1c4143ea606a60a11d623a0bd751c8b0af0119e0' '2cc325972373bf86f561eaec45680d4ab957e4eb' 'aa851ed8746e20d4123efad82fa4ea3dd1f2b14e' 'cf3c9aa075313898ac1669d0214946f0af84d18b' '5b54c4ae6c880024699a7c0391991e45ca053477' '8c827c1c78b7260bfdd33cd8ddf82e3c19c24bb2' 'd0764e1476da99ad781745b5b6a7d583cbf1d7e5' '7fb390727decac201fc9370bb14b17d91c334da6' 'f74aa1e909e7ea960effb5d054cb9ff7e39d84b8' '063e91a9141e708fe07a2a06d50fb0b3ed1ecbea' 'be0c2c4010c2e641772408c2bce1649e5db20f2f' '6a995aaaa4b63c324116f03adb94985654a6cb6e' 'a0593f6d1200bb17b69f85e89a9c6c6bcb2966b2' '2bcf4235819c22b8a6dcbae9d9101d6202ea949c' '49b476efceb52a8cefd4e1c74672b55826680c4e' 'a3fc90c52ad0d525918aa5039cb07d937dc1d841' 'e70ef8a90d431157ad1b314a78125aab50eeda3e' '26de56c382450acd5d397631f490dc2d50e7a9c4' '85216ea34568352c7c06ba7152ab1e9ccc73eff7' '6603016167aace5d4c78bdfa5eb1f928a42d7e47' '84496142859a4c66aab3d82ce110952541614ef3' 'cacd21b61750c998f74e3d6aabfca1315298e699' 'bc8ec019290f00d605423db6ba5fd9db81e83bac' '32fc5bf2ea987504d10afe329727ab8c6fd8c836' '02697e7049ddca90e128a89ca276e3ccaeff447a' 'fbada860c48112083c6212d2a0df5e4d919998c9' '6353596bb18588c5d24c96501081c1ef9cce5754' '2f558a84ced9e7ad6d5e53617d6848df9696ecd9' '256cfc79911658a90caa7655e88c255255068ad1' '857c31f15cef81745855a53262b021bec95ca7a8' '0e435dd54f5e3f4077ae3c8db1d1c3c2e5823a8a' 'd02bbabec7e330346b85a43c0f89ee61144985a8' 'ab947ebd0e06b1bba68702cadec93359371f0c16' 'b6ba1a11ea626dd82672e06ffa7809beb671277a' '8d5a933224b59fa5a4d540f37b77fee5d22b3eb9' '638d14a0a0658f7fd8b868ebb93289d21fb64006' 'a6aa027dffef1b6d6ebc9ef87210493bcf7abf22' '0019f120bfe862a583aa1a4347c1761e947e0054' '8c62a765be9ece320dd4da00dcd038f6b83ea2f2' 'e758b08d7170ee96e3461cec5c5b33e9aff5a75e' '6c65a8c3ad82616e6654c82ec0da33233e6f4e02' '088fa2cc7c4c4ec80a293692456f48c43b4da0e7' 'f0762675a23396e80a30fdb6d71a89dcfb972154' '2990b658ed446abd3b509854e2b266eaef666ac5' '9c95183d4e0cd557bcae21608b03ae9cf0eb87cb' 'beb865ab8e5a2d92ae11daca23bdfa51aea587cb' 'e6995aa8165571c0f85ca259de15edde713bfa7c' 'fd73d62f5f8b61bd396e4e4185ba2f8ad3a712db' '9abf16750a7bb052799229b5cff5b14459d3bb21' '9defa941d828ad48bb24249363b0f956b3dd3770' '620fd65ec496b4fe0f49f3b6bc1fd1f890432fd6' '5fca4f6a958c51f2c82d0838735b1cc6ba4da3dd' '047191ae16d336b5343b3eec1ad3532dedacee2d' '805759446d9e8f278881f03d5b4b6bcacf9f0ce3' '9188f03aa5157abbe8eba3130df4381245fe4cd0' 'c9d3bcafc711794efe493b7ad26ff704d560b725' '21838f53f47e2fa4dfdda71a9bc895f599e7cf9f' '738a0014640cc1a2ee75d66e13327db87fca8a58' 'f1ead097d7ddb7014215518f290e49e7fc374266' 'f39ee5784bb848d320e7e387ef2c11ef21c97e83' '5a3c54a94c9101ce126cae1fe21df0cab5d80c63' 'd4dfae0a980c30ecb4c9c3bd90d36171735dcc79' 'af535c76426ebbd3df12a804fa6da5b9e8c0339e' '5a2b4ddc2cf7f150328f4d1ae617b62a3a3df45c' '1014ec36b43a997a8d8fcf68728d1fd6c891b004' '311c12f7031d845c27e383f2987e53a35b684e5c' '5969275804b8f033618ff8f7309b016deb5e5117' '03258f7765e8c0210ea35b865f8e5c03fd9ccab9' 'eb802ba07bcc1fbcf637b9ccf1ffcdc74277cca4' '97a57e6ea996a773b278106755368755a9c2dc18' '4fb45e5d4abda09bfd805b08fa3446686aa3c6bb' 'e3c9bc1f31214f33e24fae70ade0d748ed5cba95' '0c02abd9f443565b00f79f5d84166d610200b1a5' '22219a4af85457a7ab8fdfa925a1fa2bffe66dd2' 'eac123525d4214d4ef4d751cb48649130b4cf51f' '8124fe3d5b254a0c17da853591616c8d8ee602e7' '61907c9a0d25200ea4681a0ae6b2f7c7673ea5bb' 'a30381e9cc2d32d8e0d4077cfa6aab0c70d34105' '0813e864070dd6fca79619bb4f99e38c97422383' '574a59412d32e8e094e636feeaa4d29b55d35410' 'cb5aaf1bab5f7ef98f04eea303eb39ed05325dfe' '194c43841d9e5695837789f288889ca02c023074' 'dd929092b3122a9cfd1f7115c4eecdb7944c45a8' 'a4230eea80ddb9d08e4098f980ddf1d22bbc18ef' '2988abc11bb7c4cf756b4197de0879ed92bb39bf' 'fdc5ce7f7272e59a1c32919b1ba8ab73d1bc28ab' '1b30118264fc75f164ebf8a9888a16ddd2ebb3d4' 'ca7f864f91bff158a697349dae2c81afae2b3073' '0da5d1fb00c1108e543fa94c92419006a1008d84' 'fef5517fff15652903f56f153938eaf93c2c53a0' '9e1666651224267179d1e4f1e366cd526c384ae8' '7d6ca6db9b6ee95adfdfabe84819b3b1900dfc05' '1a8870e2e2eeb7819ad540f0027faec500833351' '9bcc76f58217c1701deeb4b4092e54dc64ce38bc' '11c256c49d75c3d795339c93f52a19a202db5a15' '7881b033afa1a2f6ec587dabfe2e12d0bfdef022' '636819c3962b9a2377bfdd3baa11e187cba39916' 'aa57703fd738d0d9f1328d44bc6b9ee82daf0326' 'c13e3d24dbe84b398bc9e5ea767e4d3515b06771' '128cd2206689ebddb6f7a9682f5671190a570a56' '46f360c51cfeb7b29340815885e4a7e82b411bc8' 'da9ad75afba487013bd4345d4b77477f35ed25ec' 'bdc210a4485b640b65f3cf9cfd6fbe1e564e7645' '58fc7681c470d39332f01aae88b2259e613e53e3' 'a5a6abd02d100913327a150e764dbc29045970a1' '2049f715599eb677722a2e91d3616c4feff0e1fc' '8b1c56ae39d3bf304ab0c8de456e8f02081be9ed' '8df50a4f2f9faaf336d3640174a828e9b979544b' '24bca18098187d56257a938dfcae2673f8fa847a' '3affdae134871f7be7d28fe5a763f2a17d1c18ab' '8efb94a90e6558a2aeffac518e7b7479fdaa14d9' '92a3e4bb1813cd5eaba1050b9d82356b17daca2e' 'a9b239d098dbbfa8dd0cc56001cfc80053da1317' '2416360daf6e17c10af12f40139bfa84c99f7f2f' '12cdef76abbbd5209829e73cc58d44a0921deeda' '5b35bb517f27fc2401ec3cfd8c02a127627a0188' '88539900888437c446f44b8248797b958e10d14f' '10d5d21a3379b56a0655609918fcc4b4ccb4909d' '8a19eabf16304642a1c281f6676b7248b234ca1d' '0c8392fd64e8be4fab28d8f568582f4cd7092196' '5d9958b2eff8ef283bb70fd0ae82cc6e65372876' '21bb118ea37adc40027771b1f45a246eb3956265' '453dfd1c9b486f2289d60871ee11fa05809303ab' '1ddafb83a431b2d4c847b2640b352cfca0d1b077' 'a3de4a9c3e0627989efe88ea31f422f1da896dcc' '697f92e637d1c0de9ec9d1dc31a6fa3c921be831' '6254032a219c7581df8203b31480f06a2babd46d' 'cffb8638191684198b4e696393c678498791c590' '4f9d291e6828aa92503988e319b175c0fa3303af' '5b08edab386559a7d8549fa94688ea3878e9574f' '46db12032942a47d6c2b1158633514a739f780eb' '588dc19b1246fc05bd62b3471a01b19cafdc3809' 'c153d6da9cde3efc5fbfcf39515d224c2b6ad48f' 'cf3937987c18980c3dffc456c91fc1f7b1cec4ca' 'dde23aa355aa0fc1fbafd40dc53ffb2d67a7e8b9' '3eca632e86ac79012fbbeb3b7753d345f0baad04' '3f553ac8005e07e7c3d84edfdc08727439e4f5c7' '60b6a547a88313022ca10af6a3029891bc638b1d' '5d1a88f50569d1c4637fe29620e6d194f7e50014' '2d8a091725e6f71c87a7ae35e51550693afdef77' '7b10c284495c7f9a666f7057351e67d6bc9c54bd' '9050b9a872a9bb78874ff6b7ba933ca0a90dcecd' 'f72c526d2e7a512ab969c3492a365633f5ff2f33' '52239c555cbd7369b4009dbf72791f0bb56469b6' '664ae39314732223786b1c94ac6031b01886fc84' '0e8d0789ded09e681cb3ba48beaceca125d441f5' '04b80a9aa21f2d02ac6ccbfacc33eb0827eb9899' 'c3282edee7bd18cde6cfdf31a918dbd112dd794b' '00246ae603ef2ee4765fd71923519849a1690286' '0ba0819b8fb4ea27d700e5784e150217e2d9f295' '82d779d150ed27232c940a7908c0f42c248a981f' '6f917e47db4984e784a5245b4e3f35ca16b01ccf' '098c486f07000c70fe18abd879a0abdb96ccfc23' '02dbbb7e982a6873f81e69e4fef59a42decb7b1a' '8d27adc88a5461b08990d33a16d215e3e7e890dc' 'ca6cd3d3f1d7f98908dbc8342a28663346067b9a' '4e111f49809959bd568cf68bea83155908872164' 'ab7963a3adc1dec9dd6efb9f4755fedb9b56d35d' 'e727c8d01a5d6049e756fc15dfec08fbf4f7b0cf' 'acee6bda8872f715dcc6d3ff1fe387606d752661' 'd1b06d4a37c301a09403746f1d12bdcaeed40830' '1b6934ca2a68bca2d6a776497f47ba319419d7dd' 'cf61841233bd0c905b3493f879b45fa9f7d44ff4' '863a396f09f559344e41ccaf2cab6aea12bcc3bc' '23805d059a597b001bd5e3655ee0bae652142944' '8000e233e920062dc08222fc6911e13c193d7719' 'd6abe1e31fef77aab57f252c842acb92517ba9d3' 'f8b494b28806efc8e6dece1f66f30c371376788c' '6ad9058c495691f004e8e697e1acbb2b4775c7e2' 'c5b5f6d75ecc87eb543ca8e867ca1585c03b242a' 'b637319d1fdc3686b669ab9c0ec6faee5e8954c2' '3f2848403a817345c2d0984c49f128fc9dc28b3b' 'eba22be8d911dea8132ae6bf8e83e26740e068b2' 'd4ed7da08c767437400b2550092b7ea67cfac294' '69532b4263a264c5bf9ad11046ad1acb2ab3534e' '42f4ff0c0b21ac72ae3af23c67ce51209809e2a4' 'cd0cd7e57d2b300d3d63cb9befa9edcf3e05ad68' '49a22e9811e1524cf4ee6c84e795526104f4fc3c' 'c2b942fbf952d734646352ff56f04bea6c151d3a' '0314d78729a936de02f97daabd8756eb361922d4' '62d12a8e945605468a8d119960c988014c44f6c0' '6db8ba3439c995a08def399f9eda4e1d2b95afa8' '8268a73d6c93b9cadf323bb9ef92fd01d4eb6529' '8454435315e5a6e2f8e531b5ec038c6db4d6ebfe' '19bf3fb9baf235cca9f9bb52cee488662e5bba0c' 'eeb152d9636f3a1355ce1d2bdc481cb70e0ca2bb' '78ad27bc554d33963fa829632c789ab8fb191a09' '87bb65464be16f83e5a7a6642f33084b6ce22d2d' 'e765581ceba401ef62329a4699403d00cb17ce2b' '2bed4e2a81bcc1e59eb747d0a51e46a24bf3a1b3' 'f7e9d040d2b3243f9682c21ac7dda77a54b03c62' 'c475f3468361bb37815aa5beaac056568d18fb56' '5328d3d6b68d1819c199555aff88edad3ccd8bf3' 'daaf102aee4069d2fcaa4da60949b6e02f845c37' 'f3e9bca581321403c9614a075002e29d8e29c60d' 'bbbd507ab3c76f0ed262a29af89f36177a43d65b' '64f90b329ffadae00b187ac797837ba598289707' 'bdd43845598cae0591f0203d6e70f510c4af852d' '7a7ecd8d0a428e96f9271ea59b067b2a5c318de9' 'b42c7f40d2aa8be24601eaa80c672e2c34742c18' '696713199ea5128a3d031a45f275bbfd275b87db' '0ad0505f61402ddcd08d3a6c8f741f0e1487bdc0' 'cdc8feb84f7f22a9b21483b706382cea3cd248cd' '464ac2a1085749dbfeee899ac06d51bea7ef1041' '8855eb7d29400fb7b2882da33725db2801c410e4' '2dcb4f1d27e9c7a610ca40ac37d2dd177db35a52' 'd6b8ebab363bf3dd46cf4d64727910d648bd35cf' 'efdc8c78a5191c85096ee772a36330ca280873ce' '4d5c668c268b7812ff15452d303974ce247ad378' '31f28cda703d442b75c4ae9f3d0855d787c6e909' '638d7077e67f03926f94201f5508c886d0683adc' '46fb094b54c89ddf314f7e807e5457be9b185bbc' '476fb171ed43ce544d54cd082ec56df2e8f8aa5d' 'aaf1f90c99d920f8ab95413cab7abc627316fdf6' 'eec3b674f2a5c2334d2f1a2200b231975277e2c9' '3df33a9e0faae857dc4c594a915404afa6a7a8f8' 'f7a7e796a36f487758a9444408cef41f94460158' '098931e46c0ed4e0203bcdc626ff94bcd58dbf36' '1505741d727e210650c7f2cbe16ab450f0b1b0bf' '770f17143988f1804d24cb5ab5a4bb9087a715b2' '2e20e32c2e3ac17aae4f53ce6205a5d8bf34e895' 'cc1bb5d435d601035be92d98295cea0b973ae5c2' '729a995620ca34f589135f9852d2ef1c9ee7ebb2' 'af9a1da6c3ae130fc44f218293410859f39bbd80' '16d4e8616e78e5edbcb99268050b431afcfe4610' '45da4a452aeb4f5f44bcbf50079291b70bec9dc5' 'f97289522b7910921d771af00b7c0ec915f4e21f' '5dc2dcbb10614ca77b32340d88d24cef47cd8d58' 'e195aad3cd473030ff696fdecb76a04d11f1d3b4' 'f2d57e22b51c0aa1b2d222b30c07cdf80e4a6ca5' '9c3c02de4883862fdccbc38943ade051fc0de0e2' '75dc6bf1d4dd08d78dd9568eeb041a3e0787084d' '96b0a24773849ca2d051a552a19eee262f1f7444' '6ef8e2f7e5eb6933e0897575e7869741ac34e302' '09ff5df2edbea67256209eb814bca4570436b7c6' 'a7ab96cf1366f2e35d05b1e773db2a4c1b5e4a45' 'bccf1a2e2a7819f2f9caebd5a135fe119f09dbd1' 'f0962b1a35043b8d0e3384b7004f3785b7c7ffd0' '4b9ac2be5fc62ea65c032c3e6e554e60757e5944' '6b1b50ed3c9409a85ff28335ca9b471ed399e652' '57bea9f5f02e76cb81d84de17576faf336526167' '797e4164fe55a03db19930ae94d021173c8e726b' 'c9ba54d36ac20fd94c9479e8b82b32610a4cb979' '6d2188983a12e036531ecc67d89c6e1388855ea1' '4db61b8cd50441c846ac8e6afb53781f62f92029' '4ddf34b148dfb08d47b3e920fdf831eabd973130' '91f20e5167d38eb943a087e3087927d1ce7a5c78' '80ac220583627912ef478a25e22b02bd813d5268' 'f4435734559b8b4bd2dd6c457705b8a22baca0a9' '946d58ae54f8cd9ead30571a71c75a3981d7862a' 'dd9896d41fdf1050934d6a46a1c5ca2164284e72' '82e5de78dbdeec04257d8bdba8465217ad9447d9' '436a4d82d2460926aa2a772acf5399accedfbc00' '12223b4534dd70bc13d792a836f53369db89caa1' '4a9eb06f9b36ca43106e6e12231b5171a660e4ce' '7bb1edac96067d523a30b4ef5808d21c86725f64' '7fdcd1d1add9c3abc1379563c98bcfcc5ce26343' '4d01421647562d3f16319d79197d2afa8f3f1c84' 'ea9d164c67245f5d94da8d46eac1de25fd1059d6' '4c0f28830a4f96487cccfb0e8b78efd628766d91' 'd5c8b7902a41625ea328b52c78ebe750fbf6fef7' '97062ef891393c82107cf44a0eb50c6e1a399e18' '0682c592ec98ba18ac65d1cd4b9c345e2995c121' 'a5511a6e10f1e1462d61aec8420d6ac4cd9e3efe' 'ce99b1dd71340e78d5f8f94af1fdf929bc386b61' 'ee0f171f4b9d21af0b202cec35c44c1eaf6e0beb' '170bc7a0e26ca3d36f543db4111cb7195137c145' '118eb2cb97b8fc0d515bb0449495959247db58f0' '335f0d947f52113110e2ed2ad2c030050d220ad8' 'a3151663d0274f79c52bb7ae11e69c0dad1f2267' 'fa2defb800d7c7b67fa7d9fa95824b7780a90575' '2e20be4b48c1d2886ff11147b302caab76f3c4b7' 'ed71deee195d5d2478363d606b0fbe5665e99d7f' '4bac87b4742d479384ee133c0b591db9234acf0e' 'e6f48607ac801864b99ff93e5193f3f422d55f7b' 'b4403cf77e39c2d0e2694b1cf5311683cbfac789' '0a87517ae3e7af2019f80e32df01252dd8604080' 'c94f134729491ab60fc68fcd919821014334db97' 'c2561572031a1a0ac94d1112fd9b768f65d9bdd5' '05b0e94153e313132b594783fa6a3fe9e0100595' 'af330925a18ff483adace1c4ed2dcc975a31d5c0' '8ce9a2ed153bcaa750aa494e91ce2e70c3b0cdc5' '059f545832be85d29ac9ccc416a16f647aa78485' '71c814e98696f2cd53e9e6cef7501c2d667d4c5a' '6402ddf3027d8975f135cf2b2014d6bbeb2d3436' 'e65b871c9b5af9265aefc5b8cd34993586d93aab' '8d63e85c5b50f1dbfa0ccb214bd91fe5d7e2e860' '55b5d192bab5e152bda8f8cefe837c4ed0ec60c5' 'fd94857a934cbe613353810a024c84d54826ead3' '7c63b5a8ed972a2c8c03d984f6a43349007cea93' '675f41b8d1675d9a51a6dcb978ff76b95cbb4b92' '8ff3dcb0e8a8bf6c41f23ed4aa62d066d3948a10' '5a0438622b49df1e723960ac9c0bf75e04370fdc' '65d03e84d8b8e42cf4636fcabb81c9b130cec710' '893e2fd509e968cc1d76caadee0f5d2f2c72f137' '4e00135b2dd1d7924a58bffa551b6ceb3bd836f2' '123cd174a3782307787268adf45f22de4d290128' '1d562ba0aa7df81335bf96c02be77efe8d5bab87' '4d6e2211aeb932e096f673c88475016b1cc0f8ab' '23523e9398efeee6a5741cf97052babf6fb2a808' '2b0d5d9b39489b9d8171896e8e24d8632c3dc807' '660d946ce6fd64678b4ed6b083dfc24c0f4d5a69' '32172cf3cb543a04c41a1677c97a38e60cad05b6' 'f78d96c289da743d517f700d3fe7c6c43df667b5' '69f3474a01e9867dd99fc4b703fa834ea1835c7d' '38ecd1384079d5f7b6942b68496ee7e85aa9a883' 'b3a5302484033331af37569f7277d00131694b57' '1d5bf23aa1e8aea43beca78dc5d69c68955e8871' '873bc94689d832878befbcadc10b6ad5bb4e0027' 'dfbb57c2767dc128f6360c3e41136bff8fa6f924' '4acbfcf11cbe6c46c42091b49875c7002b0bff3d' '772ada50282b0c80343c8989147db816961f571d' 'e32c402533cc68abe20fcf3246b9ff53e1f96021' '6985defd1d832f1dd9d1977a6a2cc2cef7632704' 'fb1ebb10468da414d57153ddebaab29c38ef1a78' '2089f086303b773e181567fd8d5df3038bd85937' '4db4ce15706d6423cc4cac4b05114b0469507bad' '4e92abd0a11b91af3742197a9ca962c3c00d0948' 'b6f4bd64f453183954184ffbc2b89d73ed8fb135' '6951be397ca8b8b167c9f99b5a11c541148c38cb' '4a58f60df578bb55a1d0adca0c356e03b2818de1' 'bd352547df647be8a1e6c9d4ca2b54b459f3abc1' '1797d254f5c4b46b295527a635af7321a3fe1318' '106d7641e55a472e7523c1f525c77fb6d420064d' '873a46141460d209bb62eaa0dc9e7b67bff924a6' '47d98d432f19030faf06d4b2ca7fc7493057cae1' '6b398c1d3da7a673b13b1857f9fff4c15ee20cef' '56931105074fe7e5fc9d54e3163df3b95075643c' 'abc9a349b87ac0fd3ba8787ca00971b59c2e1257' '1044821176a64a8dc7f861e878506cde34c8b658' '6d34b66fb726a613b98c936adee70d42aa5e4aa7' '55d03b5b5bdd04daf9a35ce49db18d8bb488dffb' '1b0f3f9ee41ee2bdd206667f85ea2aa36dfe6e69' '6bd1ad97eb790570c167d4de4ca59fbc9c33722a' '7a381e373a4243926a41b8e6ebbdeb90fe9afda3' '81c53b52de21b8d5a3de55ebd06b6bf188bf7efd' 'bcb3f6f5286b96ff0a7106d2ca7208744b20b37e' '3c36965df80801344850388592e95033eceea05b' '6e00112d31c86029ad0440f8c29ee0d131984cda' 'bfbb12dfa144d45575bcfe139a71360b3ce80237' 'b828059f86f67729aae3934650ae9e44a59ff9d8' '89cf2223ee7bd83d45c6bef3ac52bedd018f77dd' '655079ac8a7721ac215a0596e3f33b740e01144a' 'a354f030dbce17ab3cf299660a38b162d97f7151' '4b1edbb028fb95b546893068b321a983b1ef50bc' '680c683c8f49455bcfa1604eac4e508ba96ccbfa' 'b6a1d7243ec3a3fe1a3c2969fcd32c802fb5d2a9' '950a4e5788fc7dc6e8e93614a7d4d0449c39fb8d' '896f1e4e106449ac2b39f0ca62b6afaed540f758' '74cc4f3ea4e99262ba0d619c6a4ee33e2cd47f65' 'a87a7b3530728fb7477a74c27fc27e060bf5684f' 'd0f9f5b7a3356b43f78b37b9cc3671ecc7469356' 'f678ea19e8f60b05b8d3a5bcda9a238d5355bd50' 'de8e95773c48bad9d7339ccb376ad22d93932468' '88a5f8e628ef745bb94bdf2fa019d9f8e069ccea' 'e386c2cf3df2b62a6cc78d2f7984102b64753bee' '44432315444cbbedadbb9d808c9fc8c8b21cb4bf' '55094e55ae36c7566e29ae0473d205dbc9d2f4a8' '40a7c5db9020079547358f486ef12d57c1a7aa1f' '8b6faa7fddf0ae69c5f1a9315a64edee6f022037' 'aa897ffc396b48cc39eee133b6b43175d0df9eb5' '2f538ef9f6f7c3d700c68536f21447dfc598f8c8' '380fd29d57abe6679d87ec56babe65ddc5873a37' 'af9c8092d84244ca54ffb590435735f788e7a170' 'b4e002d8a7cee3b1d70efad0e222567f92a73000' 'c4e68959af66df525d71db619ffe44af9178bb22' 'ecaba8b7990d8c6d8ba097cd4499b3b92d9df6ea' '9de2057bbdfb58f4d9bb1476135317cd3fe6aa52' '3b071bdd26849172101081573a18022af108fb21' '6022eacdda8b0b06a2e1d4122e5268099b62ff5d' '84194c66aaf78fed150edb217b9f341518b1cba2' '1a88479436a52452429f6d50ec5bd0a546406485' '2ecc8c089802e033d2e5204d21a9f467e2517df9' '252abf2d07d33b1c70a59ba1c9395ba42bbd793e' 'a2d4691b3fec6a2360e4ec953d06819ea055c3e7' '164d1037c4f86828b77a15c9071f051c5acddc93' 'f47e782c45f8f0c3d7b84edd7e94bfce9eb00b64' '947eaef21577f52db046992c887ed9698f9d883c' '0144a2b29d95af8523c308116de65d398d6e935b' '44c1a444b030647803d900e60f5a8af31a782f0e' '27fef3048fe95934f6f2f87341eb33ef6581a075' '94a3a95f03154d8d4c6206950a7f6ef9a30baec6' 'ed5d499b5c9cc11dd3edae1a7a55db7dfa4f1bdc' 'bda6f8749c8e0b10f083dc7a1edf169f349fb776' 'ecd0de438c1f0ee86cf8f6d5047965a2a181444b' '3d8096fd378b0b4dae356e6f037d03da83f559d2' 'f1dfbc1b5cf8650ae9a0d543e5f5335fc0f478ce' 'db66398fb03ee3599ec7387ceb5121b95e648eb3' 'ef37146360385282b5f6a5b4bf695db30d609887' '6ef8e042cdcaabe3e3c68592ba8bfbaee2fa10a3' 'e73b743bfe8a6ff4e05b5657d3f7586a17ac3ba0' 'e2ff7154813a3834692703852604b2099ecf043a' '5fa671232f703a404caa05c581411ea858c4cf16' '28039efa4d8e8bbf98b066133a906bd4e307d496' '541aecd34383a85eaf7c8556779466e394554fed' '20bcda681f8597e86070a4b3b12d1e4f541865d3' '1e5351ba60f5355809f30c61bbd27e97611d2be9' 'cf6bf51b53252284bafc7377a4d8dbf10f048b4d' '9c7f7262bc1affb9b9acd2ec2fb1f6314d5d474c' 'd02460317ed9c95aa2c5f6ff1c70e22e1857d95d' '2b4d53eb5cf32c5b7b7616f23f08471fd8b3708e' 'e062bdfdd6adbb2dee7751d054c1d8df63ddb8b8' '8fdb030fe283c84fd8d378c97ad0f32d6cdec6ce' '3cd523ba270665861647304aa30500f238ebf26e' 'bcba17279327c6e85dee6a97014dc642e2dc93cc' 'eda4a53f8fe021328c468175a02da8d7ad155494' 'ac479277c24c859553a6db340aa1518d320bc5e2' 'a4619aadb308db0418b56b237019dc5ab928aa0b' '060028c7fcdd7bc1c6ed61fb25380c0d6c36475b' '6e8146909eae3a42807c750ad84e3fa29c192822' '250eddd69da21999007fd09e17a7c43b92cc6220' 'f034c16a4663eaf3198dc18b201ba50533fb5b81' '7a78e387654f20e798ceab5bae9c1f5557416431' '66fecfa91deb536a12ddf3d878a99590d7900277' 'dc74a00c7661a14a672ea7660caca5c4aa661a79' '67e4b0dfcc6702a31fbb6a3015c0dc867e295eb4' '4a5ac6cd05a7e54f1585d7779464d6ed6272c134' '6ddcd78aa7f85e1d94ab7f90c72d1ad0c0c7b6ea' 'e92e25f777483b7cc3e170214cc84337d7a415cf' '12aa3160c10a3179c73c4f99a2d5aec0fd907d0c' 'ef042df96d0e1089764f39ede61bc8f140a4be00' '0723affa1bee50c3bd7ca00e00dee07fcef224b8' 'aeaf27ec6571527e750eed84bb3865a0664ae316' '4c33cef58965eb655a0ac8e243aa323581ec025f' '390c05f47d0749b24db65586482308c5fd680fe5' 'c7b6c6b60594fd1efe35c61bc6a2176b25263ccc' '8d557cc4867f2008f440c54b4423464301a1ef4b' 'dfe7c3401ed3d3bd8e61be8d6d452896513eb52e' '7b6be935e7eff06025e18cea4c6620194450abe2' 'cdd27fa3298ad2f39788804f7d09ab31af2b416c' '64670a6c062c4d183c366d46e71ee76395af6a15' '86df0030b71d7172317d957df17524a7fd6232d4' '1afc05996299b4546e8be9b13c89f78e19912c7d' '56bbda23d4bece7ce998666118a068e4f71d59fb' '191a27faf53edf9e9101901e402bfee49c44073c' '7159816707dc7040fe3ac4fa3d7ac3d173bd772a' 'f7097161e94cd39df7a8848ad0de5f394124ed69' '01313661b248c5ba586acae09bff57077dbec0a5' '907364ea3db47530751add6d2d62122ca17329cb' '013a3a66f25af3fb614f45df43983657514944c4' '715159314dfafee66e6deb50b4e3431539a919d8' '46a3df50b0cab466099515f2375b01c5be4fb95c' 'daab108504be73182c16a72b9cfe47ac3b1928ca' '0a5e9769d088bd1d8faf01207210911b9341b62c' 'cf6290eebe3cc4eb677d11aa061d10cb1df12ab9' 'c4d096c3ca425562192a3626c30e82651d0f2c1c' '4795375d8aa072e9aacb0b278e6203c6ca41816a' '71f7990a34cdb11f82d3cbbcddaca77a55635466' '2b62e66626f05e277c8fdeb50d4c1e0cbab2fe0e' 'ef24466ee1912997c2bd526194006bbca424c24f' '959400caf51eb31f95d1ab754a285b5546ebd3e4' 'ddbcd2f396116581ad035fb76a99fc2ed865a85f' '77a58ba7c64ccca20616aa03599766ccb0d1a330' '9957614d2b79578b6f9a2512bfbb2bc7bbdc43ce' '3941abb26ff327e53e1e8b873cab3ed3d5103eab' 'e7434adf0c53a84d548226304cdb41c8818da1cb' 'c63b2315b9cc6b705205c73dcf4591cfeb9a25ae' 'ca4d49f8a21c37be7e5aed80100ca6b13ac3cf9d' 'e973dfe9259095fb509ab12658c68d46f0e439d7' 'd29479abaded34b2b1dab2e17efe96a65eba3d61' 'bb65cb96f64e9b4ea2bbd41e4591f3da91414fdb' '38ff69586bbb3a823dd501972e17075374b685a1' 'e7ab858390f24a23ba0827066382ba0e6a8e4379' 'ee4407e1288ab85be16bacc45195b8bb23d44760' 'c37c3e5e390dcd52cbe6178ac53f5a6131ef6f8c' '1e0722a77b4e263854a812c9c106ddef8fd56720' 'f97ebfda8da28a77a0218a448829451ba7e30d5d' 'c17fa4cbc546c431ccf13e9354d5d9c1cd247b7c' '2914f6ea90772ce4a8311a6d5b3ab94e3cd31b12' 'f76dbe127f1b5910e37dfe307d2de5c13d61ed89' '310bf433c01f78e0756fd5056a43118a2f77318c' 'fd5ef3d69f8975bad16c437a337b5cb04c8217a2' 'edf5c8920240dcafe830908549a0edee4ba3b4b0' 'd054cc3a2ccfb19484f3b54d69b6e416832dc8f4' '2528c15f314ece50218d1273654f630d74109583' '7c69694cec869e3bf7c810fd94f860253aeb8053' '108fba26698a75bb8e1acc21553a6860313aebd2' '638bae3fb225a708dc67db613af62f6d14c4eff4' '8c465b1669bfeaaf0ebd504b96d689e2a2810874' '2880c42a0de68e9eb979e5331fbd5b2c79e093ad' 'e0fb9464295bca2aa92835d02147d3265a83689a' '5cd5f8fc29fa1b6d7c0a8f2b0a95b896ecadfa42' 'ecba655bf54a661ffe078856cd8dbc898270e4b5' '6a4f29bc66294d44d61a294e5bdc623eae74587b' '900da53226121c1e710ca95857806a136ab281a2' '7e1906643a7374529af74b013bba35e4fa4e6ffc' 'b573898b82abf8d2736c90b14d76f65f9bc8d1f1' 'c5c6d972d6450350ecf6b1c6dc7ec7e1462bd221' '9a1e055bb51d17a44e6e46e2af922dd0d9a1fe2a' '94cd5e54fe56165a2ac7a1ac7b3599a14b9497c0' 'ca6731ad2a4fd1e5fc549d53d4f99445fd552f97' 'c387ff80f77e6832952fb92d511bfcfda6766a54' '64d1d87d185e0cccdaff573e16af074193045167' 'f72e15146ea024f849143a1d4c65f351cdd12488' '6c8f8e79a3f796106b6de386b2055441b8256efb' '37d9425fc9f19eb92abdcf96189e74e163b94619' '0d58897657a9a31f4817a9596a200815f8c8e602' '308eee447306e885254bca4bc23b9f90534feb1a' '9846cb5a9d06c476c503abfa840251ad3c06414e' '161e3c824e63eec6515f2c7d36fba391647ccf89' 'c293d36d1037fc63430f9c4ec7e3fd1e42a50c0e' '6e3e296c564399f17e8c99623f8ba608b8fe1b7d' '3cba794c8fec8e1d5da6c3c91fafdf490de4b24f' 'a233cb3fe736647ab2a43aeb3ef1fb9a2a0744cf' '9bce11a336f0d5694340f85bdca81179a02eaa81' '5190935a1b93940c28cc4f484e17662246648f66' 'd742ebcfe524dc54023f7c520d2ed2e4b7203c19' '74fdbce5fe88f9204634e3923c86a84c3a505ecd' '7e1393622591631673c419197dda2d5ff14aacc1' 'c59fe12fa2acb1613eb77fe2d690ab76107bd4a4' '6bc24a7d35ceaffa0f7551b7bd13b495a27e203c' '3314ee8db5411e1f9adc0172ebe425c8a8066cab' '626cf62777735ca51a8d88d8dc2e234b56d4f3a7' 'd98fdbdf8bc654242d94616a7bfc233c67b43866' '95972dd17b189647e96605f4edf2177a8876eb4e' 'b5e0fc69e42b23e86b32f4810a41ff6b842d6426' 'c8e43faeb4a608f45e362c55a1687f3249479a04' '7a0fefe28cc5a127cb63aafb5fe85852d198ee7a' '01511c18f713b7579133f451332a1ccbb634e389' 'b56580c7ba8ae7ef585a7deefbb137212be6f1f0' '844db7d7e7200e303fa3974e8dae5350646813d1' '73f64a4803f1e092ed83f4e96225f35c42fe4be1' '18096d761f6be5a981f3480f7907859f3a431a0b' '96b67f9204fb63168076142777e3cfc2dd236d8c' '2d54738a39b49fc46b4b22472388d74fc698b87f' 'bc78514e84035d1a596eb558ce9da9dd5269a85e' '7e4cdef1ef8f8a3baa6806d42f6ee2f64d75cc17' 'f793a6e08f5487b4f1e1207cc4952adc753c1f4b' '8b412d3233c69a8545ca771c2b4e25774a4a90b0' '386db878776142f262c5997424832090ebe5c196' '661584c295d75ff72fae3569e4f0439325b9835e' '3576a8d214386e314e83c36c13eb098433896fbc' '9e851421266baaedf86f8a31b62aeda0c72a88d8' 'a7aa34438d6367a34555e8cda99ba379ef52938b' '1a1a66c2cee604dedbd4de3ca7d8e3c38d7de05c' '504219ea359c4545176f76bab77e0ba38a8629d7' 'ba93cda8cc9eb426c801aa8dca8a0e3874de958f' 'ecc35b2977af71d5fceef68ca723409cd9592bc3' 'a31e9992873ebf3ff66699ee13fdf847891746c6' '601e7b30670a25b4cadfb05bded9345126d82b58' 'db1bcf18fe320ff4988a196f969ca739f9c3da95' 'b9b68f3a262bce63d4d363f08b345481e4f9d23b' '024530ea1a3b6e8c64baa76f046345800b99a1bd' '801955fd9248ea5659189b0464e9e4ff0952a11a' 'ae7ac6bbf0fa4475cd169b9ad2e7f751a7cb73f3' 'a6531a0414ed50cbbe6244e82fec4d432a207842' 'fcc6ae5198bc13860053ea6e1fb7ca85415c77d0' '41a6e1032c799184586d3c5ecb594cc05b844dc7' '88b0ee610790877bd9e3efcf8f29efd53eac894e' 'ccf4bead907d8be7212e69adeca64a078712d8a6' 'c7ade18aad3e63e732a93efb5580a4b2e8563d0c' 'f4ae2720f353fe58b02f49cc935933dd9ccc8ce4' 'be9342c57e28d8e42638e4f234d8727da9ee609e' '6950709b6658fadc7db860a4ec33de8c5226c728' '2c688df0fd19d02146e031502c701c184b86cd0e' 'e7ed084f4cd49806b9ff91d800c6422f96f6f5ea' 'a5e7aaee483c7022e96181ce899944b8c7b1f1e0' '11c13a65e82afb830fa0f7df2fcf3ed415e6fa1b' '52a87e0b5ec1f7316293773f7859996d123b3ea7' '3a071bb34cea8e97e403c7946a1fc03de164f70c' 'ddd1705b62c6920b77d8cfcccc695e99dde25ad7' 'ccbfc923fe935b4f1e3b1e51df828e07473faff9' 'cf9d07738fd94e1c3cf0c3ffb61f4d2d1e4d0c57' 'afcf45b7555fcc8e2fc177e8d4e7a2d9d79b13eb' '776b160f3234f85899eb3c8d454c37176484facb' 'e621116150c5983dde75195f0eb2ea22baf4e87c' 'd7e1399884a65e868f7b121b8ca5d613cfac6538' '92ba8c4c78810b991b697af1087ba31226aa5871' '211367ef1d4031a29fa8e07f430155d68802e8ec' '6dcc15ed9e1b0ce6b1f3f53af33e2afaf0a53fa0' '73e86fbf251708bb2766fff746eea0ebb6fe0747' 'fe0b3f564f9b1ecd74180c296129486d840bc3f9' 'd9f08106c16a32cb9514986351754d36f3b3c7df' 'c4ec2c9d7329b509e10c8980e4077f823ce68df6' '2191593d061cfee48763d67e69ccb1d27655ee62' 'eb52e20b15eac1c230fb81f7481f6f1299c2a9b8' '5366dcebe65bf1dc82a7362e505b69d441b5f755' 'e548389df744bd28db1ba3923b6ee74ba6f73926' '404e8bfcb9d6a8b6af4745f23906a7382b7eb4c4' 'fd27a636d3746ae20e0853f4fa48b93080a2a43d' 'a2aa8c0a2f1cbc5be1cb6195038e4484d2af38d9' '14af3d3c4d2d8d01d91049576a59590d9c586f3a' '09b29035fb5c93fa0db24df165081cc27239b1f0' '7e90cf82a33827e174d3955d3ede1b3e7bb56325' '455f2f9509b0034f32979bcdbc5441579c3a1d0a' '3c9e6dccede1146d53f940ff8b25ceee275c0686' '13c4c81b7bc819d533c6e0aaa1bf495cd0dfd8f6' '0bfe0c0a9aa7edb88f571bfbfc8f4546d6b311cb' 'c0a750ce364b248efc0f4b993904dcef194612cf' '205a7f5c6ec0051248384e7440fb17d3f3aee372' '46c4e08bb11aeae95c364fa610ac6348896b881a' 'f7fbe0ea5f03d918418e6ad28d61bcae52bb2d22' '5ab215f2b60ec988e63ddde421eecc6ced137064' '2e6db9d197ed161702434c6c14b1c65bdef51a2c' '8f1d72f84e85a8757a0981040654be1fd18ea6e3' '6047387ab92acfc77dd981df27adca36b0c2e9ea' '8b5cc56389d08debb225d94a35e62e11a80a9b4f' '2673034a1911c215f67c64d0b844563f7903a677' 'c34209ba43623c2ea2593ee332f4e4f6b47fb921' '1ad2f1c5662fabafc8df405a2e480ba45cdaf054' '2f0a334df0fd8e0793a7dbea40fca0d4a48dc927' 'fce217449075d59b29052b8cdac567f0f3e22641' '2041666b8f4320da1f7b8efacaafab40a745e126' 'c5f73c6679ef675fdb4e35dbc8ae0ec59eb0526c' '3827b34564951febc0ffa7ce93c1d9c05888e53b' 'ac7db529b6c2d748102cef60da29f5ea281a757e' 'a43676fa0585a9988e44ec000649d20c2649a3e4' '800c666497e045c3152da516fdd6bc2392571a90' 'db25c438f9e02c0f1566d54840640d23f59a3d4f' '40c729f0a3fa14845d733f5572c283728bea0c32' 'dc5aa86741789d6dcc0c24dcff5f3ba8fbecccf9' '13b2c42b0dcf539a1ddf18374792a7a49a7afa4d' '0c933edebd96961529e01686ce6930be5decf400' '6658472a3e2de08197acfe099ba71ee0e2505ecf' 'f2b55e567172ccb248e66e8a0f1c7380279127f1' '73978d274eba0d9081bc9b5aedebb0bc6abb832c' '344af572f088e2bb1248bb752ea3a532f7fa4208' '4b5e66fc0315b9a85ce817b97a1f33829eeadd29' 'b1daca0396b91a9b436b0654b8aae6a8220aa51b' 'ef464d9c418741ac83a33c7662db3cffda655b4f' '3c000883d110b381fca21572147a8c0463ba4a25' 'aab1301cfde344f966bbc442a4e655867ab56768' '3a3271a52075d4dc34f403c0ed850801cdc4bd4d' '9549a29371c5548a4eb0d43622a25d6bc989337f' 'b805b4e062fa216302f86bd66485734683a51236' '3815962969c811bbfa32c7d503246f9abace38cb' '62b175d6ff19de91544c9047dcaec8a33a5759a1' 'ee4ac04e3ccd7f4d54e767b7ab5a4067f9737fff' 'a014442252380abd19a81ae27fdfc29b988203c6' '0cc08c8130ac8f74419f99fe707dc193b7f79d86' '0743acf746a81e0460a56fd5ff847d97fa7eb370' '902c0ea18a97b1a6eeee5799cb1fd9a79ef9208e' '0cdb2b1b7edaefb54773d790c7b5c2e4ac7db60d' '233a22687411ea053a4b169c07324ee6aa33bf38' '661856ca131c8bf6724905966e02149805660abe' '93b2838c6e79bc263e6129d88c5ab043dd793d28' '7350f8dc15bfbb7abf1ce4babea6fcace1c574c5' '80b09137aeab27e59004383058f8cc696a9ee048' 'd1ff30df1d9a4eb4c067795abb5e2a66910fd108' '5263cd81578f99a00b2dd7de1da2b570b96a1b7c' '70eec454f2d6cdfab547c262781acd38328e11a1' 'fb81b5cecb8553e3ca2b45288cf340d43c9c2991' '15afe57a874eaf104bfbb61ec598fa31627f7b19' '281c97376cfcfc8cef4f5ed5dd961a1b39f5a25e' 'd77daa49085b067137d0adbe3263f75a7ee13a1b' '367ca0688e4218e51c3d4dfdf3ef5657a62cf88d' '4673dbe9837e3eb2fecdd12f0953006c31636aac' '1e570e77392f43a3cdab2849d1f81535f8a033e2' 'cee2c8396d9c8e834fe28929bc1d8153d7e9897f' '1356c98ef911e14ccfaf374800840ce5bdcb3bbd' 'fb25114cd760c13cf177d9ac37837fafcc9657b5' '835dfb12fc389f36eb007657f163bd1c539dcd45' '79c36ecfc8994011ab0a973d3b4148aa5d9e0c91' 'c1afb0350069c3be137b5692923ad13d69648970' '65efe5404d151767653c7b7dd39bd2e7ad532c2d' '6621b0f118d500092f5f3d72ddddb22aeeb3c3a0' '6a8cdef7dc2a4c0dbde3f7d7100b3d99712a766b' '433e294c3c5b5d2020085a0e36c1cb47b694690a' 'a758314f71ba90cca2a5813bbf96c4954a15b613' '7d9c2924f61dcabcbc5868bec6054ab4f4de01d1' '92a42edd347c3b5a9045bb137a33204c6ddc0803' '0b0eb7702a9fa410755e86124b4b7cd36e7d1cb4' '0d202ae0256e8e7dcea862ead5904fa27cf4ce6a' '7f47685b150dbc20f881d029a7366a81b1d66322' '7e7e2c6e2a1cb250f8d03bb99eed01f6d982d5dd' '55f8b5a96597a7b88c323a7de7228f9eae8c9943' 'bb940b13998c40d55e186f0cf5d65c592ea1677a' '225d70b8074502acee3943bf0c2e839e867cd38c' 'aa1ee85ce3576defd29f2a389d7508d2036af977' '3180c7b1575d635851f0ceab6bdb176bb15e69dd' '506cbe36a2ac7b504a2362476dc53cd548b7a29e' 'b926b15547d29a88932de3c24a05c12826fc1dbc' '454cd43a283f7697297c52981c7a499a16725656' 'd25de16477657f9eddd4be9abd409515edcc3b9e' '6277a486a7faaa6c87f4bf1d59a2de233a093248' '31dcc7e1f8a9377d8fd9f967f84c121c5ba8f89c' '4d410ba9aa275e7990a270f63ce436990ace1bea' '9797329220a2c6622411eb9ecf6a35b24ce09d04' 'efb79de36e947d136517bac14c139d494fcc72fa' '4412ab501677606436e5c49e41151a1e6eac7ac0' '5e537031f322d55315cd384398b726a9a0748d47' '630a185fd06109193574d10f38b29812986c21de' '64d87ccfae3326a9561fe41dc6073064a083e0df' 'fe8cc44dd173cde5788ab4e3730ac61f3d316d9c' 'b546e0023a203e7edf9377ac8f4f490a6965afd6' '7196fc4e482928a276da853e2687f31cd8ea2611' '6937ff42f28a13ffdbe2d1f5b9a51a35f626e93a' '2b92b98cc4765fbb0748742e7e0dd94d15d6f178' '5226d19d4cae5398caeb93a6052bfb614e0099c7' '5ed60e45c59d66e61586a10433e2b5527d4d72b5' '0586b53d4a0c7c5a132629f99da934cc674ea4cd' '99c159279c6dfa2c4867c7f76875f58263f8f43b' 'ea97713903784286ef1ce45456f404ed288f19b1' '29fa213c6ab00c6749db47b47e384cab760c109e' '4ebe64f507ca921c5109eb37eae6058b77413d93' '3c89238ca35bfe176ba34bc688541f90f6fa7bdb' 'b83fb1b14c06bdd765903ac852ba20a14e24f227'
# test job: [041e502d4ca66341696fcf54a76f3724b80f0b1f] https://lava.sirena.org.uk/scheduler/job/2111676
# test job: [d5089fffe1db04a802b028c2ef4875be1ed452a3] https://lava.sirena.org.uk/scheduler/job/2111683
# test job: [be2b723d94f2fa3f8ca028a55d313ba0b3007d44] https://lava.sirena.org.uk/scheduler/job/2111589
# test job: [bb52dc1d0342a4d2dccbfb1aedda019b8415cce1] https://lava.sirena.org.uk/scheduler/job/2111704
# test job: [902f497a1ff5d275d2f48c2422d8a9eaa2174dee] https://lava.sirena.org.uk/scheduler/job/2109490
# test job: [bdf96e9135a0cf53a853a19c30fa11131a744062] https://lava.sirena.org.uk/scheduler/job/2109456
# test job: [c47f28ef62cb03de1e5a97844a90eda0415da233] https://lava.sirena.org.uk/scheduler/job/2109421
# test job: [50c28498e9fd6784dea82378d509572d118111f9] https://lava.sirena.org.uk/scheduler/job/2109429
# test job: [c22f7a5cd205492c48c6deb72f5efe2ae63c931e] https://lava.sirena.org.uk/scheduler/job/2111002
# test job: [7ec95f46759ba0843a2695decba3cea028cb84ea] https://lava.sirena.org.uk/scheduler/job/2109653
# test job: [3ea1b0dbc684191b2e0b5697356a74571ab28155] https://lava.sirena.org.uk/scheduler/job/2109604
# test job: [38fc5addd2a0e51ba750c1a401efe7e3c84a2916] https://lava.sirena.org.uk/scheduler/job/2109676
# test job: [da49a21b3fe9fbee7be0418916c43f27ed91ad0a] https://lava.sirena.org.uk/scheduler/job/2109511
# test job: [b871d9adffe5a64a1fd9edcb1aebbcc995b17901] https://lava.sirena.org.uk/scheduler/job/2107631
# test job: [7a0a87712120329c034b0aae88bdaa05bd046f10] https://lava.sirena.org.uk/scheduler/job/2107439
# test job: [d01fbee5c0d3d3061fb16235b71f5a117128e2c1] https://lava.sirena.org.uk/scheduler/job/2107461
# test job: [e2c48498a93404743e0565dcac29450fec02e6a3] https://lava.sirena.org.uk/scheduler/job/2107270
# test job: [d9813cd23d5a7b254cc1b1c1ea042634d8da62e6] https://lava.sirena.org.uk/scheduler/job/2105293
# test job: [683c03b95c2ae95ff05ebf0dcc040bf3db633135] https://lava.sirena.org.uk/scheduler/job/2106024
# test job: [7eb1535855983f67b95c2ba777d686cc17188285] https://lava.sirena.org.uk/scheduler/job/2104203
# test job: [4980df101676f598ad515725a94424d244a5c5a3] https://lava.sirena.org.uk/scheduler/job/2104240
# test job: [dc637ffeed6c32c951520774897601ebd9ffc3d5] https://lava.sirena.org.uk/scheduler/job/2104107
# test job: [21e68bcb1b0c688c2d9ca0d457922febac650ac1] https://lava.sirena.org.uk/scheduler/job/2104001
# test job: [0140fc11893bf22928a6f6ebcea96315671d75dc] https://lava.sirena.org.uk/scheduler/job/2103644
# test job: [bd79452b39c21599e2cff42e9fbeb182656b6f6a] https://lava.sirena.org.uk/scheduler/job/2104003
# test job: [402ff043395fd9444810c723056fe741c77dbc21] https://lava.sirena.org.uk/scheduler/job/2103986
# test job: [d218ea171430e49412804efb794942dd121a8032] https://lava.sirena.org.uk/scheduler/job/2104162
# test job: [4dbb5f6e1b4eed64037d4462977c196acab2af16] https://lava.sirena.org.uk/scheduler/job/2103972
# test job: [e45979641a9a9dbb48fc77c5b36a5091a92e7227] https://lava.sirena.org.uk/scheduler/job/2104266
# test job: [f5cb3ee251b4f9db2761aced191f10579bd7e64e] https://lava.sirena.org.uk/scheduler/job/2104245
# test job: [93218e3f2cfeb1e7992768dd3e64c9012e705992] https://lava.sirena.org.uk/scheduler/job/2103964
# test job: [e3acef6ef89ffbd6c80950d5fa0d024a8d11c1c4] https://lava.sirena.org.uk/scheduler/job/2104220
# test job: [22e9bd51e518e8fcad269c0c1f4bd84467db093e] https://lava.sirena.org.uk/scheduler/job/2103963
# test job: [34e437097247f92fba6fac3d6e40e33af5f32e3d] https://lava.sirena.org.uk/scheduler/job/2104020
# test job: [81f8f29a48defd572ec34d2b9c84374fd05158cc] https://lava.sirena.org.uk/scheduler/job/2104259
# test job: [9f202872ba04d71c96908c56abcc6e3f4a629a40] https://lava.sirena.org.uk/scheduler/job/2103967
# test job: [94000534e0883b4f4ba9882e4630cfcdf2af539d] https://lava.sirena.org.uk/scheduler/job/2104342
# test job: [d9d0be59be2580f2c5e4b7217aafb980e8c371cf] https://lava.sirena.org.uk/scheduler/job/2103975
# test job: [f2fb3f7bdf8b996ef7bd1ecc13c93c4dfc9b0d2a] https://lava.sirena.org.uk/scheduler/job/2102825
# test job: [20772c4e0f0b58211ebdddfb8606694677c4c4c8] https://lava.sirena.org.uk/scheduler/job/2102826
# test job: [96498e804cb6629e02747336a0a33e4955449732] https://lava.sirena.org.uk/scheduler/job/2099600
# test job: [a060b8c511abb0997381b397e52149a5e3e5259a] https://lava.sirena.org.uk/scheduler/job/2107820
# test job: [1e4f6db614a310cc34d07ffbf031c76ea9581bcf] https://lava.sirena.org.uk/scheduler/job/2107775
# test job: [197b3f3c70d61ff1c7ca24f66d567e06fe8ed3d9] https://lava.sirena.org.uk/scheduler/job/2107792
# test job: [d4340ff75eaa083f261e16d49f13191236bfad06] https://lava.sirena.org.uk/scheduler/job/2107851
# test job: [e511d484cbe44fe48a1b9f621f6a947c72503f9e] https://lava.sirena.org.uk/scheduler/job/2109891
# test job: [eb374f764a7012eda28019266a6d9191670c4fa5] https://lava.sirena.org.uk/scheduler/job/2107748
# test job: [e992d54c6f970b382ffeacd7c88f68b94a3c6caf] https://lava.sirena.org.uk/scheduler/job/2107929
# test job: [716d0a0a2ab00c601120c19bb357f4373f4722d1] https://lava.sirena.org.uk/scheduler/job/2095026
# test job: [4422df6782eb7aa9725a3c09d9ba3c38ecc85df4] https://lava.sirena.org.uk/scheduler/job/2096342
# test job: [c69951603150c2b08f69feb8eb2475d58c1ef7cd] https://lava.sirena.org.uk/scheduler/job/2096431
# test job: [508f3d3b688e1650ed383fe208b323aa6c164420] https://lava.sirena.org.uk/scheduler/job/2095064
# test job: [fefd4e0b26392ab7676b13b597166b6478eec4c6] https://lava.sirena.org.uk/scheduler/job/2096238
# test job: [be6671d3908e97a2128f5327610a1dcb4d420cfa] https://lava.sirena.org.uk/scheduler/job/2094988
# test job: [9e510e677090bb794b46348b10e1c8038286e00a] https://lava.sirena.org.uk/scheduler/job/2095100
# test job: [d10755f895eb295738c4db23ab970f8acaebd7ce] https://lava.sirena.org.uk/scheduler/job/2093750
# test job: [575eb51c72251f73d21408829ed39b8b03e67894] https://lava.sirena.org.uk/scheduler/job/2093268
# test job: [8686dd09358b92c2d3dc7b6d0cc893d1d72ef628] https://lava.sirena.org.uk/scheduler/job/2093531
# test job: [c2bdc0301db723b67b9c6cbd572f8df4b473cc4a] https://lava.sirena.org.uk/scheduler/job/2092979
# test job: [74c772546632500f64a4fc4b318cde9f87a29cf6] https://lava.sirena.org.uk/scheduler/job/2093240
# test job: [9b5a1d32da7e343e8fbdfbc536770544551ef59b] https://lava.sirena.org.uk/scheduler/job/2096111
# test job: [9c082ab3ebb8baa1f9f17a5c4a85d2345f5b305c] https://lava.sirena.org.uk/scheduler/job/2093657
# test job: [4cc345c56b2458d449bf2479cb72afbb8e9dfc1c] https://lava.sirena.org.uk/scheduler/job/2093059
# test job: [42e142c842cb9c7dbde569ff137974e76d1a0e67] https://lava.sirena.org.uk/scheduler/job/2096213
# test job: [d1cca4a46f6dfad9a383e5aa30dd8a63833d7a82] https://lava.sirena.org.uk/scheduler/job/2093151
# test job: [771f522d64303751f976d888a9d6160a345042e2] https://lava.sirena.org.uk/scheduler/job/2093749
# test job: [2f8d6a86936e10bb65042f2c4da8c84e164b3693] https://lava.sirena.org.uk/scheduler/job/2093303
# test job: [b10489e9da2850bcaa53117df1331dc371df8d35] https://lava.sirena.org.uk/scheduler/job/2096382
# test job: [b63f223ce0d402237ef339b8482ecc627320df3d] https://lava.sirena.org.uk/scheduler/job/2093572
# test job: [1c4143ea606a60a11d623a0bd751c8b0af0119e0] https://lava.sirena.org.uk/scheduler/job/2093204
# test job: [2cc325972373bf86f561eaec45680d4ab957e4eb] https://lava.sirena.org.uk/scheduler/job/2093655
# test job: [aa851ed8746e20d4123efad82fa4ea3dd1f2b14e] https://lava.sirena.org.uk/scheduler/job/2093003
# test job: [cf3c9aa075313898ac1669d0214946f0af84d18b] https://lava.sirena.org.uk/scheduler/job/2095902
# test job: [5b54c4ae6c880024699a7c0391991e45ca053477] https://lava.sirena.org.uk/scheduler/job/2095848
# test job: [8c827c1c78b7260bfdd33cd8ddf82e3c19c24bb2] https://lava.sirena.org.uk/scheduler/job/2093153
# test job: [d0764e1476da99ad781745b5b6a7d583cbf1d7e5] https://lava.sirena.org.uk/scheduler/job/2093786
# test job: [7fb390727decac201fc9370bb14b17d91c334da6] https://lava.sirena.org.uk/scheduler/job/2093590
# test job: [f74aa1e909e7ea960effb5d054cb9ff7e39d84b8] https://lava.sirena.org.uk/scheduler/job/2093597
# test job: [063e91a9141e708fe07a2a06d50fb0b3ed1ecbea] https://lava.sirena.org.uk/scheduler/job/2093405
# test job: [be0c2c4010c2e641772408c2bce1649e5db20f2f] https://lava.sirena.org.uk/scheduler/job/2093198
# test job: [6a995aaaa4b63c324116f03adb94985654a6cb6e] https://lava.sirena.org.uk/scheduler/job/2093702
# test job: [a0593f6d1200bb17b69f85e89a9c6c6bcb2966b2] https://lava.sirena.org.uk/scheduler/job/2093955
# test job: [2bcf4235819c22b8a6dcbae9d9101d6202ea949c] https://lava.sirena.org.uk/scheduler/job/2095804
# test job: [49b476efceb52a8cefd4e1c74672b55826680c4e] https://lava.sirena.org.uk/scheduler/job/2096364
# test job: [a3fc90c52ad0d525918aa5039cb07d937dc1d841] https://lava.sirena.org.uk/scheduler/job/2096315
# test job: [e70ef8a90d431157ad1b314a78125aab50eeda3e] https://lava.sirena.org.uk/scheduler/job/2093722
# test job: [26de56c382450acd5d397631f490dc2d50e7a9c4] https://lava.sirena.org.uk/scheduler/job/2093046
# test job: [85216ea34568352c7c06ba7152ab1e9ccc73eff7] https://lava.sirena.org.uk/scheduler/job/2093937
# test job: [6603016167aace5d4c78bdfa5eb1f928a42d7e47] https://lava.sirena.org.uk/scheduler/job/2093676
# test job: [84496142859a4c66aab3d82ce110952541614ef3] https://lava.sirena.org.uk/scheduler/job/2093700
# test job: [cacd21b61750c998f74e3d6aabfca1315298e699] https://lava.sirena.org.uk/scheduler/job/2093601
# test job: [bc8ec019290f00d605423db6ba5fd9db81e83bac] https://lava.sirena.org.uk/scheduler/job/2096087
# test job: [32fc5bf2ea987504d10afe329727ab8c6fd8c836] https://lava.sirena.org.uk/scheduler/job/2093647
# test job: [02697e7049ddca90e128a89ca276e3ccaeff447a] https://lava.sirena.org.uk/scheduler/job/2093287
# test job: [fbada860c48112083c6212d2a0df5e4d919998c9] https://lava.sirena.org.uk/scheduler/job/2093242
# test job: [6353596bb18588c5d24c96501081c1ef9cce5754] https://lava.sirena.org.uk/scheduler/job/2093367
# test job: [2f558a84ced9e7ad6d5e53617d6848df9696ecd9] https://lava.sirena.org.uk/scheduler/job/2093568
# test job: [256cfc79911658a90caa7655e88c255255068ad1] https://lava.sirena.org.uk/scheduler/job/2093496
# test job: [857c31f15cef81745855a53262b021bec95ca7a8] https://lava.sirena.org.uk/scheduler/job/2093622
# test job: [0e435dd54f5e3f4077ae3c8db1d1c3c2e5823a8a] https://lava.sirena.org.uk/scheduler/job/2093001
# test job: [d02bbabec7e330346b85a43c0f89ee61144985a8] https://lava.sirena.org.uk/scheduler/job/2093469
# test job: [ab947ebd0e06b1bba68702cadec93359371f0c16] https://lava.sirena.org.uk/scheduler/job/2093678
# test job: [b6ba1a11ea626dd82672e06ffa7809beb671277a] https://lava.sirena.org.uk/scheduler/job/2093002
# test job: [8d5a933224b59fa5a4d540f37b77fee5d22b3eb9] https://lava.sirena.org.uk/scheduler/job/2093332
# test job: [638d14a0a0658f7fd8b868ebb93289d21fb64006] https://lava.sirena.org.uk/scheduler/job/2093784
# test job: [a6aa027dffef1b6d6ebc9ef87210493bcf7abf22] https://lava.sirena.org.uk/scheduler/job/2096190
# test job: [0019f120bfe862a583aa1a4347c1761e947e0054] https://lava.sirena.org.uk/scheduler/job/2096501
# test job: [8c62a765be9ece320dd4da00dcd038f6b83ea2f2] https://lava.sirena.org.uk/scheduler/job/2093035
# test job: [e758b08d7170ee96e3461cec5c5b33e9aff5a75e] https://lava.sirena.org.uk/scheduler/job/2096400
# test job: [6c65a8c3ad82616e6654c82ec0da33233e6f4e02] https://lava.sirena.org.uk/scheduler/job/2093814
# test job: [088fa2cc7c4c4ec80a293692456f48c43b4da0e7] https://lava.sirena.org.uk/scheduler/job/2095941
# test job: [f0762675a23396e80a30fdb6d71a89dcfb972154] https://lava.sirena.org.uk/scheduler/job/2095870
# test job: [2990b658ed446abd3b509854e2b266eaef666ac5] https://lava.sirena.org.uk/scheduler/job/2093364
# test job: [9c95183d4e0cd557bcae21608b03ae9cf0eb87cb] https://lava.sirena.org.uk/scheduler/job/2093366
# test job: [beb865ab8e5a2d92ae11daca23bdfa51aea587cb] https://lava.sirena.org.uk/scheduler/job/2093621
# test job: [e6995aa8165571c0f85ca259de15edde713bfa7c] https://lava.sirena.org.uk/scheduler/job/2093507
# test job: [fd73d62f5f8b61bd396e4e4185ba2f8ad3a712db] https://lava.sirena.org.uk/scheduler/job/2096509
# test job: [9abf16750a7bb052799229b5cff5b14459d3bb21] https://lava.sirena.org.uk/scheduler/job/2093851
# test job: [9defa941d828ad48bb24249363b0f956b3dd3770] https://lava.sirena.org.uk/scheduler/job/2093493
# test job: [620fd65ec496b4fe0f49f3b6bc1fd1f890432fd6] https://lava.sirena.org.uk/scheduler/job/2093653
# test job: [5fca4f6a958c51f2c82d0838735b1cc6ba4da3dd] https://lava.sirena.org.uk/scheduler/job/2093816
# test job: [047191ae16d336b5343b3eec1ad3532dedacee2d] https://lava.sirena.org.uk/scheduler/job/2093454
# test job: [805759446d9e8f278881f03d5b4b6bcacf9f0ce3] https://lava.sirena.org.uk/scheduler/job/2093336
# test job: [9188f03aa5157abbe8eba3130df4381245fe4cd0] https://lava.sirena.org.uk/scheduler/job/2096065
# test job: [c9d3bcafc711794efe493b7ad26ff704d560b725] https://lava.sirena.org.uk/scheduler/job/2093573
# test job: [21838f53f47e2fa4dfdda71a9bc895f599e7cf9f] https://lava.sirena.org.uk/scheduler/job/2093404
# test job: [738a0014640cc1a2ee75d66e13327db87fca8a58] https://lava.sirena.org.uk/scheduler/job/2093097
# test job: [f1ead097d7ddb7014215518f290e49e7fc374266] https://lava.sirena.org.uk/scheduler/job/2093849
# test job: [f39ee5784bb848d320e7e387ef2c11ef21c97e83] https://lava.sirena.org.uk/scheduler/job/2093533
# test job: [5a3c54a94c9101ce126cae1fe21df0cab5d80c63] https://lava.sirena.org.uk/scheduler/job/2093286
# test job: [d4dfae0a980c30ecb4c9c3bd90d36171735dcc79] https://lava.sirena.org.uk/scheduler/job/2096044
# test job: [af535c76426ebbd3df12a804fa6da5b9e8c0339e] https://lava.sirena.org.uk/scheduler/job/2093088
# test job: [5a2b4ddc2cf7f150328f4d1ae617b62a3a3df45c] https://lava.sirena.org.uk/scheduler/job/2093956
# test job: [1014ec36b43a997a8d8fcf68728d1fd6c891b004] https://lava.sirena.org.uk/scheduler/job/2093636
# test job: [311c12f7031d845c27e383f2987e53a35b684e5c] https://lava.sirena.org.uk/scheduler/job/2093406
# test job: [5969275804b8f033618ff8f7309b016deb5e5117] https://lava.sirena.org.uk/scheduler/job/2093614
# test job: [03258f7765e8c0210ea35b865f8e5c03fd9ccab9] https://lava.sirena.org.uk/scheduler/job/2096472
# test job: [eb802ba07bcc1fbcf637b9ccf1ffcdc74277cca4] https://lava.sirena.org.uk/scheduler/job/2096007
# test job: [97a57e6ea996a773b278106755368755a9c2dc18] https://lava.sirena.org.uk/scheduler/job/2093931
# test job: [4fb45e5d4abda09bfd805b08fa3446686aa3c6bb] https://lava.sirena.org.uk/scheduler/job/2095824
# test job: [e3c9bc1f31214f33e24fae70ade0d748ed5cba95] https://lava.sirena.org.uk/scheduler/job/2093532
# test job: [0c02abd9f443565b00f79f5d84166d610200b1a5] https://lava.sirena.org.uk/scheduler/job/2093203
# test job: [22219a4af85457a7ab8fdfa925a1fa2bffe66dd2] https://lava.sirena.org.uk/scheduler/job/2093150
# test job: [eac123525d4214d4ef4d751cb48649130b4cf51f] https://lava.sirena.org.uk/scheduler/job/2093455
# test job: [8124fe3d5b254a0c17da853591616c8d8ee602e7] https://lava.sirena.org.uk/scheduler/job/2093045
# test job: [61907c9a0d25200ea4681a0ae6b2f7c7673ea5bb] https://lava.sirena.org.uk/scheduler/job/2093341
# test job: [a30381e9cc2d32d8e0d4077cfa6aab0c70d34105] https://lava.sirena.org.uk/scheduler/job/2092982
# test job: [0813e864070dd6fca79619bb4f99e38c97422383] https://lava.sirena.org.uk/scheduler/job/2096453
# test job: [574a59412d32e8e094e636feeaa4d29b55d35410] https://lava.sirena.org.uk/scheduler/job/2093089
# test job: [cb5aaf1bab5f7ef98f04eea303eb39ed05325dfe] https://lava.sirena.org.uk/scheduler/job/2093679
# test job: [194c43841d9e5695837789f288889ca02c023074] https://lava.sirena.org.uk/scheduler/job/2093434
# test job: [dd929092b3122a9cfd1f7115c4eecdb7944c45a8] https://lava.sirena.org.uk/scheduler/job/2093571
# test job: [a4230eea80ddb9d08e4098f980ddf1d22bbc18ef] https://lava.sirena.org.uk/scheduler/job/2093704
# test job: [2988abc11bb7c4cf756b4197de0879ed92bb39bf] https://lava.sirena.org.uk/scheduler/job/2093237
# test job: [fdc5ce7f7272e59a1c32919b1ba8ab73d1bc28ab] https://lava.sirena.org.uk/scheduler/job/2096418
# test job: [1b30118264fc75f164ebf8a9888a16ddd2ebb3d4] https://lava.sirena.org.uk/scheduler/job/2093698
# test job: [ca7f864f91bff158a697349dae2c81afae2b3073] https://lava.sirena.org.uk/scheduler/job/2093058
# test job: [0da5d1fb00c1108e543fa94c92419006a1008d84] https://lava.sirena.org.uk/scheduler/job/2093435
# test job: [fef5517fff15652903f56f153938eaf93c2c53a0] https://lava.sirena.org.uk/scheduler/job/2093144
# test job: [9e1666651224267179d1e4f1e366cd526c384ae8] https://lava.sirena.org.uk/scheduler/job/2093673
# test job: [7d6ca6db9b6ee95adfdfabe84819b3b1900dfc05] https://lava.sirena.org.uk/scheduler/job/2094227
# test job: [1a8870e2e2eeb7819ad540f0027faec500833351] https://lava.sirena.org.uk/scheduler/job/2093611
# test job: [9bcc76f58217c1701deeb4b4092e54dc64ce38bc] https://lava.sirena.org.uk/scheduler/job/2094374
# test job: [11c256c49d75c3d795339c93f52a19a202db5a15] https://lava.sirena.org.uk/scheduler/job/2093829
# test job: [7881b033afa1a2f6ec587dabfe2e12d0bfdef022] https://lava.sirena.org.uk/scheduler/job/2093674
# test job: [636819c3962b9a2377bfdd3baa11e187cba39916] https://lava.sirena.org.uk/scheduler/job/2093197
# test job: [aa57703fd738d0d9f1328d44bc6b9ee82daf0326] https://lava.sirena.org.uk/scheduler/job/2093331
# test job: [c13e3d24dbe84b398bc9e5ea767e4d3515b06771] https://lava.sirena.org.uk/scheduler/job/2093418
# test job: [128cd2206689ebddb6f7a9682f5671190a570a56] https://lava.sirena.org.uk/scheduler/job/2093635
# test job: [46f360c51cfeb7b29340815885e4a7e82b411bc8] https://lava.sirena.org.uk/scheduler/job/2092994
# test job: [da9ad75afba487013bd4345d4b77477f35ed25ec] https://lava.sirena.org.uk/scheduler/job/2093334
# test job: [bdc210a4485b640b65f3cf9cfd6fbe1e564e7645] https://lava.sirena.org.uk/scheduler/job/2093530
# test job: [58fc7681c470d39332f01aae88b2259e613e53e3] https://lava.sirena.org.uk/scheduler/job/2093358
# test job: [a5a6abd02d100913327a150e764dbc29045970a1] https://lava.sirena.org.uk/scheduler/job/2093419
# test job: [2049f715599eb677722a2e91d3616c4feff0e1fc] https://lava.sirena.org.uk/scheduler/job/2094188
# test job: [8b1c56ae39d3bf304ab0c8de456e8f02081be9ed] https://lava.sirena.org.uk/scheduler/job/2093777
# test job: [8df50a4f2f9faaf336d3640174a828e9b979544b] https://lava.sirena.org.uk/scheduler/job/2093585
# test job: [24bca18098187d56257a938dfcae2673f8fa847a] https://lava.sirena.org.uk/scheduler/job/2093236
# test job: [3affdae134871f7be7d28fe5a763f2a17d1c18ab] https://lava.sirena.org.uk/scheduler/job/2093506
# test job: [8efb94a90e6558a2aeffac518e7b7479fdaa14d9] https://lava.sirena.org.uk/scheduler/job/2092995
# test job: [92a3e4bb1813cd5eaba1050b9d82356b17daca2e] https://lava.sirena.org.uk/scheduler/job/2093461
# test job: [a9b239d098dbbfa8dd0cc56001cfc80053da1317] https://lava.sirena.org.uk/scheduler/job/2093302
# test job: [2416360daf6e17c10af12f40139bfa84c99f7f2f] https://lava.sirena.org.uk/scheduler/job/2093292
# test job: [12cdef76abbbd5209829e73cc58d44a0921deeda] https://lava.sirena.org.uk/scheduler/job/2093152
# test job: [5b35bb517f27fc2401ec3cfd8c02a127627a0188] https://lava.sirena.org.uk/scheduler/job/2094432
# test job: [88539900888437c446f44b8248797b958e10d14f] https://lava.sirena.org.uk/scheduler/job/2092981
# test job: [10d5d21a3379b56a0655609918fcc4b4ccb4909d] https://lava.sirena.org.uk/scheduler/job/2093588
# test job: [8a19eabf16304642a1c281f6676b7248b234ca1d] https://lava.sirena.org.uk/scheduler/job/2093191
# test job: [0c8392fd64e8be4fab28d8f568582f4cd7092196] https://lava.sirena.org.uk/scheduler/job/2093690
# test job: [5d9958b2eff8ef283bb70fd0ae82cc6e65372876] https://lava.sirena.org.uk/scheduler/job/2093034
# test job: [21bb118ea37adc40027771b1f45a246eb3956265] https://lava.sirena.org.uk/scheduler/job/2093527
# test job: [453dfd1c9b486f2289d60871ee11fa05809303ab] https://lava.sirena.org.uk/scheduler/job/2094053
# test job: [1ddafb83a431b2d4c847b2640b352cfca0d1b077] https://lava.sirena.org.uk/scheduler/job/2093468
# test job: [a3de4a9c3e0627989efe88ea31f422f1da896dcc] https://lava.sirena.org.uk/scheduler/job/2092977
# test job: [697f92e637d1c0de9ec9d1dc31a6fa3c921be831] https://lava.sirena.org.uk/scheduler/job/2093565
# test job: [6254032a219c7581df8203b31480f06a2babd46d] https://lava.sirena.org.uk/scheduler/job/2093261
# test job: [cffb8638191684198b4e696393c678498791c590] https://lava.sirena.org.uk/scheduler/job/2093945
# test job: [4f9d291e6828aa92503988e319b175c0fa3303af] https://lava.sirena.org.uk/scheduler/job/2093609
# test job: [5b08edab386559a7d8549fa94688ea3878e9574f] https://lava.sirena.org.uk/scheduler/job/2093256
# test job: [46db12032942a47d6c2b1158633514a739f780eb] https://lava.sirena.org.uk/scheduler/job/2093675
# test job: [588dc19b1246fc05bd62b3471a01b19cafdc3809] https://lava.sirena.org.uk/scheduler/job/2093552
# test job: [c153d6da9cde3efc5fbfcf39515d224c2b6ad48f] https://lava.sirena.org.uk/scheduler/job/2093632
# test job: [cf3937987c18980c3dffc456c91fc1f7b1cec4ca] https://lava.sirena.org.uk/scheduler/job/2093758
# test job: [dde23aa355aa0fc1fbafd40dc53ffb2d67a7e8b9] https://lava.sirena.org.uk/scheduler/job/2093052
# test job: [3eca632e86ac79012fbbeb3b7753d345f0baad04] https://lava.sirena.org.uk/scheduler/job/2093668
# test job: [3f553ac8005e07e7c3d84edfdc08727439e4f5c7] https://lava.sirena.org.uk/scheduler/job/2093501
# test job: [60b6a547a88313022ca10af6a3029891bc638b1d] https://lava.sirena.org.uk/scheduler/job/2094020
# test job: [5d1a88f50569d1c4637fe29620e6d194f7e50014] https://lava.sirena.org.uk/scheduler/job/2093027
# test job: [2d8a091725e6f71c87a7ae35e51550693afdef77] https://lava.sirena.org.uk/scheduler/job/2094312
# test job: [7b10c284495c7f9a666f7057351e67d6bc9c54bd] https://lava.sirena.org.uk/scheduler/job/2093255
# test job: [9050b9a872a9bb78874ff6b7ba933ca0a90dcecd] https://lava.sirena.org.uk/scheduler/job/2093053
# test job: [f72c526d2e7a512ab969c3492a365633f5ff2f33] https://lava.sirena.org.uk/scheduler/job/2092987
# test job: [52239c555cbd7369b4009dbf72791f0bb56469b6] https://lava.sirena.org.uk/scheduler/job/2093656
# test job: [664ae39314732223786b1c94ac6031b01886fc84] https://lava.sirena.org.uk/scheduler/job/2093363
# test job: [0e8d0789ded09e681cb3ba48beaceca125d441f5] https://lava.sirena.org.uk/scheduler/job/2093398
# test job: [04b80a9aa21f2d02ac6ccbfacc33eb0827eb9899] https://lava.sirena.org.uk/scheduler/job/2093526
# test job: [c3282edee7bd18cde6cfdf31a918dbd112dd794b] https://lava.sirena.org.uk/scheduler/job/2094099
# test job: [00246ae603ef2ee4765fd71923519849a1690286] https://lava.sirena.org.uk/scheduler/job/2093192
# test job: [0ba0819b8fb4ea27d700e5784e150217e2d9f295] https://lava.sirena.org.uk/scheduler/job/2093145
# test job: [82d779d150ed27232c940a7908c0f42c248a981f] https://lava.sirena.org.uk/scheduler/job/2093650
# test job: [6f917e47db4984e784a5245b4e3f35ca16b01ccf] https://lava.sirena.org.uk/scheduler/job/2093503
# test job: [098c486f07000c70fe18abd879a0abdb96ccfc23] https://lava.sirena.org.uk/scheduler/job/2093697
# test job: [02dbbb7e982a6873f81e69e4fef59a42decb7b1a] https://lava.sirena.org.uk/scheduler/job/2094459
# test job: [8d27adc88a5461b08990d33a16d215e3e7e890dc] https://lava.sirena.org.uk/scheduler/job/2093090
# test job: [ca6cd3d3f1d7f98908dbc8342a28663346067b9a] https://lava.sirena.org.uk/scheduler/job/2093858
# test job: [4e111f49809959bd568cf68bea83155908872164] https://lava.sirena.org.uk/scheduler/job/2093634
# test job: [ab7963a3adc1dec9dd6efb9f4755fedb9b56d35d] https://lava.sirena.org.uk/scheduler/job/2093954
# test job: [e727c8d01a5d6049e756fc15dfec08fbf4f7b0cf] https://lava.sirena.org.uk/scheduler/job/2093720
# test job: [acee6bda8872f715dcc6d3ff1fe387606d752661] https://lava.sirena.org.uk/scheduler/job/2093029
# test job: [d1b06d4a37c301a09403746f1d12bdcaeed40830] https://lava.sirena.org.uk/scheduler/job/2094344
# test job: [1b6934ca2a68bca2d6a776497f47ba319419d7dd] https://lava.sirena.org.uk/scheduler/job/2093357
# test job: [cf61841233bd0c905b3493f879b45fa9f7d44ff4] https://lava.sirena.org.uk/scheduler/job/2093403
# test job: [863a396f09f559344e41ccaf2cab6aea12bcc3bc] https://lava.sirena.org.uk/scheduler/job/2093654
# test job: [23805d059a597b001bd5e3655ee0bae652142944] https://lava.sirena.org.uk/scheduler/job/2093397
# test job: [8000e233e920062dc08222fc6911e13c193d7719] https://lava.sirena.org.uk/scheduler/job/2093982
# test job: [d6abe1e31fef77aab57f252c842acb92517ba9d3] https://lava.sirena.org.uk/scheduler/job/2093807
# test job: [f8b494b28806efc8e6dece1f66f30c371376788c] https://lava.sirena.org.uk/scheduler/job/2093613
# test job: [6ad9058c495691f004e8e697e1acbb2b4775c7e2] https://lava.sirena.org.uk/scheduler/job/2094400
# test job: [c5b5f6d75ecc87eb543ca8e867ca1585c03b242a] https://lava.sirena.org.uk/scheduler/job/2093719
# test job: [b637319d1fdc3686b669ab9c0ec6faee5e8954c2] https://lava.sirena.org.uk/scheduler/job/2093096
# test job: [3f2848403a817345c2d0984c49f128fc9dc28b3b] https://lava.sirena.org.uk/scheduler/job/2093465
# test job: [eba22be8d911dea8132ae6bf8e83e26740e068b2] https://lava.sirena.org.uk/scheduler/job/2093229
# test job: [d4ed7da08c767437400b2550092b7ea67cfac294] https://lava.sirena.org.uk/scheduler/job/2093721
# test job: [69532b4263a264c5bf9ad11046ad1acb2ab3534e] https://lava.sirena.org.uk/scheduler/job/2093699
# test job: [42f4ff0c0b21ac72ae3af23c67ce51209809e2a4] https://lava.sirena.org.uk/scheduler/job/2094140
# test job: [cd0cd7e57d2b300d3d63cb9befa9edcf3e05ad68] https://lava.sirena.org.uk/scheduler/job/2093712
# test job: [49a22e9811e1524cf4ee6c84e795526104f4fc3c] https://lava.sirena.org.uk/scheduler/job/2093000
# test job: [c2b942fbf952d734646352ff56f04bea6c151d3a] https://lava.sirena.org.uk/scheduler/job/2094268
# test job: [0314d78729a936de02f97daabd8756eb361922d4] https://lava.sirena.org.uk/scheduler/job/2093683
# test job: [62d12a8e945605468a8d119960c988014c44f6c0] https://lava.sirena.org.uk/scheduler/job/2093091
# test job: [6db8ba3439c995a08def399f9eda4e1d2b95afa8] https://lava.sirena.org.uk/scheduler/job/2093589
# test job: [8268a73d6c93b9cadf323bb9ef92fd01d4eb6529] https://lava.sirena.org.uk/scheduler/job/2093228
# test job: [8454435315e5a6e2f8e531b5ec038c6db4d6ebfe] https://lava.sirena.org.uk/scheduler/job/2093339
# test job: [19bf3fb9baf235cca9f9bb52cee488662e5bba0c] https://lava.sirena.org.uk/scheduler/job/2093293
# test job: [eeb152d9636f3a1355ce1d2bdc481cb70e0ca2bb] https://lava.sirena.org.uk/scheduler/job/2093570
# test job: [78ad27bc554d33963fa829632c789ab8fb191a09] https://lava.sirena.org.uk/scheduler/job/2093241
# test job: [87bb65464be16f83e5a7a6642f33084b6ce22d2d] https://lava.sirena.org.uk/scheduler/job/2093411
# test job: [e765581ceba401ef62329a4699403d00cb17ce2b] https://lava.sirena.org.uk/scheduler/job/2094827
# test job: [2bed4e2a81bcc1e59eb747d0a51e46a24bf3a1b3] https://lava.sirena.org.uk/scheduler/job/2093663
# test job: [f7e9d040d2b3243f9682c21ac7dda77a54b03c62] https://lava.sirena.org.uk/scheduler/job/2093284
# test job: [c475f3468361bb37815aa5beaac056568d18fb56] https://lava.sirena.org.uk/scheduler/job/2093682
# test job: [5328d3d6b68d1819c199555aff88edad3ccd8bf3] https://lava.sirena.org.uk/scheduler/job/2093008
# test job: [daaf102aee4069d2fcaa4da60949b6e02f845c37] https://lava.sirena.org.uk/scheduler/job/2092975
# test job: [f3e9bca581321403c9614a075002e29d8e29c60d] https://lava.sirena.org.uk/scheduler/job/2094601
# test job: [bbbd507ab3c76f0ed262a29af89f36177a43d65b] https://lava.sirena.org.uk/scheduler/job/2093217
# test job: [64f90b329ffadae00b187ac797837ba598289707] https://lava.sirena.org.uk/scheduler/job/2094633
# test job: [bdd43845598cae0591f0203d6e70f510c4af852d] https://lava.sirena.org.uk/scheduler/job/2093014
# test job: [7a7ecd8d0a428e96f9271ea59b067b2a5c318de9] https://lava.sirena.org.uk/scheduler/job/2093578
# test job: [b42c7f40d2aa8be24601eaa80c672e2c34742c18] https://lava.sirena.org.uk/scheduler/job/2094664
# test job: [696713199ea5128a3d031a45f275bbfd275b87db] https://lava.sirena.org.uk/scheduler/job/2094802
# test job: [0ad0505f61402ddcd08d3a6c8f741f0e1487bdc0] https://lava.sirena.org.uk/scheduler/job/2093383
# test job: [cdc8feb84f7f22a9b21483b706382cea3cd248cd] https://lava.sirena.org.uk/scheduler/job/2094892
# test job: [464ac2a1085749dbfeee899ac06d51bea7ef1041] https://lava.sirena.org.uk/scheduler/job/2093551
# test job: [8855eb7d29400fb7b2882da33725db2801c410e4] https://lava.sirena.org.uk/scheduler/job/2094480
# test job: [2dcb4f1d27e9c7a610ca40ac37d2dd177db35a52] https://lava.sirena.org.uk/scheduler/job/2093310
# test job: [d6b8ebab363bf3dd46cf4d64727910d648bd35cf] https://lava.sirena.org.uk/scheduler/job/2094692
# test job: [efdc8c78a5191c85096ee772a36330ca280873ce] https://lava.sirena.org.uk/scheduler/job/2093044
# test job: [4d5c668c268b7812ff15452d303974ce247ad378] https://lava.sirena.org.uk/scheduler/job/2094510
# test job: [31f28cda703d442b75c4ae9f3d0855d787c6e909] https://lava.sirena.org.uk/scheduler/job/2093244
# test job: [638d7077e67f03926f94201f5508c886d0683adc] https://lava.sirena.org.uk/scheduler/job/2092986
# test job: [46fb094b54c89ddf314f7e807e5457be9b185bbc] https://lava.sirena.org.uk/scheduler/job/2093243
# test job: [476fb171ed43ce544d54cd082ec56df2e8f8aa5d] https://lava.sirena.org.uk/scheduler/job/2093080
# test job: [aaf1f90c99d920f8ab95413cab7abc627316fdf6] https://lava.sirena.org.uk/scheduler/job/2093450
# test job: [eec3b674f2a5c2334d2f1a2200b231975277e2c9] https://lava.sirena.org.uk/scheduler/job/2093218
# test job: [3df33a9e0faae857dc4c594a915404afa6a7a8f8] https://lava.sirena.org.uk/scheduler/job/2093640
# test job: [f7a7e796a36f487758a9444408cef41f94460158] https://lava.sirena.org.uk/scheduler/job/2092988
# test job: [098931e46c0ed4e0203bcdc626ff94bcd58dbf36] https://lava.sirena.org.uk/scheduler/job/2093350
# test job: [1505741d727e210650c7f2cbe16ab450f0b1b0bf] https://lava.sirena.org.uk/scheduler/job/2093042
# test job: [770f17143988f1804d24cb5ab5a4bb9087a715b2] https://lava.sirena.org.uk/scheduler/job/2093600
# test job: [2e20e32c2e3ac17aae4f53ce6205a5d8bf34e895] https://lava.sirena.org.uk/scheduler/job/2093159
# test job: [cc1bb5d435d601035be92d98295cea0b973ae5c2] https://lava.sirena.org.uk/scheduler/job/2093474
# test job: [729a995620ca34f589135f9852d2ef1c9ee7ebb2] https://lava.sirena.org.uk/scheduler/job/2093512
# test job: [af9a1da6c3ae130fc44f218293410859f39bbd80] https://lava.sirena.org.uk/scheduler/job/2093618
# test job: [16d4e8616e78e5edbcb99268050b431afcfe4610] https://lava.sirena.org.uk/scheduler/job/2093349
# test job: [45da4a452aeb4f5f44bcbf50079291b70bec9dc5] https://lava.sirena.org.uk/scheduler/job/2093043
# test job: [f97289522b7910921d771af00b7c0ec915f4e21f] https://lava.sirena.org.uk/scheduler/job/2093708
# test job: [5dc2dcbb10614ca77b32340d88d24cef47cd8d58] https://lava.sirena.org.uk/scheduler/job/2093554
# test job: [e195aad3cd473030ff696fdecb76a04d11f1d3b4] https://lava.sirena.org.uk/scheduler/job/2093386
# test job: [f2d57e22b51c0aa1b2d222b30c07cdf80e4a6ca5] https://lava.sirena.org.uk/scheduler/job/2093081
# test job: [9c3c02de4883862fdccbc38943ade051fc0de0e2] https://lava.sirena.org.uk/scheduler/job/2093122
# test job: [75dc6bf1d4dd08d78dd9568eeb041a3e0787084d] https://lava.sirena.org.uk/scheduler/job/2093707
# test job: [96b0a24773849ca2d051a552a19eee262f1f7444] https://lava.sirena.org.uk/scheduler/job/2093384
# test job: [6ef8e2f7e5eb6933e0897575e7869741ac34e302] https://lava.sirena.org.uk/scheduler/job/2093599
# test job: [09ff5df2edbea67256209eb814bca4570436b7c6] https://lava.sirena.org.uk/scheduler/job/2093684
# test job: [a7ab96cf1366f2e35d05b1e773db2a4c1b5e4a45] https://lava.sirena.org.uk/scheduler/job/2093475
# test job: [bccf1a2e2a7819f2f9caebd5a135fe119f09dbd1] https://lava.sirena.org.uk/scheduler/job/2093641
# test job: [f0962b1a35043b8d0e3384b7004f3785b7c7ffd0] https://lava.sirena.org.uk/scheduler/job/2093082
# test job: [4b9ac2be5fc62ea65c032c3e6e554e60757e5944] https://lava.sirena.org.uk/scheduler/job/2093576
# test job: [6b1b50ed3c9409a85ff28335ca9b471ed399e652] https://lava.sirena.org.uk/scheduler/job/2094540
# test job: [57bea9f5f02e76cb81d84de17576faf336526167] https://lava.sirena.org.uk/scheduler/job/2093515
# test job: [797e4164fe55a03db19930ae94d021173c8e726b] https://lava.sirena.org.uk/scheduler/job/2093282
# test job: [c9ba54d36ac20fd94c9479e8b82b32610a4cb979] https://lava.sirena.org.uk/scheduler/job/2092976
# test job: [6d2188983a12e036531ecc67d89c6e1388855ea1] https://lava.sirena.org.uk/scheduler/job/2094952
# test job: [4db61b8cd50441c846ac8e6afb53781f62f92029] https://lava.sirena.org.uk/scheduler/job/2093639
# test job: [4ddf34b148dfb08d47b3e920fdf831eabd973130] https://lava.sirena.org.uk/scheduler/job/2093514
# test job: [91f20e5167d38eb943a087e3087927d1ce7a5c78] https://lava.sirena.org.uk/scheduler/job/2094922
# test job: [80ac220583627912ef478a25e22b02bd813d5268] https://lava.sirena.org.uk/scheduler/job/2092974
# test job: [f4435734559b8b4bd2dd6c457705b8a22baca0a9] https://lava.sirena.org.uk/scheduler/job/2093451
# test job: [946d58ae54f8cd9ead30571a71c75a3981d7862a] https://lava.sirena.org.uk/scheduler/job/2094768
# test job: [dd9896d41fdf1050934d6a46a1c5ca2164284e72] https://lava.sirena.org.uk/scheduler/job/2092373
# test job: [82e5de78dbdeec04257d8bdba8465217ad9447d9] https://lava.sirena.org.uk/scheduler/job/2093598
# test job: [436a4d82d2460926aa2a772acf5399accedfbc00] https://lava.sirena.org.uk/scheduler/job/2093219
# test job: [12223b4534dd70bc13d792a836f53369db89caa1] https://lava.sirena.org.uk/scheduler/job/2093348
# test job: [4a9eb06f9b36ca43106e6e12231b5171a660e4ce] https://lava.sirena.org.uk/scheduler/job/2093163
# test job: [7bb1edac96067d523a30b4ef5808d21c86725f64] https://lava.sirena.org.uk/scheduler/job/2093452
# test job: [7fdcd1d1add9c3abc1379563c98bcfcc5ce26343] https://lava.sirena.org.uk/scheduler/job/2093121
# test job: [4d01421647562d3f16319d79197d2afa8f3f1c84] https://lava.sirena.org.uk/scheduler/job/2093123
# test job: [ea9d164c67245f5d94da8d46eac1de25fd1059d6] https://lava.sirena.org.uk/scheduler/job/2093577
# test job: [4c0f28830a4f96487cccfb0e8b78efd628766d91] https://lava.sirena.org.uk/scheduler/job/2093617
# test job: [d5c8b7902a41625ea328b52c78ebe750fbf6fef7] https://lava.sirena.org.uk/scheduler/job/2092377
# test job: [97062ef891393c82107cf44a0eb50c6e1a399e18] https://lava.sirena.org.uk/scheduler/job/2093006
# test job: [0682c592ec98ba18ac65d1cd4b9c345e2995c121] https://lava.sirena.org.uk/scheduler/job/2093160
# test job: [a5511a6e10f1e1462d61aec8420d6ac4cd9e3efe] https://lava.sirena.org.uk/scheduler/job/2093409
# test job: [ce99b1dd71340e78d5f8f94af1fdf929bc386b61] https://lava.sirena.org.uk/scheduler/job/2093283
# test job: [ee0f171f4b9d21af0b202cec35c44c1eaf6e0beb] https://lava.sirena.org.uk/scheduler/job/2094572
# test job: [170bc7a0e26ca3d36f543db4111cb7195137c145] https://lava.sirena.org.uk/scheduler/job/2094861
# test job: [118eb2cb97b8fc0d515bb0449495959247db58f0] https://lava.sirena.org.uk/scheduler/job/2092372
# test job: [335f0d947f52113110e2ed2ad2c030050d220ad8] https://lava.sirena.org.uk/scheduler/job/2093619
# test job: [a3151663d0274f79c52bb7ae11e69c0dad1f2267] https://lava.sirena.org.uk/scheduler/job/2093308
# test job: [fa2defb800d7c7b67fa7d9fa95824b7780a90575] https://lava.sirena.org.uk/scheduler/job/2093706
# test job: [2e20be4b48c1d2886ff11147b302caab76f3c4b7] https://lava.sirena.org.uk/scheduler/job/2093662
# test job: [ed71deee195d5d2478363d606b0fbe5665e99d7f] https://lava.sirena.org.uk/scheduler/job/2094741
# test job: [4bac87b4742d479384ee133c0b591db9234acf0e] https://lava.sirena.org.uk/scheduler/job/2093312
# test job: [e6f48607ac801864b99ff93e5193f3f422d55f7b] https://lava.sirena.org.uk/scheduler/job/2093661
# test job: [b4403cf77e39c2d0e2694b1cf5311683cbfac789] https://lava.sirena.org.uk/scheduler/job/2093410
# test job: [0a87517ae3e7af2019f80e32df01252dd8604080] https://lava.sirena.org.uk/scheduler/job/2093473
# test job: [c94f134729491ab60fc68fcd919821014334db97] https://lava.sirena.org.uk/scheduler/job/2086972
# test job: [c2561572031a1a0ac94d1112fd9b768f65d9bdd5] https://lava.sirena.org.uk/scheduler/job/2086978
# test job: [05b0e94153e313132b594783fa6a3fe9e0100595] https://lava.sirena.org.uk/scheduler/job/2086984
# test job: [af330925a18ff483adace1c4ed2dcc975a31d5c0] https://lava.sirena.org.uk/scheduler/job/2086969
# test job: [8ce9a2ed153bcaa750aa494e91ce2e70c3b0cdc5] https://lava.sirena.org.uk/scheduler/job/2086641
# test job: [059f545832be85d29ac9ccc416a16f647aa78485] https://lava.sirena.org.uk/scheduler/job/2086666
# test job: [71c814e98696f2cd53e9e6cef7501c2d667d4c5a] https://lava.sirena.org.uk/scheduler/job/2086688
# test job: [6402ddf3027d8975f135cf2b2014d6bbeb2d3436] https://lava.sirena.org.uk/scheduler/job/2086571
# test job: [e65b871c9b5af9265aefc5b8cd34993586d93aab] https://lava.sirena.org.uk/scheduler/job/2082961
# test job: [8d63e85c5b50f1dbfa0ccb214bd91fe5d7e2e860] https://lava.sirena.org.uk/scheduler/job/2082443
# test job: [55b5d192bab5e152bda8f8cefe837c4ed0ec60c5] https://lava.sirena.org.uk/scheduler/job/2082456
# test job: [fd94857a934cbe613353810a024c84d54826ead3] https://lava.sirena.org.uk/scheduler/job/2082978
# test job: [7c63b5a8ed972a2c8c03d984f6a43349007cea93] https://lava.sirena.org.uk/scheduler/job/2082455
# test job: [675f41b8d1675d9a51a6dcb978ff76b95cbb4b92] https://lava.sirena.org.uk/scheduler/job/2082528
# test job: [8ff3dcb0e8a8bf6c41f23ed4aa62d066d3948a10] https://lava.sirena.org.uk/scheduler/job/2082638
# test job: [5a0438622b49df1e723960ac9c0bf75e04370fdc] https://lava.sirena.org.uk/scheduler/job/2082676
# test job: [65d03e84d8b8e42cf4636fcabb81c9b130cec710] https://lava.sirena.org.uk/scheduler/job/2082553
# test job: [893e2fd509e968cc1d76caadee0f5d2f2c72f137] https://lava.sirena.org.uk/scheduler/job/2082444
# test job: [4e00135b2dd1d7924a58bffa551b6ceb3bd836f2] https://lava.sirena.org.uk/scheduler/job/2082445
# test job: [123cd174a3782307787268adf45f22de4d290128] https://lava.sirena.org.uk/scheduler/job/2078902
# test job: [1d562ba0aa7df81335bf96c02be77efe8d5bab87] https://lava.sirena.org.uk/scheduler/job/2078312
# test job: [4d6e2211aeb932e096f673c88475016b1cc0f8ab] https://lava.sirena.org.uk/scheduler/job/2077977
# test job: [23523e9398efeee6a5741cf97052babf6fb2a808] https://lava.sirena.org.uk/scheduler/job/2075586
# test job: [2b0d5d9b39489b9d8171896e8e24d8632c3dc807] https://lava.sirena.org.uk/scheduler/job/2075590
# test job: [660d946ce6fd64678b4ed6b083dfc24c0f4d5a69] https://lava.sirena.org.uk/scheduler/job/2074678
# test job: [32172cf3cb543a04c41a1677c97a38e60cad05b6] https://lava.sirena.org.uk/scheduler/job/2074987
# test job: [f78d96c289da743d517f700d3fe7c6c43df667b5] https://lava.sirena.org.uk/scheduler/job/2074656
# test job: [69f3474a01e9867dd99fc4b703fa834ea1835c7d] https://lava.sirena.org.uk/scheduler/job/2074980
# test job: [38ecd1384079d5f7b6942b68496ee7e85aa9a883] https://lava.sirena.org.uk/scheduler/job/2074664
# test job: [b3a5302484033331af37569f7277d00131694b57] https://lava.sirena.org.uk/scheduler/job/2074473
# test job: [1d5bf23aa1e8aea43beca78dc5d69c68955e8871] https://lava.sirena.org.uk/scheduler/job/2074705
# test job: [873bc94689d832878befbcadc10b6ad5bb4e0027] https://lava.sirena.org.uk/scheduler/job/2074693
# test job: [dfbb57c2767dc128f6360c3e41136bff8fa6f924] https://lava.sirena.org.uk/scheduler/job/2074651
# test job: [4acbfcf11cbe6c46c42091b49875c7002b0bff3d] https://lava.sirena.org.uk/scheduler/job/2069096
# test job: [772ada50282b0c80343c8989147db816961f571d] https://lava.sirena.org.uk/scheduler/job/2069076
# test job: [e32c402533cc68abe20fcf3246b9ff53e1f96021] https://lava.sirena.org.uk/scheduler/job/2060042
# test job: [6985defd1d832f1dd9d1977a6a2cc2cef7632704] https://lava.sirena.org.uk/scheduler/job/2059055
# test job: [fb1ebb10468da414d57153ddebaab29c38ef1a78] https://lava.sirena.org.uk/scheduler/job/2059726
# test job: [2089f086303b773e181567fd8d5df3038bd85937] https://lava.sirena.org.uk/scheduler/job/2058101
# test job: [4db4ce15706d6423cc4cac4b05114b0469507bad] https://lava.sirena.org.uk/scheduler/job/2055719
# test job: [4e92abd0a11b91af3742197a9ca962c3c00d0948] https://lava.sirena.org.uk/scheduler/job/2055704
# test job: [b6f4bd64f453183954184ffbc2b89d73ed8fb135] https://lava.sirena.org.uk/scheduler/job/2058033
# test job: [6951be397ca8b8b167c9f99b5a11c541148c38cb] https://lava.sirena.org.uk/scheduler/job/2055707
# test job: [4a58f60df578bb55a1d0adca0c356e03b2818de1] https://lava.sirena.org.uk/scheduler/job/2057851
# test job: [bd352547df647be8a1e6c9d4ca2b54b459f3abc1] https://lava.sirena.org.uk/scheduler/job/2054669
# test job: [1797d254f5c4b46b295527a635af7321a3fe1318] https://lava.sirena.org.uk/scheduler/job/2054530
# test job: [106d7641e55a472e7523c1f525c77fb6d420064d] https://lava.sirena.org.uk/scheduler/job/2054630
# test job: [873a46141460d209bb62eaa0dc9e7b67bff924a6] https://lava.sirena.org.uk/scheduler/job/2054391
# test job: [47d98d432f19030faf06d4b2ca7fc7493057cae1] https://lava.sirena.org.uk/scheduler/job/2054441
# test job: [6b398c1d3da7a673b13b1857f9fff4c15ee20cef] https://lava.sirena.org.uk/scheduler/job/2054408
# test job: [56931105074fe7e5fc9d54e3163df3b95075643c] https://lava.sirena.org.uk/scheduler/job/2054397
# test job: [abc9a349b87ac0fd3ba8787ca00971b59c2e1257] https://lava.sirena.org.uk/scheduler/job/2054452
# test job: [1044821176a64a8dc7f861e878506cde34c8b658] https://lava.sirena.org.uk/scheduler/job/2053054
# test job: [6d34b66fb726a613b98c936adee70d42aa5e4aa7] https://lava.sirena.org.uk/scheduler/job/2053409
# test job: [55d03b5b5bdd04daf9a35ce49db18d8bb488dffb] https://lava.sirena.org.uk/scheduler/job/2053804
# test job: [1b0f3f9ee41ee2bdd206667f85ea2aa36dfe6e69] https://lava.sirena.org.uk/scheduler/job/2053404
# test job: [6bd1ad97eb790570c167d4de4ca59fbc9c33722a] https://lava.sirena.org.uk/scheduler/job/2053405
# test job: [7a381e373a4243926a41b8e6ebbdeb90fe9afda3] https://lava.sirena.org.uk/scheduler/job/2053058
# test job: [81c53b52de21b8d5a3de55ebd06b6bf188bf7efd] https://lava.sirena.org.uk/scheduler/job/2050010
# test job: [bcb3f6f5286b96ff0a7106d2ca7208744b20b37e] https://lava.sirena.org.uk/scheduler/job/2049534
# test job: [3c36965df80801344850388592e95033eceea05b] https://lava.sirena.org.uk/scheduler/job/2049387
# test job: [6e00112d31c86029ad0440f8c29ee0d131984cda] https://lava.sirena.org.uk/scheduler/job/2049772
# test job: [bfbb12dfa144d45575bcfe139a71360b3ce80237] https://lava.sirena.org.uk/scheduler/job/2049754
# test job: [b828059f86f67729aae3934650ae9e44a59ff9d8] https://lava.sirena.org.uk/scheduler/job/2049568
# test job: [89cf2223ee7bd83d45c6bef3ac52bedd018f77dd] https://lava.sirena.org.uk/scheduler/job/2050030
# test job: [655079ac8a7721ac215a0596e3f33b740e01144a] https://lava.sirena.org.uk/scheduler/job/2049869
# test job: [a354f030dbce17ab3cf299660a38b162d97f7151] https://lava.sirena.org.uk/scheduler/job/2050072
# test job: [4b1edbb028fb95b546893068b321a983b1ef50bc] https://lava.sirena.org.uk/scheduler/job/2049818
# test job: [680c683c8f49455bcfa1604eac4e508ba96ccbfa] https://lava.sirena.org.uk/scheduler/job/2049613
# test job: [b6a1d7243ec3a3fe1a3c2969fcd32c802fb5d2a9] https://lava.sirena.org.uk/scheduler/job/2049994
# test job: [950a4e5788fc7dc6e8e93614a7d4d0449c39fb8d] https://lava.sirena.org.uk/scheduler/job/2050045
# test job: [896f1e4e106449ac2b39f0ca62b6afaed540f758] https://lava.sirena.org.uk/scheduler/job/2049729
# test job: [74cc4f3ea4e99262ba0d619c6a4ee33e2cd47f65] https://lava.sirena.org.uk/scheduler/job/2049795
# test job: [a87a7b3530728fb7477a74c27fc27e060bf5684f] https://lava.sirena.org.uk/scheduler/job/2049419
# test job: [d0f9f5b7a3356b43f78b37b9cc3671ecc7469356] https://lava.sirena.org.uk/scheduler/job/2049295
# test job: [f678ea19e8f60b05b8d3a5bcda9a238d5355bd50] https://lava.sirena.org.uk/scheduler/job/2049882
# test job: [de8e95773c48bad9d7339ccb376ad22d93932468] https://lava.sirena.org.uk/scheduler/job/2049837
# test job: [88a5f8e628ef745bb94bdf2fa019d9f8e069ccea] https://lava.sirena.org.uk/scheduler/job/2050064
# test job: [e386c2cf3df2b62a6cc78d2f7984102b64753bee] https://lava.sirena.org.uk/scheduler/job/2049698
# test job: [44432315444cbbedadbb9d808c9fc8c8b21cb4bf] https://lava.sirena.org.uk/scheduler/job/2049908
# test job: [55094e55ae36c7566e29ae0473d205dbc9d2f4a8] https://lava.sirena.org.uk/scheduler/job/2049639
# test job: [40a7c5db9020079547358f486ef12d57c1a7aa1f] https://lava.sirena.org.uk/scheduler/job/2049275
# test job: [8b6faa7fddf0ae69c5f1a9315a64edee6f022037] https://lava.sirena.org.uk/scheduler/job/2048807
# test job: [aa897ffc396b48cc39eee133b6b43175d0df9eb5] https://lava.sirena.org.uk/scheduler/job/2048500
# test job: [2f538ef9f6f7c3d700c68536f21447dfc598f8c8] https://lava.sirena.org.uk/scheduler/job/2048486
# test job: [380fd29d57abe6679d87ec56babe65ddc5873a37] https://lava.sirena.org.uk/scheduler/job/2044585
# test job: [af9c8092d84244ca54ffb590435735f788e7a170] https://lava.sirena.org.uk/scheduler/job/2043582
# test job: [b4e002d8a7cee3b1d70efad0e222567f92a73000] https://lava.sirena.org.uk/scheduler/job/2044525
# test job: [c4e68959af66df525d71db619ffe44af9178bb22] https://lava.sirena.org.uk/scheduler/job/2043962
# test job: [ecaba8b7990d8c6d8ba097cd4499b3b92d9df6ea] https://lava.sirena.org.uk/scheduler/job/2043980
# test job: [9de2057bbdfb58f4d9bb1476135317cd3fe6aa52] https://lava.sirena.org.uk/scheduler/job/2043501
# test job: [3b071bdd26849172101081573a18022af108fb21] https://lava.sirena.org.uk/scheduler/job/2043945
# test job: [6022eacdda8b0b06a2e1d4122e5268099b62ff5d] https://lava.sirena.org.uk/scheduler/job/2044503
# test job: [84194c66aaf78fed150edb217b9f341518b1cba2] https://lava.sirena.org.uk/scheduler/job/2038174
# test job: [1a88479436a52452429f6d50ec5bd0a546406485] https://lava.sirena.org.uk/scheduler/job/2038219
# test job: [2ecc8c089802e033d2e5204d21a9f467e2517df9] https://lava.sirena.org.uk/scheduler/job/2038177
# test job: [252abf2d07d33b1c70a59ba1c9395ba42bbd793e] https://lava.sirena.org.uk/scheduler/job/2038171
# test job: [a2d4691b3fec6a2360e4ec953d06819ea055c3e7] https://lava.sirena.org.uk/scheduler/job/2038166
# test job: [164d1037c4f86828b77a15c9071f051c5acddc93] https://lava.sirena.org.uk/scheduler/job/2038302
# test job: [f47e782c45f8f0c3d7b84edd7e94bfce9eb00b64] https://lava.sirena.org.uk/scheduler/job/2038376
# test job: [947eaef21577f52db046992c887ed9698f9d883c] https://lava.sirena.org.uk/scheduler/job/2038167
# test job: [0144a2b29d95af8523c308116de65d398d6e935b] https://lava.sirena.org.uk/scheduler/job/2038169
# test job: [44c1a444b030647803d900e60f5a8af31a782f0e] https://lava.sirena.org.uk/scheduler/job/2028969
# test job: [27fef3048fe95934f6f2f87341eb33ef6581a075] https://lava.sirena.org.uk/scheduler/job/2028964
# test job: [94a3a95f03154d8d4c6206950a7f6ef9a30baec6] https://lava.sirena.org.uk/scheduler/job/2028972
# test job: [ed5d499b5c9cc11dd3edae1a7a55db7dfa4f1bdc] https://lava.sirena.org.uk/scheduler/job/2028953
# test job: [bda6f8749c8e0b10f083dc7a1edf169f349fb776] https://lava.sirena.org.uk/scheduler/job/2028960
# test job: [ecd0de438c1f0ee86cf8f6d5047965a2a181444b] https://lava.sirena.org.uk/scheduler/job/2026053
# test job: [3d8096fd378b0b4dae356e6f037d03da83f559d2] https://lava.sirena.org.uk/scheduler/job/2026661
# test job: [f1dfbc1b5cf8650ae9a0d543e5f5335fc0f478ce] https://lava.sirena.org.uk/scheduler/job/2025457
# test job: [db66398fb03ee3599ec7387ceb5121b95e648eb3] https://lava.sirena.org.uk/scheduler/job/2026646
# test job: [ef37146360385282b5f6a5b4bf695db30d609887] https://lava.sirena.org.uk/scheduler/job/2025815
# test job: [6ef8e042cdcaabe3e3c68592ba8bfbaee2fa10a3] https://lava.sirena.org.uk/scheduler/job/2025807
# test job: [e73b743bfe8a6ff4e05b5657d3f7586a17ac3ba0] https://lava.sirena.org.uk/scheduler/job/2026375
# test job: [e2ff7154813a3834692703852604b2099ecf043a] https://lava.sirena.org.uk/scheduler/job/2023152
# test job: [5fa671232f703a404caa05c581411ea858c4cf16] https://lava.sirena.org.uk/scheduler/job/2021242
# test job: [28039efa4d8e8bbf98b066133a906bd4e307d496] https://lava.sirena.org.uk/scheduler/job/2019967
# test job: [541aecd34383a85eaf7c8556779466e394554fed] https://lava.sirena.org.uk/scheduler/job/2022785
# test job: [20bcda681f8597e86070a4b3b12d1e4f541865d3] https://lava.sirena.org.uk/scheduler/job/2022837
# test job: [1e5351ba60f5355809f30c61bbd27e97611d2be9] https://lava.sirena.org.uk/scheduler/job/2022784
# test job: [cf6bf51b53252284bafc7377a4d8dbf10f048b4d] https://lava.sirena.org.uk/scheduler/job/2022830
# test job: [9c7f7262bc1affb9b9acd2ec2fb1f6314d5d474c] https://lava.sirena.org.uk/scheduler/job/2019964
# test job: [d02460317ed9c95aa2c5f6ff1c70e22e1857d95d] https://lava.sirena.org.uk/scheduler/job/2021243
# test job: [2b4d53eb5cf32c5b7b7616f23f08471fd8b3708e] https://lava.sirena.org.uk/scheduler/job/2022792
# test job: [e062bdfdd6adbb2dee7751d054c1d8df63ddb8b8] https://lava.sirena.org.uk/scheduler/job/2019974
# test job: [8fdb030fe283c84fd8d378c97ad0f32d6cdec6ce] https://lava.sirena.org.uk/scheduler/job/2021247
# test job: [3cd523ba270665861647304aa30500f238ebf26e] https://lava.sirena.org.uk/scheduler/job/2022788
# test job: [bcba17279327c6e85dee6a97014dc642e2dc93cc] https://lava.sirena.org.uk/scheduler/job/2021197
# test job: [eda4a53f8fe021328c468175a02da8d7ad155494] https://lava.sirena.org.uk/scheduler/job/2018191
# test job: [ac479277c24c859553a6db340aa1518d320bc5e2] https://lava.sirena.org.uk/scheduler/job/2016617
# test job: [a4619aadb308db0418b56b237019dc5ab928aa0b] https://lava.sirena.org.uk/scheduler/job/2016585
# test job: [060028c7fcdd7bc1c6ed61fb25380c0d6c36475b] https://lava.sirena.org.uk/scheduler/job/2016607
# test job: [6e8146909eae3a42807c750ad84e3fa29c192822] https://lava.sirena.org.uk/scheduler/job/2016579
# test job: [250eddd69da21999007fd09e17a7c43b92cc6220] https://lava.sirena.org.uk/scheduler/job/2016574
# test job: [f034c16a4663eaf3198dc18b201ba50533fb5b81] https://lava.sirena.org.uk/scheduler/job/2015249
# test job: [7a78e387654f20e798ceab5bae9c1f5557416431] https://lava.sirena.org.uk/scheduler/job/2015347
# test job: [66fecfa91deb536a12ddf3d878a99590d7900277] https://lava.sirena.org.uk/scheduler/job/2015257
# test job: [dc74a00c7661a14a672ea7660caca5c4aa661a79] https://lava.sirena.org.uk/scheduler/job/2015860
# test job: [67e4b0dfcc6702a31fbb6a3015c0dc867e295eb4] https://lava.sirena.org.uk/scheduler/job/2011121
# test job: [4a5ac6cd05a7e54f1585d7779464d6ed6272c134] https://lava.sirena.org.uk/scheduler/job/2011125
# test job: [6ddcd78aa7f85e1d94ab7f90c72d1ad0c0c7b6ea] https://lava.sirena.org.uk/scheduler/job/2011115
# test job: [e92e25f777483b7cc3e170214cc84337d7a415cf] https://lava.sirena.org.uk/scheduler/job/2010368
# test job: [12aa3160c10a3179c73c4f99a2d5aec0fd907d0c] https://lava.sirena.org.uk/scheduler/job/2010437
# test job: [ef042df96d0e1089764f39ede61bc8f140a4be00] https://lava.sirena.org.uk/scheduler/job/2010255
# test job: [0723affa1bee50c3bd7ca00e00dee07fcef224b8] https://lava.sirena.org.uk/scheduler/job/2010205
# test job: [aeaf27ec6571527e750eed84bb3865a0664ae316] https://lava.sirena.org.uk/scheduler/job/2010684
# test job: [4c33cef58965eb655a0ac8e243aa323581ec025f] https://lava.sirena.org.uk/scheduler/job/2009387
# test job: [390c05f47d0749b24db65586482308c5fd680fe5] https://lava.sirena.org.uk/scheduler/job/2010200
# test job: [c7b6c6b60594fd1efe35c61bc6a2176b25263ccc] https://lava.sirena.org.uk/scheduler/job/2010051
# test job: [8d557cc4867f2008f440c54b4423464301a1ef4b] https://lava.sirena.org.uk/scheduler/job/2010531
# test job: [dfe7c3401ed3d3bd8e61be8d6d452896513eb52e] https://lava.sirena.org.uk/scheduler/job/2010310
# test job: [7b6be935e7eff06025e18cea4c6620194450abe2] https://lava.sirena.org.uk/scheduler/job/2010645
# test job: [cdd27fa3298ad2f39788804f7d09ab31af2b416c] https://lava.sirena.org.uk/scheduler/job/2009477
# test job: [64670a6c062c4d183c366d46e71ee76395af6a15] https://lava.sirena.org.uk/scheduler/job/2009588
# test job: [86df0030b71d7172317d957df17524a7fd6232d4] https://lava.sirena.org.uk/scheduler/job/2009380
# test job: [1afc05996299b4546e8be9b13c89f78e19912c7d] https://lava.sirena.org.uk/scheduler/job/2009796
# test job: [56bbda23d4bece7ce998666118a068e4f71d59fb] https://lava.sirena.org.uk/scheduler/job/2010283
# test job: [191a27faf53edf9e9101901e402bfee49c44073c] https://lava.sirena.org.uk/scheduler/job/2009722
# test job: [7159816707dc7040fe3ac4fa3d7ac3d173bd772a] https://lava.sirena.org.uk/scheduler/job/2010587
# test job: [f7097161e94cd39df7a8848ad0de5f394124ed69] https://lava.sirena.org.uk/scheduler/job/2009475
# test job: [01313661b248c5ba586acae09bff57077dbec0a5] https://lava.sirena.org.uk/scheduler/job/2008723
# test job: [907364ea3db47530751add6d2d62122ca17329cb] https://lava.sirena.org.uk/scheduler/job/2010058
# test job: [013a3a66f25af3fb614f45df43983657514944c4] https://lava.sirena.org.uk/scheduler/job/2010612
# test job: [715159314dfafee66e6deb50b4e3431539a919d8] https://lava.sirena.org.uk/scheduler/job/2010092
# test job: [46a3df50b0cab466099515f2375b01c5be4fb95c] https://lava.sirena.org.uk/scheduler/job/2009528
# test job: [daab108504be73182c16a72b9cfe47ac3b1928ca] https://lava.sirena.org.uk/scheduler/job/2010409
# test job: [0a5e9769d088bd1d8faf01207210911b9341b62c] https://lava.sirena.org.uk/scheduler/job/2010556
# test job: [cf6290eebe3cc4eb677d11aa061d10cb1df12ab9] https://lava.sirena.org.uk/scheduler/job/2009786
# test job: [c4d096c3ca425562192a3626c30e82651d0f2c1c] https://lava.sirena.org.uk/scheduler/job/2010075
# test job: [4795375d8aa072e9aacb0b278e6203c6ca41816a] https://lava.sirena.org.uk/scheduler/job/2009512
# test job: [71f7990a34cdb11f82d3cbbcddaca77a55635466] https://lava.sirena.org.uk/scheduler/job/2010339
# test job: [2b62e66626f05e277c8fdeb50d4c1e0cbab2fe0e] https://lava.sirena.org.uk/scheduler/job/2009813
# test job: [ef24466ee1912997c2bd526194006bbca424c24f] https://lava.sirena.org.uk/scheduler/job/2009770
# test job: [959400caf51eb31f95d1ab754a285b5546ebd3e4] https://lava.sirena.org.uk/scheduler/job/2009747
# test job: [ddbcd2f396116581ad035fb76a99fc2ed865a85f] https://lava.sirena.org.uk/scheduler/job/2007395
# test job: [77a58ba7c64ccca20616aa03599766ccb0d1a330] https://lava.sirena.org.uk/scheduler/job/2007263
# test job: [9957614d2b79578b6f9a2512bfbb2bc7bbdc43ce] https://lava.sirena.org.uk/scheduler/job/2007546
# test job: [3941abb26ff327e53e1e8b873cab3ed3d5103eab] https://lava.sirena.org.uk/scheduler/job/2007701
# test job: [e7434adf0c53a84d548226304cdb41c8818da1cb] https://lava.sirena.org.uk/scheduler/job/2007665
# test job: [c63b2315b9cc6b705205c73dcf4591cfeb9a25ae] https://lava.sirena.org.uk/scheduler/job/2007521
# test job: [ca4d49f8a21c37be7e5aed80100ca6b13ac3cf9d] https://lava.sirena.org.uk/scheduler/job/2007281
# test job: [e973dfe9259095fb509ab12658c68d46f0e439d7] https://lava.sirena.org.uk/scheduler/job/2007676
# test job: [d29479abaded34b2b1dab2e17efe96a65eba3d61] https://lava.sirena.org.uk/scheduler/job/2008331
# test job: [bb65cb96f64e9b4ea2bbd41e4591f3da91414fdb] https://lava.sirena.org.uk/scheduler/job/2007495
# test job: [38ff69586bbb3a823dd501972e17075374b685a1] https://lava.sirena.org.uk/scheduler/job/2007611
# test job: [e7ab858390f24a23ba0827066382ba0e6a8e4379] https://lava.sirena.org.uk/scheduler/job/2007264
# test job: [ee4407e1288ab85be16bacc45195b8bb23d44760] https://lava.sirena.org.uk/scheduler/job/2007622
# test job: [c37c3e5e390dcd52cbe6178ac53f5a6131ef6f8c] https://lava.sirena.org.uk/scheduler/job/2007578
# test job: [1e0722a77b4e263854a812c9c106ddef8fd56720] https://lava.sirena.org.uk/scheduler/job/2007455
# test job: [f97ebfda8da28a77a0218a448829451ba7e30d5d] https://lava.sirena.org.uk/scheduler/job/2007368
# test job: [c17fa4cbc546c431ccf13e9354d5d9c1cd247b7c] https://lava.sirena.org.uk/scheduler/job/1999963
# test job: [2914f6ea90772ce4a8311a6d5b3ab94e3cd31b12] https://lava.sirena.org.uk/scheduler/job/1999961
# test job: [f76dbe127f1b5910e37dfe307d2de5c13d61ed89] https://lava.sirena.org.uk/scheduler/job/1995515
# test job: [310bf433c01f78e0756fd5056a43118a2f77318c] https://lava.sirena.org.uk/scheduler/job/1995860
# test job: [fd5ef3d69f8975bad16c437a337b5cb04c8217a2] https://lava.sirena.org.uk/scheduler/job/1996084
# test job: [edf5c8920240dcafe830908549a0edee4ba3b4b0] https://lava.sirena.org.uk/scheduler/job/1995780
# test job: [d054cc3a2ccfb19484f3b54d69b6e416832dc8f4] https://lava.sirena.org.uk/scheduler/job/1995546
# test job: [2528c15f314ece50218d1273654f630d74109583] https://lava.sirena.org.uk/scheduler/job/1997582
# test job: [7c69694cec869e3bf7c810fd94f860253aeb8053] https://lava.sirena.org.uk/scheduler/job/1996087
# test job: [108fba26698a75bb8e1acc21553a6860313aebd2] https://lava.sirena.org.uk/scheduler/job/1992065
# test job: [638bae3fb225a708dc67db613af62f6d14c4eff4] https://lava.sirena.org.uk/scheduler/job/1991739
# test job: [8c465b1669bfeaaf0ebd504b96d689e2a2810874] https://lava.sirena.org.uk/scheduler/job/1985678
# test job: [2880c42a0de68e9eb979e5331fbd5b2c79e093ad] https://lava.sirena.org.uk/scheduler/job/1985659
# test job: [e0fb9464295bca2aa92835d02147d3265a83689a] https://lava.sirena.org.uk/scheduler/job/1985668
# test job: [5cd5f8fc29fa1b6d7c0a8f2b0a95b896ecadfa42] https://lava.sirena.org.uk/scheduler/job/1984949
# test job: [ecba655bf54a661ffe078856cd8dbc898270e4b5] https://lava.sirena.org.uk/scheduler/job/1984960
# test job: [6a4f29bc66294d44d61a294e5bdc623eae74587b] https://lava.sirena.org.uk/scheduler/job/1984953
# test job: [900da53226121c1e710ca95857806a136ab281a2] https://lava.sirena.org.uk/scheduler/job/1984976
# test job: [7e1906643a7374529af74b013bba35e4fa4e6ffc] https://lava.sirena.org.uk/scheduler/job/1978561
# test job: [b573898b82abf8d2736c90b14d76f65f9bc8d1f1] https://lava.sirena.org.uk/scheduler/job/1974408
# test job: [c5c6d972d6450350ecf6b1c6dc7ec7e1462bd221] https://lava.sirena.org.uk/scheduler/job/1975376
# test job: [9a1e055bb51d17a44e6e46e2af922dd0d9a1fe2a] https://lava.sirena.org.uk/scheduler/job/1975281
# test job: [94cd5e54fe56165a2ac7a1ac7b3599a14b9497c0] https://lava.sirena.org.uk/scheduler/job/1975116
# test job: [ca6731ad2a4fd1e5fc549d53d4f99445fd552f97] https://lava.sirena.org.uk/scheduler/job/1975322
# test job: [c387ff80f77e6832952fb92d511bfcfda6766a54] https://lava.sirena.org.uk/scheduler/job/1975077
# test job: [64d1d87d185e0cccdaff573e16af074193045167] https://lava.sirena.org.uk/scheduler/job/1974328
# test job: [f72e15146ea024f849143a1d4c65f351cdd12488] https://lava.sirena.org.uk/scheduler/job/1975013
# test job: [6c8f8e79a3f796106b6de386b2055441b8256efb] https://lava.sirena.org.uk/scheduler/job/1974576
# test job: [37d9425fc9f19eb92abdcf96189e74e163b94619] https://lava.sirena.org.uk/scheduler/job/1973958
# test job: [0d58897657a9a31f4817a9596a200815f8c8e602] https://lava.sirena.org.uk/scheduler/job/1974721
# test job: [308eee447306e885254bca4bc23b9f90534feb1a] https://lava.sirena.org.uk/scheduler/job/1974660
# test job: [9846cb5a9d06c476c503abfa840251ad3c06414e] https://lava.sirena.org.uk/scheduler/job/1974426
# test job: [161e3c824e63eec6515f2c7d36fba391647ccf89] https://lava.sirena.org.uk/scheduler/job/1974932
# test job: [c293d36d1037fc63430f9c4ec7e3fd1e42a50c0e] https://lava.sirena.org.uk/scheduler/job/1974754
# test job: [6e3e296c564399f17e8c99623f8ba608b8fe1b7d] https://lava.sirena.org.uk/scheduler/job/1973858
# test job: [3cba794c8fec8e1d5da6c3c91fafdf490de4b24f] https://lava.sirena.org.uk/scheduler/job/1973832
# test job: [a233cb3fe736647ab2a43aeb3ef1fb9a2a0744cf] https://lava.sirena.org.uk/scheduler/job/1974084
# test job: [9bce11a336f0d5694340f85bdca81179a02eaa81] https://lava.sirena.org.uk/scheduler/job/1975458
# test job: [5190935a1b93940c28cc4f484e17662246648f66] https://lava.sirena.org.uk/scheduler/job/1975165
# test job: [d742ebcfe524dc54023f7c520d2ed2e4b7203c19] https://lava.sirena.org.uk/scheduler/job/1975412
# test job: [74fdbce5fe88f9204634e3923c86a84c3a505ecd] https://lava.sirena.org.uk/scheduler/job/1974708
# test job: [7e1393622591631673c419197dda2d5ff14aacc1] https://lava.sirena.org.uk/scheduler/job/1973883
# test job: [c59fe12fa2acb1613eb77fe2d690ab76107bd4a4] https://lava.sirena.org.uk/scheduler/job/1975400
# test job: [6bc24a7d35ceaffa0f7551b7bd13b495a27e203c] https://lava.sirena.org.uk/scheduler/job/1975352
# test job: [3314ee8db5411e1f9adc0172ebe425c8a8066cab] https://lava.sirena.org.uk/scheduler/job/1974828
# test job: [626cf62777735ca51a8d88d8dc2e234b56d4f3a7] https://lava.sirena.org.uk/scheduler/job/1974803
# test job: [d98fdbdf8bc654242d94616a7bfc233c67b43866] https://lava.sirena.org.uk/scheduler/job/1974053
# test job: [95972dd17b189647e96605f4edf2177a8876eb4e] https://lava.sirena.org.uk/scheduler/job/1975158
# test job: [b5e0fc69e42b23e86b32f4810a41ff6b842d6426] https://lava.sirena.org.uk/scheduler/job/1974872
# test job: [c8e43faeb4a608f45e362c55a1687f3249479a04] https://lava.sirena.org.uk/scheduler/job/1974463
# test job: [7a0fefe28cc5a127cb63aafb5fe85852d198ee7a] https://lava.sirena.org.uk/scheduler/job/1975094
# test job: [01511c18f713b7579133f451332a1ccbb634e389] https://lava.sirena.org.uk/scheduler/job/1974171
# test job: [b56580c7ba8ae7ef585a7deefbb137212be6f1f0] https://lava.sirena.org.uk/scheduler/job/1974070
# test job: [844db7d7e7200e303fa3974e8dae5350646813d1] https://lava.sirena.org.uk/scheduler/job/1975384
# test job: [73f64a4803f1e092ed83f4e96225f35c42fe4be1] https://lava.sirena.org.uk/scheduler/job/1974971
# test job: [18096d761f6be5a981f3480f7907859f3a431a0b] https://lava.sirena.org.uk/scheduler/job/1974798
# test job: [96b67f9204fb63168076142777e3cfc2dd236d8c] https://lava.sirena.org.uk/scheduler/job/1975014
# test job: [2d54738a39b49fc46b4b22472388d74fc698b87f] https://lava.sirena.org.uk/scheduler/job/1975484
# test job: [bc78514e84035d1a596eb558ce9da9dd5269a85e] https://lava.sirena.org.uk/scheduler/job/1974522
# test job: [7e4cdef1ef8f8a3baa6806d42f6ee2f64d75cc17] https://lava.sirena.org.uk/scheduler/job/1975358
# test job: [f793a6e08f5487b4f1e1207cc4952adc753c1f4b] https://lava.sirena.org.uk/scheduler/job/1974577
# test job: [8b412d3233c69a8545ca771c2b4e25774a4a90b0] https://lava.sirena.org.uk/scheduler/job/1974458
# test job: [386db878776142f262c5997424832090ebe5c196] https://lava.sirena.org.uk/scheduler/job/1974483
# test job: [661584c295d75ff72fae3569e4f0439325b9835e] https://lava.sirena.org.uk/scheduler/job/1974659
# test job: [3576a8d214386e314e83c36c13eb098433896fbc] https://lava.sirena.org.uk/scheduler/job/1974488
# test job: [9e851421266baaedf86f8a31b62aeda0c72a88d8] https://lava.sirena.org.uk/scheduler/job/1974842
# test job: [a7aa34438d6367a34555e8cda99ba379ef52938b] https://lava.sirena.org.uk/scheduler/job/1974132
# test job: [1a1a66c2cee604dedbd4de3ca7d8e3c38d7de05c] https://lava.sirena.org.uk/scheduler/job/1975316
# test job: [504219ea359c4545176f76bab77e0ba38a8629d7] https://lava.sirena.org.uk/scheduler/job/1974477
# test job: [ba93cda8cc9eb426c801aa8dca8a0e3874de958f] https://lava.sirena.org.uk/scheduler/job/1974428
# test job: [ecc35b2977af71d5fceef68ca723409cd9592bc3] https://lava.sirena.org.uk/scheduler/job/1973990
# test job: [a31e9992873ebf3ff66699ee13fdf847891746c6] https://lava.sirena.org.uk/scheduler/job/1975430
# test job: [601e7b30670a25b4cadfb05bded9345126d82b58] https://lava.sirena.org.uk/scheduler/job/1975291
# test job: [db1bcf18fe320ff4988a196f969ca739f9c3da95] https://lava.sirena.org.uk/scheduler/job/1974386
# test job: [b9b68f3a262bce63d4d363f08b345481e4f9d23b] https://lava.sirena.org.uk/scheduler/job/1974611
# test job: [024530ea1a3b6e8c64baa76f046345800b99a1bd] https://lava.sirena.org.uk/scheduler/job/1975050
# test job: [801955fd9248ea5659189b0464e9e4ff0952a11a] https://lava.sirena.org.uk/scheduler/job/1975435
# test job: [ae7ac6bbf0fa4475cd169b9ad2e7f751a7cb73f3] https://lava.sirena.org.uk/scheduler/job/1974743
# test job: [a6531a0414ed50cbbe6244e82fec4d432a207842] https://lava.sirena.org.uk/scheduler/job/1974517
# test job: [fcc6ae5198bc13860053ea6e1fb7ca85415c77d0] https://lava.sirena.org.uk/scheduler/job/1974608
# test job: [41a6e1032c799184586d3c5ecb594cc05b844dc7] https://lava.sirena.org.uk/scheduler/job/1974959
# test job: [88b0ee610790877bd9e3efcf8f29efd53eac894e] https://lava.sirena.org.uk/scheduler/job/1974938
# test job: [ccf4bead907d8be7212e69adeca64a078712d8a6] https://lava.sirena.org.uk/scheduler/job/1974525
# test job: [c7ade18aad3e63e732a93efb5580a4b2e8563d0c] https://lava.sirena.org.uk/scheduler/job/1974035
# test job: [f4ae2720f353fe58b02f49cc935933dd9ccc8ce4] https://lava.sirena.org.uk/scheduler/job/1974870
# test job: [be9342c57e28d8e42638e4f234d8727da9ee609e] https://lava.sirena.org.uk/scheduler/job/1973983
# test job: [6950709b6658fadc7db860a4ec33de8c5226c728] https://lava.sirena.org.uk/scheduler/job/1974194
# test job: [2c688df0fd19d02146e031502c701c184b86cd0e] https://lava.sirena.org.uk/scheduler/job/1974412
# test job: [e7ed084f4cd49806b9ff91d800c6422f96f6f5ea] https://lava.sirena.org.uk/scheduler/job/1974413
# test job: [a5e7aaee483c7022e96181ce899944b8c7b1f1e0] https://lava.sirena.org.uk/scheduler/job/1974392
# test job: [11c13a65e82afb830fa0f7df2fcf3ed415e6fa1b] https://lava.sirena.org.uk/scheduler/job/1974195
# test job: [52a87e0b5ec1f7316293773f7859996d123b3ea7] https://lava.sirena.org.uk/scheduler/job/1974117
# test job: [3a071bb34cea8e97e403c7946a1fc03de164f70c] https://lava.sirena.org.uk/scheduler/job/1973863
# test job: [ddd1705b62c6920b77d8cfcccc695e99dde25ad7] https://lava.sirena.org.uk/scheduler/job/1974015
# test job: [ccbfc923fe935b4f1e3b1e51df828e07473faff9] https://lava.sirena.org.uk/scheduler/job/1973980
# test job: [cf9d07738fd94e1c3cf0c3ffb61f4d2d1e4d0c57] https://lava.sirena.org.uk/scheduler/job/1973397
# test job: [afcf45b7555fcc8e2fc177e8d4e7a2d9d79b13eb] https://lava.sirena.org.uk/scheduler/job/1973314
# test job: [776b160f3234f85899eb3c8d454c37176484facb] https://lava.sirena.org.uk/scheduler/job/1974460
# test job: [e621116150c5983dde75195f0eb2ea22baf4e87c] https://lava.sirena.org.uk/scheduler/job/1974227
# test job: [d7e1399884a65e868f7b121b8ca5d613cfac6538] https://lava.sirena.org.uk/scheduler/job/1973981
# test job: [92ba8c4c78810b991b697af1087ba31226aa5871] https://lava.sirena.org.uk/scheduler/job/1973724
# test job: [211367ef1d4031a29fa8e07f430155d68802e8ec] https://lava.sirena.org.uk/scheduler/job/1973992
# test job: [6dcc15ed9e1b0ce6b1f3f53af33e2afaf0a53fa0] https://lava.sirena.org.uk/scheduler/job/1974309
# test job: [73e86fbf251708bb2766fff746eea0ebb6fe0747] https://lava.sirena.org.uk/scheduler/job/1974140
# test job: [fe0b3f564f9b1ecd74180c296129486d840bc3f9] https://lava.sirena.org.uk/scheduler/job/1974016
# test job: [d9f08106c16a32cb9514986351754d36f3b3c7df] https://lava.sirena.org.uk/scheduler/job/1974073
# test job: [c4ec2c9d7329b509e10c8980e4077f823ce68df6] https://lava.sirena.org.uk/scheduler/job/1974175
# test job: [2191593d061cfee48763d67e69ccb1d27655ee62] https://lava.sirena.org.uk/scheduler/job/1974367
# test job: [eb52e20b15eac1c230fb81f7481f6f1299c2a9b8] https://lava.sirena.org.uk/scheduler/job/1973831
# test job: [5366dcebe65bf1dc82a7362e505b69d441b5f755] https://lava.sirena.org.uk/scheduler/job/1974209
# test job: [e548389df744bd28db1ba3923b6ee74ba6f73926] https://lava.sirena.org.uk/scheduler/job/1974056
# test job: [404e8bfcb9d6a8b6af4745f23906a7382b7eb4c4] https://lava.sirena.org.uk/scheduler/job/1973657
# test job: [fd27a636d3746ae20e0853f4fa48b93080a2a43d] https://lava.sirena.org.uk/scheduler/job/1974340
# test job: [a2aa8c0a2f1cbc5be1cb6195038e4484d2af38d9] https://lava.sirena.org.uk/scheduler/job/1974036
# test job: [14af3d3c4d2d8d01d91049576a59590d9c586f3a] https://lava.sirena.org.uk/scheduler/job/1974368
# test job: [09b29035fb5c93fa0db24df165081cc27239b1f0] https://lava.sirena.org.uk/scheduler/job/1974219
# test job: [7e90cf82a33827e174d3955d3ede1b3e7bb56325] https://lava.sirena.org.uk/scheduler/job/1973993
# test job: [455f2f9509b0034f32979bcdbc5441579c3a1d0a] https://lava.sirena.org.uk/scheduler/job/1973704
# test job: [3c9e6dccede1146d53f940ff8b25ceee275c0686] https://lava.sirena.org.uk/scheduler/job/1973839
# test job: [13c4c81b7bc819d533c6e0aaa1bf495cd0dfd8f6] https://lava.sirena.org.uk/scheduler/job/1973803
# test job: [0bfe0c0a9aa7edb88f571bfbfc8f4546d6b311cb] https://lava.sirena.org.uk/scheduler/job/1974071
# test job: [c0a750ce364b248efc0f4b993904dcef194612cf] https://lava.sirena.org.uk/scheduler/job/1974370
# test job: [205a7f5c6ec0051248384e7440fb17d3f3aee372] https://lava.sirena.org.uk/scheduler/job/1973741
# test job: [46c4e08bb11aeae95c364fa610ac6348896b881a] https://lava.sirena.org.uk/scheduler/job/1973780
# test job: [f7fbe0ea5f03d918418e6ad28d61bcae52bb2d22] https://lava.sirena.org.uk/scheduler/job/1973882
# test job: [5ab215f2b60ec988e63ddde421eecc6ced137064] https://lava.sirena.org.uk/scheduler/job/1974456
# test job: [2e6db9d197ed161702434c6c14b1c65bdef51a2c] https://lava.sirena.org.uk/scheduler/job/1973623
# test job: [8f1d72f84e85a8757a0981040654be1fd18ea6e3] https://lava.sirena.org.uk/scheduler/job/1974182
# test job: [6047387ab92acfc77dd981df27adca36b0c2e9ea] https://lava.sirena.org.uk/scheduler/job/1974303
# test job: [8b5cc56389d08debb225d94a35e62e11a80a9b4f] https://lava.sirena.org.uk/scheduler/job/1973854
# test job: [2673034a1911c215f67c64d0b844563f7903a677] https://lava.sirena.org.uk/scheduler/job/1974480
# test job: [c34209ba43623c2ea2593ee332f4e4f6b47fb921] https://lava.sirena.org.uk/scheduler/job/1974436
# test job: [1ad2f1c5662fabafc8df405a2e480ba45cdaf054] https://lava.sirena.org.uk/scheduler/job/1974012
# test job: [2f0a334df0fd8e0793a7dbea40fca0d4a48dc927] https://lava.sirena.org.uk/scheduler/job/1974087
# test job: [fce217449075d59b29052b8cdac567f0f3e22641] https://lava.sirena.org.uk/scheduler/job/1974899
# test job: [2041666b8f4320da1f7b8efacaafab40a745e126] https://lava.sirena.org.uk/scheduler/job/1973886
# test job: [c5f73c6679ef675fdb4e35dbc8ae0ec59eb0526c] https://lava.sirena.org.uk/scheduler/job/1973682
# test job: [3827b34564951febc0ffa7ce93c1d9c05888e53b] https://lava.sirena.org.uk/scheduler/job/1973779
# test job: [ac7db529b6c2d748102cef60da29f5ea281a757e] https://lava.sirena.org.uk/scheduler/job/1974037
# test job: [a43676fa0585a9988e44ec000649d20c2649a3e4] https://lava.sirena.org.uk/scheduler/job/1973545
# test job: [800c666497e045c3152da516fdd6bc2392571a90] https://lava.sirena.org.uk/scheduler/job/1974387
# test job: [db25c438f9e02c0f1566d54840640d23f59a3d4f] https://lava.sirena.org.uk/scheduler/job/1973804
# test job: [40c729f0a3fa14845d733f5572c283728bea0c32] https://lava.sirena.org.uk/scheduler/job/1974450
# test job: [dc5aa86741789d6dcc0c24dcff5f3ba8fbecccf9] https://lava.sirena.org.uk/scheduler/job/1974137
# test job: [13b2c42b0dcf539a1ddf18374792a7a49a7afa4d] https://lava.sirena.org.uk/scheduler/job/1974220
# test job: [0c933edebd96961529e01686ce6930be5decf400] https://lava.sirena.org.uk/scheduler/job/1973814
# test job: [6658472a3e2de08197acfe099ba71ee0e2505ecf] https://lava.sirena.org.uk/scheduler/job/1974545
# test job: [f2b55e567172ccb248e66e8a0f1c7380279127f1] https://lava.sirena.org.uk/scheduler/job/1974311
# test job: [73978d274eba0d9081bc9b5aedebb0bc6abb832c] https://lava.sirena.org.uk/scheduler/job/1974865
# test job: [344af572f088e2bb1248bb752ea3a532f7fa4208] https://lava.sirena.org.uk/scheduler/job/1973842
# test job: [4b5e66fc0315b9a85ce817b97a1f33829eeadd29] https://lava.sirena.org.uk/scheduler/job/1973957
# test job: [b1daca0396b91a9b436b0654b8aae6a8220aa51b] https://lava.sirena.org.uk/scheduler/job/1973961
# test job: [ef464d9c418741ac83a33c7662db3cffda655b4f] https://lava.sirena.org.uk/scheduler/job/1974329
# test job: [3c000883d110b381fca21572147a8c0463ba4a25] https://lava.sirena.org.uk/scheduler/job/1973759
# test job: [aab1301cfde344f966bbc442a4e655867ab56768] https://lava.sirena.org.uk/scheduler/job/1974589
# test job: [3a3271a52075d4dc34f403c0ed850801cdc4bd4d] https://lava.sirena.org.uk/scheduler/job/1974055
# test job: [9549a29371c5548a4eb0d43622a25d6bc989337f] https://lava.sirena.org.uk/scheduler/job/1974118
# test job: [b805b4e062fa216302f86bd66485734683a51236] https://lava.sirena.org.uk/scheduler/job/1974116
# test job: [3815962969c811bbfa32c7d503246f9abace38cb] https://lava.sirena.org.uk/scheduler/job/1974420
# test job: [62b175d6ff19de91544c9047dcaec8a33a5759a1] https://lava.sirena.org.uk/scheduler/job/1973602
# test job: [ee4ac04e3ccd7f4d54e767b7ab5a4067f9737fff] https://lava.sirena.org.uk/scheduler/job/1974088
# test job: [a014442252380abd19a81ae27fdfc29b988203c6] https://lava.sirena.org.uk/scheduler/job/1973761
# test job: [0cc08c8130ac8f74419f99fe707dc193b7f79d86] https://lava.sirena.org.uk/scheduler/job/1965828
# test job: [0743acf746a81e0460a56fd5ff847d97fa7eb370] https://lava.sirena.org.uk/scheduler/job/1964801
# test job: [902c0ea18a97b1a6eeee5799cb1fd9a79ef9208e] https://lava.sirena.org.uk/scheduler/job/1964805
# test job: [0cdb2b1b7edaefb54773d790c7b5c2e4ac7db60d] https://lava.sirena.org.uk/scheduler/job/1965934
# test job: [233a22687411ea053a4b169c07324ee6aa33bf38] https://lava.sirena.org.uk/scheduler/job/1964973
# test job: [661856ca131c8bf6724905966e02149805660abe] https://lava.sirena.org.uk/scheduler/job/1964989
# test job: [93b2838c6e79bc263e6129d88c5ab043dd793d28] https://lava.sirena.org.uk/scheduler/job/1964752
# test job: [7350f8dc15bfbb7abf1ce4babea6fcace1c574c5] https://lava.sirena.org.uk/scheduler/job/1965005
# test job: [80b09137aeab27e59004383058f8cc696a9ee048] https://lava.sirena.org.uk/scheduler/job/1964893
# test job: [d1ff30df1d9a4eb4c067795abb5e2a66910fd108] https://lava.sirena.org.uk/scheduler/job/1964872
# test job: [5263cd81578f99a00b2dd7de1da2b570b96a1b7c] https://lava.sirena.org.uk/scheduler/job/1964729
# test job: [70eec454f2d6cdfab547c262781acd38328e11a1] https://lava.sirena.org.uk/scheduler/job/1964802
# test job: [fb81b5cecb8553e3ca2b45288cf340d43c9c2991] https://lava.sirena.org.uk/scheduler/job/1964806
# test job: [15afe57a874eaf104bfbb61ec598fa31627f7b19] https://lava.sirena.org.uk/scheduler/job/1962843
# test job: [281c97376cfcfc8cef4f5ed5dd961a1b39f5a25e] https://lava.sirena.org.uk/scheduler/job/1962733
# test job: [d77daa49085b067137d0adbe3263f75a7ee13a1b] https://lava.sirena.org.uk/scheduler/job/1962731
# test job: [367ca0688e4218e51c3d4dfdf3ef5657a62cf88d] https://lava.sirena.org.uk/scheduler/job/1962744
# test job: [4673dbe9837e3eb2fecdd12f0953006c31636aac] https://lava.sirena.org.uk/scheduler/job/1962824
# test job: [1e570e77392f43a3cdab2849d1f81535f8a033e2] https://lava.sirena.org.uk/scheduler/job/1962090
# test job: [cee2c8396d9c8e834fe28929bc1d8153d7e9897f] https://lava.sirena.org.uk/scheduler/job/1959131
# test job: [1356c98ef911e14ccfaf374800840ce5bdcb3bbd] https://lava.sirena.org.uk/scheduler/job/1959872
# test job: [fb25114cd760c13cf177d9ac37837fafcc9657b5] https://lava.sirena.org.uk/scheduler/job/1960106
# test job: [835dfb12fc389f36eb007657f163bd1c539dcd45] https://lava.sirena.org.uk/scheduler/job/1959893
# test job: [79c36ecfc8994011ab0a973d3b4148aa5d9e0c91] https://lava.sirena.org.uk/scheduler/job/1959142
# test job: [c1afb0350069c3be137b5692923ad13d69648970] https://lava.sirena.org.uk/scheduler/job/1959128
# test job: [65efe5404d151767653c7b7dd39bd2e7ad532c2d] https://lava.sirena.org.uk/scheduler/job/1959881
# test job: [6621b0f118d500092f5f3d72ddddb22aeeb3c3a0] https://lava.sirena.org.uk/scheduler/job/1959694
# test job: [6a8cdef7dc2a4c0dbde3f7d7100b3d99712a766b] https://lava.sirena.org.uk/scheduler/job/1959787
# test job: [433e294c3c5b5d2020085a0e36c1cb47b694690a] https://lava.sirena.org.uk/scheduler/job/1957473
# test job: [a758314f71ba90cca2a5813bbf96c4954a15b613] https://lava.sirena.org.uk/scheduler/job/1957722
# test job: [7d9c2924f61dcabcbc5868bec6054ab4f4de01d1] https://lava.sirena.org.uk/scheduler/job/1957726
# test job: [92a42edd347c3b5a9045bb137a33204c6ddc0803] https://lava.sirena.org.uk/scheduler/job/1957755
# test job: [0b0eb7702a9fa410755e86124b4b7cd36e7d1cb4] https://lava.sirena.org.uk/scheduler/job/1957285
# test job: [0d202ae0256e8e7dcea862ead5904fa27cf4ce6a] https://lava.sirena.org.uk/scheduler/job/1954202
# test job: [7f47685b150dbc20f881d029a7366a81b1d66322] https://lava.sirena.org.uk/scheduler/job/1954114
# test job: [7e7e2c6e2a1cb250f8d03bb99eed01f6d982d5dd] https://lava.sirena.org.uk/scheduler/job/1954164
# test job: [55f8b5a96597a7b88c323a7de7228f9eae8c9943] https://lava.sirena.org.uk/scheduler/job/1954220
# test job: [bb940b13998c40d55e186f0cf5d65c592ea1677a] https://lava.sirena.org.uk/scheduler/job/1954123
# test job: [225d70b8074502acee3943bf0c2e839e867cd38c] https://lava.sirena.org.uk/scheduler/job/1954173
# test job: [aa1ee85ce3576defd29f2a389d7508d2036af977] https://lava.sirena.org.uk/scheduler/job/1954135
# test job: [3180c7b1575d635851f0ceab6bdb176bb15e69dd] https://lava.sirena.org.uk/scheduler/job/1954106
# test job: [506cbe36a2ac7b504a2362476dc53cd548b7a29e] https://lava.sirena.org.uk/scheduler/job/1954097
# test job: [b926b15547d29a88932de3c24a05c12826fc1dbc] https://lava.sirena.org.uk/scheduler/job/1945992
# test job: [454cd43a283f7697297c52981c7a499a16725656] https://lava.sirena.org.uk/scheduler/job/1946471
# test job: [d25de16477657f9eddd4be9abd409515edcc3b9e] https://lava.sirena.org.uk/scheduler/job/1946751
# test job: [6277a486a7faaa6c87f4bf1d59a2de233a093248] https://lava.sirena.org.uk/scheduler/job/1946550
# test job: [31dcc7e1f8a9377d8fd9f967f84c121c5ba8f89c] https://lava.sirena.org.uk/scheduler/job/1946712
# test job: [4d410ba9aa275e7990a270f63ce436990ace1bea] https://lava.sirena.org.uk/scheduler/job/1947640
# test job: [9797329220a2c6622411eb9ecf6a35b24ce09d04] https://lava.sirena.org.uk/scheduler/job/1946616
# test job: [efb79de36e947d136517bac14c139d494fcc72fa] https://lava.sirena.org.uk/scheduler/job/1946693
# test job: [4412ab501677606436e5c49e41151a1e6eac7ac0] https://lava.sirena.org.uk/scheduler/job/1946492
# test job: [5e537031f322d55315cd384398b726a9a0748d47] https://lava.sirena.org.uk/scheduler/job/1946499
# test job: [630a185fd06109193574d10f38b29812986c21de] https://lava.sirena.org.uk/scheduler/job/1946792
# test job: [64d87ccfae3326a9561fe41dc6073064a083e0df] https://lava.sirena.org.uk/scheduler/job/1946747
# test job: [fe8cc44dd173cde5788ab4e3730ac61f3d316d9c] https://lava.sirena.org.uk/scheduler/job/1945985
# test job: [b546e0023a203e7edf9377ac8f4f490a6965afd6] https://lava.sirena.org.uk/scheduler/job/1946613
# test job: [7196fc4e482928a276da853e2687f31cd8ea2611] https://lava.sirena.org.uk/scheduler/job/1946720
# test job: [6937ff42f28a13ffdbe2d1f5b9a51a35f626e93a] https://lava.sirena.org.uk/scheduler/job/1946793
# test job: [2b92b98cc4765fbb0748742e7e0dd94d15d6f178] https://lava.sirena.org.uk/scheduler/job/1946573
# test job: [5226d19d4cae5398caeb93a6052bfb614e0099c7] https://lava.sirena.org.uk/scheduler/job/1946778
# test job: [5ed60e45c59d66e61586a10433e2b5527d4d72b5] https://lava.sirena.org.uk/scheduler/job/1946694
# test job: [0586b53d4a0c7c5a132629f99da934cc674ea4cd] https://lava.sirena.org.uk/scheduler/job/1946770
# test job: [99c159279c6dfa2c4867c7f76875f58263f8f43b] https://lava.sirena.org.uk/scheduler/job/1946864
# test job: [ea97713903784286ef1ce45456f404ed288f19b1] https://lava.sirena.org.uk/scheduler/job/1946601
# test job: [29fa213c6ab00c6749db47b47e384cab760c109e] https://lava.sirena.org.uk/scheduler/job/1946468
# test job: [4ebe64f507ca921c5109eb37eae6058b77413d93] https://lava.sirena.org.uk/scheduler/job/1946511
# test job: [3c89238ca35bfe176ba34bc688541f90f6fa7bdb] https://lava.sirena.org.uk/scheduler/job/1946491
# test job: [b83fb1b14c06bdd765903ac852ba20a14e24f227] https://lava.sirena.org.uk/scheduler/job/1946513
# test job: [d724c6f85e80a23ed46b7ebc6e38b527c09d64f5] https://lava.sirena.org.uk/scheduler/job/2118455
# bad: [d724c6f85e80a23ed46b7ebc6e38b527c09d64f5] Add linux-next specific files for 20251121
git bisect bad d724c6f85e80a23ed46b7ebc6e38b527c09d64f5
# test job: [f8b3ebbb82900f343dd4ffec060178451ac41a27] https://lava.sirena.org.uk/scheduler/job/2118688
# good: [f8b3ebbb82900f343dd4ffec060178451ac41a27] Merge branch 'for-next' of https://git.kernel.org/pub/scm/linux/kernel/git/wireless/wireless-next.git
git bisect good f8b3ebbb82900f343dd4ffec060178451ac41a27
# test job: [f9a122a438df4810da86c7e4e7d73b1ac92a2537] https://lava.sirena.org.uk/scheduler/job/2118725
# good: [f9a122a438df4810da86c7e4e7d73b1ac92a2537] Merge branch 'next' of https://git.kernel.org/pub/scm/linux/kernel/git/pcmoore/lsm.git
git bisect good f9a122a438df4810da86c7e4e7d73b1ac92a2537
# test job: [2c07519d6684791ae164b5ea77733bbbe11e2dfe] https://lava.sirena.org.uk/scheduler/job/2118832
# bad: [2c07519d6684791ae164b5ea77733bbbe11e2dfe] Merge branch 'usb-next' of https://git.kernel.org/pub/scm/linux/kernel/git/gregkh/usb.git
git bisect bad 2c07519d6684791ae164b5ea77733bbbe11e2dfe
# skip: [8d00858e9326dafe5e27072e3cee5c23b60e7281] Merge branch 'master' of https://git.kernel.org/pub/scm/linux/kernel/git/tip/tip.git
git bisect skip 8d00858e9326dafe5e27072e3cee5c23b60e7281
# test job: [082c8dc13a3b63713ca6f3b2b5f31a47fb345e17] https://lava.sirena.org.uk/scheduler/job/2118902
# good: [082c8dc13a3b63713ca6f3b2b5f31a47fb345e17] dt-bindings: usb: samsung,exynos-dwc3 add exynos8890 compatible
git bisect good 082c8dc13a3b63713ca6f3b2b5f31a47fb345e17
# test job: [79eaabc61dfbf5a4b680f42d3a113d05333c3960] https://lava.sirena.org.uk/scheduler/job/2119049
# good: [79eaabc61dfbf5a4b680f42d3a113d05333c3960] irqchip/riscv-imsic: Embed the vector array in lpriv
git bisect good 79eaabc61dfbf5a4b680f42d3a113d05333c3960
# test job: [4e75697faa7af5a254def4c0939d06d0f5b9ed17] https://lava.sirena.org.uk/scheduler/job/2119125
# good: [4e75697faa7af5a254def4c0939d06d0f5b9ed17] x86/sgx: Implement ENCLS[EUPDATESVN]
git bisect good 4e75697faa7af5a254def4c0939d06d0f5b9ed17
# test job: [3b603955f2423cf668ebd5ba670019a5b4960cc5] https://lava.sirena.org.uk/scheduler/job/2119186
# good: [3b603955f2423cf668ebd5ba670019a5b4960cc5] platform/x86:intel/pmc: Add support for multiple DMU GUIDs
git bisect good 3b603955f2423cf668ebd5ba670019a5b4960cc5
# test job: [a3a4ca462fdbc270dfc5ec97d75f93a3418cf2bb] https://lava.sirena.org.uk/scheduler/job/2119276
# good: [a3a4ca462fdbc270dfc5ec97d75f93a3418cf2bb] KVM: arm64: GICv3: Preserve EOIcount on exit
git bisect good a3a4ca462fdbc270dfc5ec97d75f93a3418cf2bb
# skip: [5892cbd85dbf9059b8a3a7dd8ab64c0fce671029] sched: Match __task_rq_{,un}lock()
git bisect skip 5892cbd85dbf9059b8a3a7dd8ab64c0fce671029
# test job: [3a4703e72e7fe16778e9d0edb59c7ff2295ab231] https://lava.sirena.org.uk/scheduler/job/2119531
# bad: [3a4703e72e7fe16778e9d0edb59c7ff2295ab231] Merge branch into tip/master: 'x86/apic'
git bisect bad 3a4703e72e7fe16778e9d0edb59c7ff2295ab231
# test job: [1b81535b2ca1cccd9ced1fc69a320f0f7de9c8f0] https://lava.sirena.org.uk/scheduler/job/2119603
# bad: [1b81535b2ca1cccd9ced1fc69a320f0f7de9c8f0] Merge branch into tip/master: 'irq/drivers'
git bisect bad 1b81535b2ca1cccd9ced1fc69a320f0f7de9c8f0
# test job: [705d7cad382be9b3b52b89b220e77b17cf24322a] https://lava.sirena.org.uk/scheduler/job/2119627
# bad: [705d7cad382be9b3b52b89b220e77b17cf24322a] Merge branch 'sched/core' into core/rseq, to resolve conflict
git bisect bad 705d7cad382be9b3b52b89b220e77b17cf24322a
# test job: [472931e757fb3dfad1f78ce6f5abd821155433b2] https://lava.sirena.org.uk/scheduler/job/2119740
# good: [472931e757fb3dfad1f78ce6f5abd821155433b2] sched/mmcid: Convert mm CID mask to a bitmap
git bisect good 472931e757fb3dfad1f78ce6f5abd821155433b2
# test job: [7f829bde94b1c97b1804fa5860e066ea49dbfca3] https://lava.sirena.org.uk/scheduler/job/2119828
# good: [7f829bde94b1c97b1804fa5860e066ea49dbfca3] sched/core: Optimize core cookie matching check
git bisect good 7f829bde94b1c97b1804fa5860e066ea49dbfca3
# test job: [d206fbad9328ddb68ebabd7cf7413392acd38081] https://lava.sirena.org.uk/scheduler/job/2119924
# good: [d206fbad9328ddb68ebabd7cf7413392acd38081] sched/fair: Revert max_newidle_lb_cost bump
git bisect good d206fbad9328ddb68ebabd7cf7413392acd38081
# test job: [340af997d25dab0f05c4de8399d656b112592a93] https://lava.sirena.org.uk/scheduler/job/2119994
# good: [340af997d25dab0f05c4de8399d656b112592a93] sched/mmcid: Provide CID ownership mode fixup functions
git bisect good 340af997d25dab0f05c4de8399d656b112592a93
# test job: [2635fb0f0973c57c45f03708d52e827ec99ac78e] https://lava.sirena.org.uk/scheduler/job/2120065
# bad: [2635fb0f0973c57c45f03708d52e827ec99ac78e] sched/mmcid: Switch over to the new mechanism
git bisect bad 2635fb0f0973c57c45f03708d52e827ec99ac78e
# test job: [2644779ec144d3e8cce5fed9623b47e70b3e0422] https://lava.sirena.org.uk/scheduler/job/2120120
# good: [2644779ec144d3e8cce5fed9623b47e70b3e0422] irqwork: Move data struct to a types header
git bisect good 2644779ec144d3e8cce5fed9623b47e70b3e0422
# test job: [cba5e581161e379037a94f5a75d1a61bd1ccce3b] https://lava.sirena.org.uk/scheduler/job/2120376
# good: [cba5e581161e379037a94f5a75d1a61bd1ccce3b] sched/mmcid: Implement deferred mode change
git bisect good cba5e581161e379037a94f5a75d1a61bd1ccce3b
# first bad commit: [2635fb0f0973c57c45f03708d52e827ec99ac78e] sched/mmcid: Switch over to the new mechanism

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

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

* Re: [patch V3 20/20] sched/mmcid: Switch over to the new mechanism
  2025-11-24 12:10   ` Mark Brown
@ 2025-11-24 12:27     ` Florian Weimer
  2025-11-24 13:28       ` Mark Brown
  0 siblings, 1 reply; 65+ messages in thread
From: Florian Weimer @ 2025-11-24 12:27 UTC (permalink / raw)
  To: Mark Brown
  Cc: Thomas Gleixner, LKML, Peter Zijlstra, Gabriele Monaco,
	Mathieu Desnoyers, Michael Jeanson, Jens Axboe, Paul E. McKenney,
	Gautham R. Shenoy, Tim Chen, Yury Norov, Shrikanth Hegde

* Mark Brown:

> On Wed, Oct 29, 2025 at 02:09:32PM +0100, Thomas Gleixner wrote:
>> Now that all pieces are in place, change the implementations of
>> sched_mm_cid_fork() and sched_mm_cid_exit() to adhere to the new strict
>> ownership scheme and switch context_switch() over to use the new
>> mm_cid_schedin() functionality.
>> 
>> The common case is that there is no mode change required, which makes
>> fork() and exit() just update the user count and the constraints.
>
> I'm seeing boot failures on x86 qemu in -next which bisect to this
> patch.  The boot gets to userspace but then grinds to a halt:
>
>     https://lava.sirena.org.uk/scheduler/job/2118455#L542
>
> which isn't terribly informative unfortunately.  I don't have any x86
> hardware, and nothing else seems affected.

I believe that's already discussed here:

  <https://lore.kernel.org/873466jekm.ffs@tglx/>

Thanks,
Florian


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

* Re: [patch V3 20/20] sched/mmcid: Switch over to the new mechanism
  2025-11-24 12:27     ` Florian Weimer
@ 2025-11-24 13:28       ` Mark Brown
  0 siblings, 0 replies; 65+ messages in thread
From: Mark Brown @ 2025-11-24 13:28 UTC (permalink / raw)
  To: Florian Weimer
  Cc: Thomas Gleixner, LKML, Peter Zijlstra, Gabriele Monaco,
	Mathieu Desnoyers, Michael Jeanson, Jens Axboe, Paul E. McKenney,
	Gautham R. Shenoy, Tim Chen, Yury Norov, Shrikanth Hegde

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

On Mon, Nov 24, 2025 at 01:27:15PM +0100, Florian Weimer wrote:
> * Mark Brown:

> > I'm seeing boot failures on x86 qemu in -next which bisect to this
> > patch.  The boot gets to userspace but then grinds to a halt:

> >     https://lava.sirena.org.uk/scheduler/job/2118455#L542

> > which isn't terribly informative unfortunately.  I don't have any x86
> > hardware, and nothing else seems affected.

> I believe that's already discussed here:

>   <https://lore.kernel.org/873466jekm.ffs@tglx/>

Yes, that looks like the same thing - thanks!

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

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

end of thread, other threads:[~2025-11-24 13:28 UTC | newest]

Thread overview: 65+ messages (download: mbox.gz follow: Atom feed
-- links below jump to the message on this page --
2025-10-29 13:08 [patch V3 00/20] sched: Rewrite MM CID management Thomas Gleixner
2025-10-29 13:08 ` [patch V3 01/20] sched/mmcid: Revert the complex " Thomas Gleixner
2025-10-29 13:08 ` [patch V3 02/20] sched/mmcid: Use proper data structures Thomas Gleixner
2025-10-29 15:31   ` Mathieu Desnoyers
2025-10-29 13:08 ` [patch V3 03/20] sched/mmcid: Cacheline align MM CID storage Thomas Gleixner
2025-10-29 15:39   ` Mathieu Desnoyers
2025-10-29 21:09     ` Thomas Gleixner
2025-10-30 14:15       ` Mathieu Desnoyers
2025-10-29 13:09 ` [patch V3 04/20] sched: Fixup whitespace damage Thomas Gleixner
2025-10-29 15:42   ` Mathieu Desnoyers
2025-10-29 21:11     ` Thomas Gleixner
2025-10-29 13:09 ` [patch V3 05/20] sched/mmcid: Move scheduler code out of global header Thomas Gleixner
2025-10-29 15:43   ` Mathieu Desnoyers
2025-10-29 13:09 ` [patch V3 06/20] sched/mmcid: Prevent pointless work in mm_update_cpus_allowed() Thomas Gleixner
2025-10-29 15:45   ` Mathieu Desnoyers
2025-10-29 13:09 ` [patch V3 07/20] cpumask: Introduce cpumask_weighted_or() Thomas Gleixner
2025-10-29 15:49   ` Mathieu Desnoyers
2025-11-03  9:15   ` Shrikanth Hegde
2025-11-03 13:29     ` Thomas Gleixner
2025-11-10 16:11       ` Peter Zijlstra
2025-11-10 16:42         ` Mathieu Desnoyers
2025-10-29 13:09 ` [patch V3 08/20] sched/mmcid: Use cpumask_weighted_or() Thomas Gleixner
2025-10-29 15:51   ` Mathieu Desnoyers
2025-10-29 13:09 ` [patch V3 09/20] cpumask: Cache num_possible_cpus() Thomas Gleixner
2025-10-29 15:54   ` Mathieu Desnoyers
2025-10-29 21:11     ` Thomas Gleixner
2025-11-01 22:59       ` Thomas Gleixner
2025-11-03 10:06   ` Shrikanth Hegde
2025-11-03 13:28     ` Thomas Gleixner
2025-10-29 13:09 ` [patch V3 10/20] sched/mmcid: Convert mm CID mask to a bitmap Thomas Gleixner
2025-10-30 13:59   ` Mathieu Desnoyers
2025-10-29 13:09 ` [patch V3 11/20] signal: Move MMCID exit out of sighand lock Thomas Gleixner
2025-10-29 13:09 ` [patch V3 12/20] sched/mmcid: Move initialization out of line Thomas Gleixner
2025-10-30 14:02   ` Mathieu Desnoyers
2025-10-29 13:09 ` [patch V3 13/20] sched/mmcid: Provide precomputed maximal value Thomas Gleixner
2025-10-30 14:23   ` Mathieu Desnoyers
2025-10-31 15:06     ` Thomas Gleixner
2025-10-31 15:14       ` Mathieu Desnoyers
2025-10-29 13:09 ` [patch V3 14/20] sched/mmcid: Serialize sched_mm_cid_fork()/exit() with a mutex Thomas Gleixner
2025-10-30 14:25   ` Mathieu Desnoyers
2025-10-29 13:09 ` [patch V3 15/20] sched/mmcid: Introduce per task/CPU ownership infrastrcuture Thomas Gleixner
2025-10-30 14:45   ` Mathieu Desnoyers
2025-10-29 13:09 ` [patch V3 16/20] sched/mmcid: Provide new scheduler CID mechanism Thomas Gleixner
2025-10-30 15:09   ` Mathieu Desnoyers
2025-10-29 13:09 ` [patch V3 17/20] sched/mmcid: Provide CID ownership mode fixup functions Thomas Gleixner
2025-10-30 15:51   ` Mathieu Desnoyers
2025-10-31 16:54     ` Thomas Gleixner
2025-10-31 19:15       ` Mathieu Desnoyers
2025-10-29 13:09 ` [patch V3 18/20] irqwork: Move data struct to a types header Thomas Gleixner
2025-10-30 15:52   ` Mathieu Desnoyers
2025-10-29 13:09 ` [patch V3 19/20] sched/mmcid: Implement deferred mode change Thomas Gleixner
2025-10-30 15:56   ` Mathieu Desnoyers
2025-10-29 13:09 ` [patch V3 20/20] sched/mmcid: Switch over to the new mechanism Thomas Gleixner
2025-10-30 16:07   ` Mathieu Desnoyers
2025-10-31 16:57     ` Thomas Gleixner
2025-10-31 19:17       ` Mathieu Desnoyers
2025-11-24 12:10   ` Mark Brown
2025-11-24 12:27     ` Florian Weimer
2025-11-24 13:28       ` Mark Brown
2025-10-30  5:00 ` [patch V3 00/20] sched: Rewrite MM CID management Shrikanth Hegde
2025-10-30  6:40   ` Shrikanth Hegde
2025-10-31 19:36     ` Thomas Gleixner
2025-11-01  7:56       ` Shrikanth Hegde
2025-11-01 12:27         ` Thomas Gleixner
2025-11-10 17:09 ` Gabriele Monaco

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