2023-04-19 15:55:31

by Mathieu Desnoyers

[permalink] [raw]
Subject: [RFC PATCH v9 2/2] sched: Fix performance regression introduced by mm_cid

Introduce per-mm/cpu current concurrency id (mm_cid) to fix a PostgreSQL
sysbench regression reported by Aaron Lu.

Keep track of the currently allocated mm_cid for each mm/cpu rather than
freeing them immediately on context switch. This eliminates most atomic
operations when context switching back and forth between threads
belonging to different memory spaces in multi-threaded scenarios (many
processes, each with many threads). The per-mm/per-cpu mm_cid values are
serialized by their respective runqueue locks.

Thread migration is handled by introducing invocation to
sched_mm_cid_migrate_to() (with destination runqueue lock held) in
activate_task() for migrating tasks. If the destination cpu's mm_cid is
unset, and if the source runqueue is not actively using its mm_cid, then
the source cpu's mm_cid is moved to the destination cpu on migration.

Introduce a task-work executed periodically, similarly to NUMA work,
which delays reclaim of cid values when they are unused for a period of
time.

Keep track of the allocation time for each per-cpu cid, and let the task
work clear them when they are observed to be older than
SCHED_MM_CID_PERIOD_NS and unused. This task work also clears all
mm_cids which are greater or equal to the Hamming weight of the mm
cidmask to keep concurrency ids compact.

Because we want to ensure the mm_cid converges towards the smaller
values as migrations happen, the prior optimization that was done when
context switching between threads belonging to the same mm is removed,
because it could delay the lazy release of the destination runqueue
mm_cid after it has been replaced by a migration. Removing this prior
optimization is not an issue performance-wise because the introduced
per-mm/per-cpu mm_cid tracking also covers this more specific case.

This patch is based on v6.3-rc7 with this patch applied:

("mm: Fix memory leak on mm_init error handling")

https://lore.kernel.org/lkml/[email protected]/

Fixes: af7f588d8f73 ("sched: Introduce per-memory-map concurrency ID")
Link: https://lore.kernel.org/lkml/20230327080502.GA570847@ziqianlu-desk2/
Signed-off-by: Mathieu Desnoyers <[email protected]>
Cc: Aaron Lu <[email protected]>
Cc: Peter Zijlstra <[email protected]>
Cc: Olivier Dion <[email protected]>
Cc: [email protected]
---
Changes since v8:
- In order to take care of infrequently scheduled tasks, bump the time
snapshot associated with old cids if an active task using the mm is
observed on their associated rq.
- Eliminate this_cpu_ptr() overhead on fast-paths by using
__this_cpu_read() and __this_cpu_write().
- Eliminate duplicated this_cpu_ptr() by handing the resulting pointer
as argument to the callees in sched_mm_cid_migrate_to and
sched_mm_cid_remote_clear.

Changes since v7:
- Call sched_mm_cid_migrate_to() for try-to-wake-up migrations as well,
- Remove next_scan bogus variable which shadows actual initialized
variable. Fixed issue reported by kernel test robot,
- Clear the src_cid in sched_mm_cid_migrate_to() in situations where the
dst_cid is already set to prevent over-allocation of cids in frequent
migration scenarios when atomic_read(&mm->mm_users) < t->nr_cpus_allowed.
- Rebase on v6.3-rc7.

Changes since v6:
- Reintroduce migrate-to hook to hand-off cid from source to destination
runqueue. This ensures that a single-threaded process always observes
mm_cid==0 when migrating across CPUs.
- Ensure compactness of mm_cid in task-work by clearing mm_cid per-cpu
entries that are above the Hamming weight of the mm cidmask.

Changes since v5:
- Introduce task-work and delay-based reclaim.

Changes since v4:
- Retry on filled mm_cid mask,
- Disable interrupts during ownership of cid in migrate-from without rq
lock,
- Implement lock-free cid-get, using spinlock only to guarantee
forward progress when contention is detected.
- Use try_cmpxchg where appropriate.

Changes since v3:
- Rebase on v6.3-rc6,
- Move switch_mm_cid() from prepare_task_switch() to context_switch()
after the switch_to() barrier,
- Adding missing barriers at the beginning of switch_mm_cid(),
- Document the lock-free migrate-from algorithm.
- Add memory barriers in sched_mm_cid_exit_signals(),
sched_mm_cid_before_execve(), and sched_mm_cid_after_execve() to
order t->mm_cid_active store, per-mm/cpu cid load wrt migrate-from
per-mm-cid cid cmpxchg, t->mm_cid_active load.
---
include/linux/mm_types.h | 82 +++++-
include/linux/sched.h | 3 +
include/linux/sched/mm.h | 5 +
kernel/fork.c | 9 +-
kernel/sched/core.c | 524 +++++++++++++++++++++++++++++++++++++--
kernel/sched/sched.h | 235 +++++++++++++++---
6 files changed, 803 insertions(+), 55 deletions(-)

diff --git a/include/linux/mm_types.h b/include/linux/mm_types.h
index a57e6ae78e65..5eab61156f0e 100644
--- a/include/linux/mm_types.h
+++ b/include/linux/mm_types.h
@@ -550,6 +550,13 @@ struct vm_area_struct {
struct vm_userfaultfd_ctx vm_userfaultfd_ctx;
} __randomize_layout;

+#ifdef CONFIG_SCHED_MM_CID
+struct mm_cid {
+ u64 time;
+ int cid;
+};
+#endif
+
struct kioctx_table;
struct mm_struct {
struct {
@@ -600,15 +607,19 @@ struct mm_struct {
atomic_t mm_count;
#ifdef CONFIG_SCHED_MM_CID
/**
- * @cid_lock: Protect cid bitmap updates vs lookups.
+ * @pcpu_cid: Per-cpu current cid.
*
- * Prevent situations where updates to the cid bitmap happen
- * concurrently with lookups. Those can lead to situations
- * where a lookup cannot find a free bit simply because it was
- * unlucky enough to load, non-atomically, bitmap words as they
- * were being concurrently updated by the updaters.
+ * Keep track of the currently allocated mm_cid for each cpu.
+ * The per-cpu mm_cid values are serialized by their respective
+ * runqueue locks.
*/
- raw_spinlock_t cid_lock;
+ 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;
#endif
#ifdef CONFIG_MMU
atomic_long_t pgtables_bytes; /* size of all page tables */
@@ -873,6 +884,37 @@ static inline void vma_iter_init(struct vma_iterator *vmi,
}

#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;
+}
+
/* Accessor for struct mm_struct's cidmask. */
static inline cpumask_t *mm_cidmask(struct mm_struct *mm)
{
@@ -886,16 +928,40 @@ static inline cpumask_t *mm_cidmask(struct mm_struct *mm)

static inline void mm_init_cid(struct mm_struct *mm)
{
- raw_spin_lock_init(&mm->cid_lock);
+ int i;
+
+ for_each_possible_cpu(i) {
+ struct mm_cid *pcpu_cid = per_cpu_ptr(mm->pcpu_cid, i);
+
+ pcpu_cid->cid = MM_CID_UNSET;
+ pcpu_cid->time = 0;
+ }
cpumask_clear(mm_cidmask(mm));
}

+static inline int mm_alloc_cid(struct mm_struct *mm)
+{
+ mm->pcpu_cid = alloc_percpu(struct mm_cid);
+ if (!mm->pcpu_cid)
+ return -ENOMEM;
+ mm_init_cid(mm);
+ return 0;
+}
+
+static inline void mm_destroy_cid(struct mm_struct *mm)
+{
+ free_percpu(mm->pcpu_cid);
+ mm->pcpu_cid = NULL;
+}
+
static inline unsigned int mm_cid_size(void)
{
return cpumask_size();
}
#else /* CONFIG_SCHED_MM_CID */
static inline void mm_init_cid(struct mm_struct *mm) { }
+static inline int mm_alloc_cid(struct mm_struct *mm) { return 0; }
+static inline void mm_destroy_cid(struct mm_struct *mm) { }
static inline unsigned int mm_cid_size(void)
{
return 0;
diff --git a/include/linux/sched.h b/include/linux/sched.h
index 63d242164b1a..41c24f406fcf 100644
--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -1313,7 +1313,10 @@ struct task_struct {

#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 tlbflush_unmap_batch tlb_ubc;
diff --git a/include/linux/sched/mm.h b/include/linux/sched/mm.h
index 2a243616f222..f20fc0600fcc 100644
--- a/include/linux/sched/mm.h
+++ b/include/linux/sched/mm.h
@@ -37,6 +37,11 @@ static inline void mmgrab(struct mm_struct *mm)
atomic_inc(&mm->mm_count);
}

+static inline void smp_mb__after_mmgrab(void)
+{
+ smp_mb__after_atomic();
+}
+
extern void __mmdrop(struct mm_struct *mm);

static inline void mmdrop(struct mm_struct *mm)
diff --git a/kernel/fork.c b/kernel/fork.c
index ea332319dffe..ff198be38a11 100644
--- a/kernel/fork.c
+++ b/kernel/fork.c
@@ -793,6 +793,7 @@ void __mmdrop(struct mm_struct *mm)
check_mm(mm);
put_user_ns(mm->user_ns);
mm_pasid_drop(mm);
+ mm_destroy_cid(mm);

for (i = 0; i < NR_MM_COUNTERS; i++)
percpu_counter_destroy(&mm->rss_stat[i]);
@@ -1057,7 +1058,9 @@ static struct task_struct *dup_task_struct(struct task_struct *orig, int node)

#ifdef CONFIG_SCHED_MM_CID
tsk->mm_cid = -1;
+ tsk->last_mm_cid = -1;
tsk->mm_cid_active = 0;
+ tsk->migrate_from_cpu = -1;
#endif
return tsk;

@@ -1162,18 +1165,22 @@ static struct mm_struct *mm_init(struct mm_struct *mm, struct task_struct *p,
if (init_new_context(p, mm))
goto fail_nocontext;

+ if (mm_alloc_cid(mm))
+ goto fail_cid;
+
for (i = 0; i < NR_MM_COUNTERS; i++)
if (percpu_counter_init(&mm->rss_stat[i], 0, GFP_KERNEL_ACCOUNT))
goto fail_pcpu;

mm->user_ns = get_user_ns(user_ns);
lru_gen_init_mm(mm);
- mm_init_cid(mm);
return mm;

fail_pcpu:
while (i > 0)
percpu_counter_destroy(&mm->rss_stat[--i]);
+ mm_destroy_cid(mm);
+fail_cid:
destroy_context(mm);
fail_nocontext:
mm_free_pgd(mm);
diff --git a/kernel/sched/core.c b/kernel/sched/core.c
index 0d18c3969f90..c6e2dd8f4ee3 100644
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -2086,6 +2086,8 @@ void activate_task(struct rq *rq, struct task_struct *p, int flags)
{
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);

@@ -3195,6 +3197,7 @@ void set_task_cpu(struct task_struct *p, unsigned int new_cpu)
p->sched_class->migrate_task_rq(p, new_cpu);
p->se.nr_migrations++;
rseq_migrate(p);
+ sched_mm_cid_migrate_from(p);
perf_event_task_migrate(p);
}

@@ -4468,6 +4471,7 @@ static void __sched_fork(unsigned long clone_flags, struct task_struct *p)
p->wake_entry.u_flags = CSD_TYPE_TTWU;
p->migration_pending = NULL;
#endif
+ init_sched_mm_cid(p);
}

DEFINE_STATIC_KEY_FALSE(sched_numa_balancing);
@@ -5114,7 +5118,6 @@ prepare_task_switch(struct rq *rq, struct task_struct *prev,
sched_info_switch(rq, prev, next);
perf_event_task_sched_out(prev, next);
rseq_preempt(prev);
- switch_mm_cid(prev, next);
fire_sched_out_preempt_notifiers(prev, next);
kmap_local_sched_out();
prepare_task(next);
@@ -5270,6 +5273,9 @@ context_switch(struct rq *rq, struct task_struct *prev,
*
* kernel -> user switch + mmdrop() 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
enter_lazy_tlb(prev->active_mm, next);
@@ -5299,6 +5305,9 @@ context_switch(struct rq *rq, struct task_struct *prev,
}
}

+ /* switch_mm_cid() requires the memory barriers above. */
+ switch_mm_cid(prev, next);
+
rq->clock_update_flags &= ~(RQCF_ACT_SKIP|RQCF_REQ_SKIP);

prepare_lock_switch(rq, next, rf);
@@ -5587,6 +5596,7 @@ void scheduler_tick(void)
resched_latency = cpu_resched_latency(rq);
calc_global_load_tick(rq);
sched_core_tick(rq);
+ task_tick_mm_cid(rq, curr);

rq_unlock(rq, &rf);

@@ -11383,45 +11393,525 @@ void call_trace_sched_update_nr_running(struct rq *rq, int count)
}

#ifdef CONFIG_SCHED_MM_CID
-void sched_mm_cid_exit_signals(struct task_struct *t)
+
+/**
+ * @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.
+ */
+
+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;
- unsigned long flags;
+ 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.
+ */
+ rcu_read_lock();
+ src_task = rcu_dereference(src_rq->curr);
+ if (READ_ONCE(src_task->mm_cid_active) && src_task->mm == mm) {
+ rcu_read_unlock();
+ t->last_mm_cid = -1;
+ return -1;
+ }
+ rcu_read_unlock();
+
+ 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.
+ */
+ rcu_read_lock();
+ src_task = rcu_dereference(src_rq->curr);
+ if (READ_ONCE(src_task->mm_cid_active) && src_task->mm == mm) {
+ rcu_read_unlock();
+ /*
+ * 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;
+ }
+ rcu_read_unlock();
+
+ /*
+ * The src_cid is unused, so it can be unset.
+ */
+ if (!try_cmpxchg(&src_pcpu_cid->cid, &lazy_cid, MM_CID_UNSET))
+ return -1;
+ 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, dst_cid, src_cpu;
+ 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 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 = READ_ONCE(dst_pcpu_cid->cid);
+ if (!mm_cid_is_unset(dst_cid) &&
+ atomic_read(&mm->mm_users) >= t->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 (!mm_cid_is_unset(dst_cid)) {
+ __mm_cid_put(mm, src_cid);
+ return;
+ }
+ /* Move src_cid to dst cpu. */
+ mm_cid_snapshot_time(mm);
+ WRITE_ONCE(dst_pcpu_cid->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;
+ unsigned long flags;
+ 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.
+ */
+ rcu_read_lock();
+ t = rcu_dereference(rq->curr);
+ if (READ_ONCE(t->mm_cid_active) && t->mm == mm) {
+ rcu_read_unlock();
+ return;
+ }
+ rcu_read_unlock();
+
+ /*
+ * The cid is unused, so it can be unset.
+ * Disable interrupts to keep the window of cid ownership without rq
+ * lock small.
+ */
local_irq_save(flags);
- mm_cid_put(mm, t->mm_cid);
- t->mm_cid = -1;
- t->mm_cid_active = 0;
+ if (try_cmpxchg(&pcpu_cid->cid, &lazy_cid, MM_CID_UNSET))
+ __mm_cid_put(mm, cid);
local_irq_restore(flags);
}

+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.
+ */
+ rcu_read_lock();
+ curr = rcu_dereference(rq->curr);
+ if (READ_ONCE(curr->mm_cid_active) && curr->mm == mm) {
+ WRITE_ONCE(pcpu_cid->time, rq_clock);
+ rcu_read_unlock();
+ return;
+ }
+ rcu_read_unlock();
+
+ 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)
+{
+ struct task_struct *t = current;
+ struct mm_struct *mm;
+ int cpu;
+ unsigned long now = jiffies, old_scan, next_scan;
+ struct cpumask *cidmask;
+ int weight;
+
+ SCHED_WARN_ON(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;
+ 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_flags rf;
+ struct rq *rq;
+
+ if (!mm)
+ return;
+
+ preempt_disable();
+ rq = this_rq();
+ rq_lock_irqsave(rq, &rf);
+ 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;
+ rq_unlock_irqrestore(rq, &rf);
+}
+
void sched_mm_cid_before_execve(struct task_struct *t)
{
struct mm_struct *mm = t->mm;
- unsigned long flags;
+ struct rq_flags rf;
+ struct rq *rq;

if (!mm)
return;
- local_irq_save(flags);
- mm_cid_put(mm, t->mm_cid);
- t->mm_cid = -1;
- t->mm_cid_active = 0;
- local_irq_restore(flags);
+
+ preempt_disable();
+ rq = this_rq();
+ rq_lock_irqsave(rq, &rf);
+ 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;
+ rq_unlock_irqrestore(rq, &rf);
}

void sched_mm_cid_after_execve(struct task_struct *t)
{
struct mm_struct *mm = t->mm;
- unsigned long flags;
+ struct rq_flags rf;
+ struct rq *rq;

if (!mm)
return;
- local_irq_save(flags);
- t->mm_cid = mm_cid_get(mm);
- t->mm_cid_active = 1;
- local_irq_restore(flags);
+
+ preempt_disable();
+ rq = this_rq();
+ rq_lock_irqsave(rq, &rf);
+ 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(mm);
+ rq_unlock_irqrestore(rq, &rf);
rseq_set_notify_resume(t);
}

diff --git a/kernel/sched/sched.h b/kernel/sched/sched.h
index 3e8df6d31c1e..d1d470441422 100644
--- a/kernel/sched/sched.h
+++ b/kernel/sched/sched.h
@@ -3249,61 +3249,238 @@ static inline void update_current_exec_runtime(struct task_struct *curr,
}

#ifdef CONFIG_SCHED_MM_CID
-static inline int __mm_cid_get(struct mm_struct *mm)
+
+#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)
+{
+ struct mm_struct *mm = t->mm;
+ struct mm_cid __percpu *pcpu_cid = mm->pcpu_cid;
+ int cid;
+
+ lockdep_assert_rq_held(this_rq());
+ 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))
+ 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_rq_held(this_rq());
+ 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;
+}
+
+static inline void mm_cid_put(struct mm_struct *mm)
+{
+ int cid;
+
+ lockdep_assert_rq_held(this_rq());
+ cid = mm_cid_pcpu_unset(mm);
+ if (cid == MM_CID_UNSET)
+ return;
+ __mm_cid_put(mm, mm_cid_clear_lazy_put(cid));
+}
+
+static inline int __mm_cid_try_get(struct mm_struct *mm)
{
struct cpumask *cpumask;
int cid;

cpumask = mm_cidmask(mm);
- cid = cpumask_first_zero(cpumask);
- if (cid >= nr_cpu_ids)
+ /*
+ * 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(cpumask);
+ if (cid < nr_cpu_ids)
+ break;
+ cpu_relax();
+ }
+ if (cpumask_test_and_set_cpu(cid, cpumask))
return -1;
- __cpumask_set_cpu(cid, cpumask);
return cid;
}

-static inline void mm_cid_put(struct mm_struct *mm, int cid)
+/*
+ * 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 mm_struct *mm)
{
- lockdep_assert_irqs_disabled();
- if (cid < 0)
- return;
- raw_spin_lock(&mm->cid_lock);
- __cpumask_clear_cpu(cid, mm_cidmask(mm));
- raw_spin_unlock(&mm->cid_lock);
+ struct rq *rq = this_rq();
+
+ lockdep_assert_rq_held(rq);
+ __this_cpu_write(mm->pcpu_cid->time, rq->clock);
+}
+
+static inline int __mm_cid_get(struct mm_struct *mm)
+{
+ 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(mm);
+ if (cid >= 0)
+ goto end;
+ raw_spin_lock(&cid_lock);
+ } else {
+ raw_spin_lock(&cid_lock);
+ cid = __mm_cid_try_get(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(mm);
+ cpu_relax();
+ } while (cid < 0);
+ /*
+ * Allocate before clearing use_cid_lock. Only care about
+ * program order because this is for forward progress.
+ */
+ barrier();
+ WRITE_ONCE(use_cid_lock, 0);
+unlock:
+ raw_spin_unlock(&cid_lock);
+end:
+ mm_cid_snapshot_time(mm);
+ return cid;
}

static inline int mm_cid_get(struct mm_struct *mm)
{
- int ret;
+ struct mm_cid __percpu *pcpu_cid = mm->pcpu_cid;
+ struct rq *rq = this_rq();
+ struct cpumask *cpumask;
+ int cid;

- lockdep_assert_irqs_disabled();
- raw_spin_lock(&mm->cid_lock);
- ret = __mm_cid_get(mm);
- raw_spin_unlock(&mm->cid_lock);
- return ret;
+ lockdep_assert_rq_held(rq);
+ cpumask = mm_cidmask(mm);
+ cid = __this_cpu_read(pcpu_cid->cid);
+ if (mm_cid_is_valid(cid)) {
+ mm_cid_snapshot_time(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));
+ }
+ cid = __mm_cid_get(mm);
+ __this_cpu_write(pcpu_cid->cid, cid);
+ return cid;
}

-static inline void switch_mm_cid(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();
+ /*
+ * user -> user transition guarantees a memory barrier through
+ * switch_mm() when current->mm changes. If current->mm is
+ * unchanged, no barrier is needed.
+ */
+ }
if (prev->mm_cid_active) {
- if (next->mm_cid_active && next->mm == prev->mm) {
- /*
- * Context switch between threads in same mm, hand over
- * the mm_cid from prev to next.
- */
- next->mm_cid = prev->mm_cid;
- prev->mm_cid = -1;
- return;
- }
- mm_cid_put(prev->mm, prev->mm_cid);
+ mm_cid_snapshot_time(prev->mm);
+ mm_cid_put_lazy(prev);
prev->mm_cid = -1;
}
if (next->mm_cid_active)
- next->mm_cid = mm_cid_get(next->mm);
+ next->last_mm_cid = next->mm_cid = mm_cid_get(next->mm);
}

#else
static inline void switch_mm_cid(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) { }
#endif

#endif /* _KERNEL_SCHED_SCHED_H */
--
2.25.1


2023-04-20 10:06:32

by Aaron Lu

[permalink] [raw]
Subject: Re: [RFC PATCH v9 2/2] sched: Fix performance regression introduced by mm_cid

On Wed, Apr 19, 2023 at 11:50:12AM -0400, Mathieu Desnoyers wrote:
> Introduce per-mm/cpu current concurrency id (mm_cid) to fix a PostgreSQL
> sysbench regression reported by Aaron Lu.

mm_cid_get() dropped to 5.x% after I disable CONFIG_DEBUG_PREEMPT, using
__this_cpu_X() doesn't help, I suppose that is because __this_cpu_X()
still needs to fetch mm->pcpu_cid.

Annotate mm_cid_get():

│ static inline int mm_cid_get(struct mm_struct *mm)
│ {
0.05 │ push %rbp
0.02 │ mov %rsp,%rbp
│ push %r15
│ push %r14
│ push %r13
│ push %r12
│ push %rbx
0.02 │ sub $0x10,%rsp
│ struct mm_cid __percpu *pcpu_cid = mm->pcpu_cid;
71.30 │ mov 0x60(%rdi),%r12
│ struct cpumask *cpumask;
│ int cid;

│ lockdep_assert_rq_held(rq);
│ cpumask = mm_cidmask(mm);
│ cid = __this_cpu_read(pcpu_cid->cid);
28.44 │ mov %gs:0x8(%r12),%edx
│ if (mm_cid_is_valid(cid)) {


sched_mm_cid_migrate_to() is 4.x% and its annotation :

│ dst_pcpu_cid = per_cpu_ptr(mm->pcpu_cid, cpu_of(dst_rq));
│ mov -0x30(%rbp),%rax
54.53 │ mov 0x60(%r13),%rbx
19.61 │ movslq 0xaf0(%rax),%r15

The reason why accessing mm->pcpu_cid is so costly is still a myth to
me...

BTW, I used below diff to mitigate the incorrect rq lock issue I
described in my reply to v8:

diff --git a/kernel/sched/core.c b/kernel/sched/core.c
index c6e2dd8f4ee3..f16418731866 100644
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -11662,7 +11662,7 @@ void sched_mm_cid_migrate_to(struct rq *dst_rq, struct task_struct *t)
return;
}
/* Move src_cid to dst cpu. */
- mm_cid_snapshot_time(mm);
+ mm_cid_snapshot_time(mm, cpu_of(dst_rq));
WRITE_ONCE(dst_pcpu_cid->cid, src_cid);
}

diff --git a/kernel/sched/sched.h b/kernel/sched/sched.h
index d1d470441422..8b6a0c8ed3d1 100644
--- a/kernel/sched/sched.h
+++ b/kernel/sched/sched.h
@@ -3348,12 +3348,13 @@ static inline int __mm_cid_try_get(struct mm_struct *mm)
* 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 mm_struct *mm)
+static inline void mm_cid_snapshot_time(struct mm_struct *mm, int cpu)
{
- struct rq *rq = this_rq();
+ struct mm_cid *pcpu_cid = per_cpu_ptr(mm->pcpu_cid, cpu);
+ struct rq *rq = cpu_rq(cpu);

lockdep_assert_rq_held(rq);
- __this_cpu_write(mm->pcpu_cid->time, rq->clock);
+ WRITE_ONCE(pcpu_cid->time, rq->clock);
}

static inline int __mm_cid_get(struct mm_struct *mm)
@@ -3404,7 +3405,7 @@ static inline int __mm_cid_get(struct mm_struct *mm)
unlock:
raw_spin_unlock(&cid_lock);
end:
- mm_cid_snapshot_time(mm);
+ mm_cid_snapshot_time(mm, raw_smp_processor_id());
return cid;
}

@@ -3419,7 +3420,7 @@ static inline int mm_cid_get(struct mm_struct *mm)
cpumask = mm_cidmask(mm);
cid = __this_cpu_read(pcpu_cid->cid);
if (mm_cid_is_valid(cid)) {
- mm_cid_snapshot_time(mm);
+ mm_cid_snapshot_time(mm, raw_smp_processor_id());
return cid;
}
if (mm_cid_is_lazy_put(cid)) {
@@ -3467,7 +3468,7 @@ static inline void switch_mm_cid(struct task_struct *prev,
*/
}
if (prev->mm_cid_active) {
- mm_cid_snapshot_time(prev->mm);
+ mm_cid_snapshot_time(prev->mm, raw_smp_processor_id());
mm_cid_put_lazy(prev);
prev->mm_cid = -1;
}

2023-04-20 12:53:01

by Mathieu Desnoyers

[permalink] [raw]
Subject: Re: [RFC PATCH v9 2/2] sched: Fix performance regression introduced by mm_cid

On 2023-04-20 05:56, Aaron Lu wrote:
> On Wed, Apr 19, 2023 at 11:50:12AM -0400, Mathieu Desnoyers wrote:
>> Introduce per-mm/cpu current concurrency id (mm_cid) to fix a PostgreSQL
>> sysbench regression reported by Aaron Lu.
>
> mm_cid_get() dropped to 5.x% after I disable CONFIG_DEBUG_PREEMPT, using
> __this_cpu_X() doesn't help, I suppose that is because __this_cpu_X()
> still needs to fetch mm->pcpu_cid.
>
> Annotate mm_cid_get():
>
> │ static inline int mm_cid_get(struct mm_struct *mm)
> │ {
> 0.05 │ push %rbp
> 0.02 │ mov %rsp,%rbp
> │ push %r15
> │ push %r14
> │ push %r13
> │ push %r12
> │ push %rbx
> 0.02 │ sub $0x10,%rsp
> │ struct mm_cid __percpu *pcpu_cid = mm->pcpu_cid;
> 71.30 │ mov 0x60(%rdi),%r12
> │ struct cpumask *cpumask;
> │ int cid;
> │
> │ lockdep_assert_rq_held(rq);
> │ cpumask = mm_cidmask(mm);
> │ cid = __this_cpu_read(pcpu_cid->cid);
> 28.44 │ mov %gs:0x8(%r12),%edx
> │ if (mm_cid_is_valid(cid)) {
>
>
> sched_mm_cid_migrate_to() is 4.x% and its annotation :
>
> │ dst_pcpu_cid = per_cpu_ptr(mm->pcpu_cid, cpu_of(dst_rq));
> │ mov -0x30(%rbp),%rax
> 54.53 │ mov 0x60(%r13),%rbx
> 19.61 │ movslq 0xaf0(%rax),%r15
>
> The reason why accessing mm->pcpu_cid is so costly is still a myth to
> me...

Then we clearly have another member of mm_struct on the same cache line
as pcpu_cid which is bouncing all over the place and causing
false-sharing. Any idea which field(s) are causing this ?


>
> BTW, I used below diff to mitigate the incorrect rq lock issue I
> described in my reply to v8:

Yes, I'll do something similar in my next version, thanks ! I'll also
drop my patch 1/2 from my patchset because clearly I was looking in the
wrong place.

Thanks,

Mathieu

>
> diff --git a/kernel/sched/core.c b/kernel/sched/core.c
> index c6e2dd8f4ee3..f16418731866 100644
> --- a/kernel/sched/core.c
> +++ b/kernel/sched/core.c
> @@ -11662,7 +11662,7 @@ void sched_mm_cid_migrate_to(struct rq *dst_rq, struct task_struct *t)
> return;
> }
> /* Move src_cid to dst cpu. */
> - mm_cid_snapshot_time(mm);
> + mm_cid_snapshot_time(mm, cpu_of(dst_rq));
> WRITE_ONCE(dst_pcpu_cid->cid, src_cid);
> }
>
> diff --git a/kernel/sched/sched.h b/kernel/sched/sched.h
> index d1d470441422..8b6a0c8ed3d1 100644
> --- a/kernel/sched/sched.h
> +++ b/kernel/sched/sched.h
> @@ -3348,12 +3348,13 @@ static inline int __mm_cid_try_get(struct mm_struct *mm)
> * 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 mm_struct *mm)
> +static inline void mm_cid_snapshot_time(struct mm_struct *mm, int cpu)
> {
> - struct rq *rq = this_rq();
> + struct mm_cid *pcpu_cid = per_cpu_ptr(mm->pcpu_cid, cpu);
> + struct rq *rq = cpu_rq(cpu);
>
> lockdep_assert_rq_held(rq);
> - __this_cpu_write(mm->pcpu_cid->time, rq->clock);
> + WRITE_ONCE(pcpu_cid->time, rq->clock);
> }
>
> static inline int __mm_cid_get(struct mm_struct *mm)
> @@ -3404,7 +3405,7 @@ static inline int __mm_cid_get(struct mm_struct *mm)
> unlock:
> raw_spin_unlock(&cid_lock);
> end:
> - mm_cid_snapshot_time(mm);
> + mm_cid_snapshot_time(mm, raw_smp_processor_id());
> return cid;
> }
>
> @@ -3419,7 +3420,7 @@ static inline int mm_cid_get(struct mm_struct *mm)
> cpumask = mm_cidmask(mm);
> cid = __this_cpu_read(pcpu_cid->cid);
> if (mm_cid_is_valid(cid)) {
> - mm_cid_snapshot_time(mm);
> + mm_cid_snapshot_time(mm, raw_smp_processor_id());
> return cid;
> }
> if (mm_cid_is_lazy_put(cid)) {
> @@ -3467,7 +3468,7 @@ static inline void switch_mm_cid(struct task_struct *prev,
> */
> }
> if (prev->mm_cid_active) {
> - mm_cid_snapshot_time(prev->mm);
> + mm_cid_snapshot_time(prev->mm, raw_smp_processor_id());
> mm_cid_put_lazy(prev);
> prev->mm_cid = -1;
> }

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

2023-04-20 13:00:15

by Aaron Lu

[permalink] [raw]
Subject: Re: [RFC PATCH v9 2/2] sched: Fix performance regression introduced by mm_cid

On Thu, Apr 20, 2023 at 08:41:05AM -0400, Mathieu Desnoyers wrote:
> On 2023-04-20 05:56, Aaron Lu wrote:
> > On Wed, Apr 19, 2023 at 11:50:12AM -0400, Mathieu Desnoyers wrote:
> > > Introduce per-mm/cpu current concurrency id (mm_cid) to fix a PostgreSQL
> > > sysbench regression reported by Aaron Lu.
> >
> > mm_cid_get() dropped to 5.x% after I disable CONFIG_DEBUG_PREEMPT, using
> > __this_cpu_X() doesn't help, I suppose that is because __this_cpu_X()
> > still needs to fetch mm->pcpu_cid.
> >
> > Annotate mm_cid_get():
> >
> > │ static inline int mm_cid_get(struct mm_struct *mm)
> > │ {
> > 0.05 │ push %rbp
> > 0.02 │ mov %rsp,%rbp
> > │ push %r15
> > │ push %r14
> > │ push %r13
> > │ push %r12
> > │ push %rbx
> > 0.02 │ sub $0x10,%rsp
> > │ struct mm_cid __percpu *pcpu_cid = mm->pcpu_cid;
> > 71.30 │ mov 0x60(%rdi),%r12
> > │ struct cpumask *cpumask;
> > │ int cid;
> > │
> > │ lockdep_assert_rq_held(rq);
> > │ cpumask = mm_cidmask(mm);
> > │ cid = __this_cpu_read(pcpu_cid->cid);
> > 28.44 │ mov %gs:0x8(%r12),%edx
> > │ if (mm_cid_is_valid(cid)) {
> >
> >
> > sched_mm_cid_migrate_to() is 4.x% and its annotation :
> >
> > │ dst_pcpu_cid = per_cpu_ptr(mm->pcpu_cid, cpu_of(dst_rq));
> > │ mov -0x30(%rbp),%rax
> > 54.53 │ mov 0x60(%r13),%rbx
> > 19.61 │ movslq 0xaf0(%rax),%r15
> >
> > The reason why accessing mm->pcpu_cid is so costly is still a myth to
> > me...
>
> Then we clearly have another member of mm_struct on the same cache line as
> pcpu_cid which is bouncing all over the place and causing false-sharing. Any
> idea which field(s) are causing this ?

That's my first reaction too but as I said in an earlier reply:
https://lore.kernel.org/lkml/20230419080606.GA4247@ziqianlu-desk2/
I've tried to place pcpu_cid into a dedicate cacheline with no other
fields sharing a cacheline with it in mm_struct but it didn't help...

Thanks,
Aaron

2023-04-20 13:15:03

by Mathieu Desnoyers

[permalink] [raw]
Subject: Re: [RFC PATCH v9 2/2] sched: Fix performance regression introduced by mm_cid

On 2023-04-20 08:50, Aaron Lu wrote:
> On Thu, Apr 20, 2023 at 08:41:05AM -0400, Mathieu Desnoyers wrote:
>> On 2023-04-20 05:56, Aaron Lu wrote:
>>> On Wed, Apr 19, 2023 at 11:50:12AM -0400, Mathieu Desnoyers wrote:
>>>> Introduce per-mm/cpu current concurrency id (mm_cid) to fix a PostgreSQL
>>>> sysbench regression reported by Aaron Lu.
>>>
>>> mm_cid_get() dropped to 5.x% after I disable CONFIG_DEBUG_PREEMPT, using
>>> __this_cpu_X() doesn't help, I suppose that is because __this_cpu_X()
>>> still needs to fetch mm->pcpu_cid.
>>>
>>> Annotate mm_cid_get():
>>>
>>> │ static inline int mm_cid_get(struct mm_struct *mm)
>>> │ {
>>> 0.05 │ push %rbp
>>> 0.02 │ mov %rsp,%rbp
>>> │ push %r15
>>> │ push %r14
>>> │ push %r13
>>> │ push %r12
>>> │ push %rbx
>>> 0.02 │ sub $0x10,%rsp
>>> │ struct mm_cid __percpu *pcpu_cid = mm->pcpu_cid;
>>> 71.30 │ mov 0x60(%rdi),%r12
>>> │ struct cpumask *cpumask;
>>> │ int cid;
>>> │
>>> │ lockdep_assert_rq_held(rq);
>>> │ cpumask = mm_cidmask(mm);
>>> │ cid = __this_cpu_read(pcpu_cid->cid);
>>> 28.44 │ mov %gs:0x8(%r12),%edx
>>> │ if (mm_cid_is_valid(cid)) {
>>>
>>>
>>> sched_mm_cid_migrate_to() is 4.x% and its annotation :
>>>
>>> │ dst_pcpu_cid = per_cpu_ptr(mm->pcpu_cid, cpu_of(dst_rq));
>>> │ mov -0x30(%rbp),%rax
>>> 54.53 │ mov 0x60(%r13),%rbx
>>> 19.61 │ movslq 0xaf0(%rax),%r15
>>>
>>> The reason why accessing mm->pcpu_cid is so costly is still a myth to
>>> me...
>>
>> Then we clearly have another member of mm_struct on the same cache line as
>> pcpu_cid which is bouncing all over the place and causing false-sharing. Any
>> idea which field(s) are causing this ?
>
> That's my first reaction too but as I said in an earlier reply:
> https://lore.kernel.org/lkml/20230419080606.GA4247@ziqianlu-desk2/
> I've tried to place pcpu_cid into a dedicate cacheline with no other
> fields sharing a cacheline with it in mm_struct but it didn't help...

I see two possible culprits there:

1) The mm_struct pcpu_cid field is suffering from false-sharing. I would be
interested to look at your attempt to move it to a separate cache line to
try to figure out what is going on.

2) (Maybe?) The task_struct mm field is suffering from false-sharing and stalling
the next instruction which needs to use its value to fetch the mm->pcpu_cid
field. We could try moving the task_struct mm field into its own cache line to
see if it helps.

Thanks,

Mathieu

>
> Thanks,
> Aaron

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

2023-04-20 13:41:32

by Aaron Lu

[permalink] [raw]
Subject: Re: [RFC PATCH v9 2/2] sched: Fix performance regression introduced by mm_cid

On Thu, Apr 20, 2023 at 09:10:35AM -0400, Mathieu Desnoyers wrote:
> On 2023-04-20 08:50, Aaron Lu wrote:
> > On Thu, Apr 20, 2023 at 08:41:05AM -0400, Mathieu Desnoyers wrote:
> > > On 2023-04-20 05:56, Aaron Lu wrote:
> > > > On Wed, Apr 19, 2023 at 11:50:12AM -0400, Mathieu Desnoyers wrote:
> > > > > Introduce per-mm/cpu current concurrency id (mm_cid) to fix a PostgreSQL
> > > > > sysbench regression reported by Aaron Lu.
> > > >
> > > > mm_cid_get() dropped to 5.x% after I disable CONFIG_DEBUG_PREEMPT, using
> > > > __this_cpu_X() doesn't help, I suppose that is because __this_cpu_X()
> > > > still needs to fetch mm->pcpu_cid.
> > > >
> > > > Annotate mm_cid_get():
> > > >
> > > > │ static inline int mm_cid_get(struct mm_struct *mm)
> > > > │ {
> > > > 0.05 │ push %rbp
> > > > 0.02 │ mov %rsp,%rbp
> > > > │ push %r15
> > > > │ push %r14
> > > > │ push %r13
> > > > │ push %r12
> > > > │ push %rbx
> > > > 0.02 │ sub $0x10,%rsp
> > > > │ struct mm_cid __percpu *pcpu_cid = mm->pcpu_cid;
> > > > 71.30 │ mov 0x60(%rdi),%r12
> > > > │ struct cpumask *cpumask;
> > > > │ int cid;
> > > > │
> > > > │ lockdep_assert_rq_held(rq);
> > > > │ cpumask = mm_cidmask(mm);
> > > > │ cid = __this_cpu_read(pcpu_cid->cid);
> > > > 28.44 │ mov %gs:0x8(%r12),%edx
> > > > │ if (mm_cid_is_valid(cid)) {
> > > >
> > > >
> > > > sched_mm_cid_migrate_to() is 4.x% and its annotation :
> > > >
> > > > │ dst_pcpu_cid = per_cpu_ptr(mm->pcpu_cid, cpu_of(dst_rq));
> > > > │ mov -0x30(%rbp),%rax
> > > > 54.53 │ mov 0x60(%r13),%rbx
> > > > 19.61 │ movslq 0xaf0(%rax),%r15
> > > >
> > > > The reason why accessing mm->pcpu_cid is so costly is still a myth to
> > > > me...
> > >
> > > Then we clearly have another member of mm_struct on the same cache line as
> > > pcpu_cid which is bouncing all over the place and causing false-sharing. Any
> > > idea which field(s) are causing this ?
> >
> > That's my first reaction too but as I said in an earlier reply:
> > https://lore.kernel.org/lkml/20230419080606.GA4247@ziqianlu-desk2/
> > I've tried to place pcpu_cid into a dedicate cacheline with no other
> > fields sharing a cacheline with it in mm_struct but it didn't help...
>
> I see two possible culprits there:
>
> 1) The mm_struct pcpu_cid field is suffering from false-sharing. I would be
> interested to look at your attempt to move it to a separate cache line to
> try to figure out what is going on.

Brain damaged...my mistake, I only made sure its following fields not
share the same cacheline but forgot to exclude its preceding fields and
turned out it's one(some?) of the preceeding fields that caused false
sharing. When I did:

diff --git a/include/linux/mm_types.h b/include/linux/mm_types.h
index 5eab61156f0e..a6f9d815991c 100644
--- a/include/linux/mm_types.h
+++ b/include/linux/mm_types.h
@@ -606,6 +606,7 @@ struct mm_struct {
*/
atomic_t mm_count;
#ifdef CONFIG_SCHED_MM_CID
+ CACHELINE_PADDING(_pad1_);
/**
* @pcpu_cid: Per-cpu current cid.
*
mm_cid_get() dropped to 0.0x% when running hackbench :-)

sched_mm_cid_migrate_to() is about 4% with most cycles spent on
accessing mm->mm_users:

│ dst_cid = READ_ONCE(dst_pcpu_cid->cid);
0.03 │ mov 0x8(%r12),%r15d
│ if (!mm_cid_is_unset(dst_cid) &&
0.07 │ cmp $0xffffffff,%r15d
│ ↓ je 87
│ arch_atomic_read():
│ {
│ /*
│ * Note for KASAN: we deliberately don't use READ_ONCE_NOCHECK() here,
│ * it's non-inlined function that increases binary size and stack usage.
│ */
│ return __READ_ONCE((v)->counter);
76.13 │ mov 0x54(%r13),%eax
│ sched_mm_cid_migrate_to():
│ cmp %eax,0x410(%rdx)
21.71 │ ↓ jle 1d8
│ atomic_read(&mm->mm_users) >= t->nr_cpus_allowed)

With this info, it should be mm_users that caused false sharing for
pcpu_cid previously. Looks like mm_users is bouncing.

Thanks,
Aaron

2023-04-20 14:05:33

by Mathieu Desnoyers

[permalink] [raw]
Subject: Re: [RFC PATCH v9 2/2] sched: Fix performance regression introduced by mm_cid

On 2023-04-20 09:35, Aaron Lu wrote:
[...]
>>>>
>>>> Then we clearly have another member of mm_struct on the same cache line as
>>>> pcpu_cid which is bouncing all over the place and causing false-sharing. Any
>>>> idea which field(s) are causing this ?
>>>
>>> That's my first reaction too but as I said in an earlier reply:
>>> https://lore.kernel.org/lkml/20230419080606.GA4247@ziqianlu-desk2/
>>> I've tried to place pcpu_cid into a dedicate cacheline with no other
>>> fields sharing a cacheline with it in mm_struct but it didn't help...
>>
>> I see two possible culprits there:
>>
>> 1) The mm_struct pcpu_cid field is suffering from false-sharing. I would be
>> interested to look at your attempt to move it to a separate cache line to
>> try to figure out what is going on.
>
> Brain damaged...my mistake, I only made sure its following fields not
> share the same cacheline but forgot to exclude its preceding fields and
> turned out it's one(some?) of the preceeding fields that caused false
> sharing. When I did:
>
> diff --git a/include/linux/mm_types.h b/include/linux/mm_types.h
> index 5eab61156f0e..a6f9d815991c 100644
> --- a/include/linux/mm_types.h
> +++ b/include/linux/mm_types.h
> @@ -606,6 +606,7 @@ struct mm_struct {
> */
> atomic_t mm_count;
> #ifdef CONFIG_SCHED_MM_CID
> + CACHELINE_PADDING(_pad1_);
> /**
> * @pcpu_cid: Per-cpu current cid.
> *
> mm_cid_get() dropped to 0.0x% when running hackbench :-)

Now we are talking! :)

>
> sched_mm_cid_migrate_to() is about 4% with most cycles spent on
> accessing mm->mm_users:
>
> │ dst_cid = READ_ONCE(dst_pcpu_cid->cid);
> 0.03 │ mov 0x8(%r12),%r15d
> │ if (!mm_cid_is_unset(dst_cid) &&
> 0.07 │ cmp $0xffffffff,%r15d
> │ ↓ je 87
> │ arch_atomic_read():
> │ {
> │ /*
> │ * Note for KASAN: we deliberately don't use READ_ONCE_NOCHECK() here,
> │ * it's non-inlined function that increases binary size and stack usage.
> │ */
> │ return __READ_ONCE((v)->counter);
> 76.13 │ mov 0x54(%r13),%eax
> │ sched_mm_cid_migrate_to():
> │ cmp %eax,0x410(%rdx)
> 21.71 │ ↓ jle 1d8
> │ atomic_read(&mm->mm_users) >= t->nr_cpus_allowed)
>
> With this info, it should be mm_users that caused false sharing for
> pcpu_cid previously. Looks like mm_users is bouncing.

I suspect that the culprit here is mm_count rather than mm_users.
mm_users just happens to share the same cache line as mm_count.

mm_count is incremented/decremented with mmgrab()/mmdrop() during
context switch.

This is likely causing other issues, for instance, the
membarrier_state field is AFAIR read-mostly, used for
membarrier_mm_sync_core_before_usermode() to issue core
sync before every return to usermode if needed.

Other things like mm_struct pgd pointer appear to be likely
read-mostly variables.

I suspect it's mm_count which should be moved to its own cache line
to eliminate false-sharing with all the other read-mostly fields
of mm_struct.

Thoughts ?

Thanks,

Mathieu

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

2023-04-20 14:22:56

by Aaron Lu

[permalink] [raw]
Subject: Re: [RFC PATCH v9 2/2] sched: Fix performance regression introduced by mm_cid

On Thu, Apr 20, 2023 at 09:54:29AM -0400, Mathieu Desnoyers wrote:
> On 2023-04-20 09:35, Aaron Lu wrote:
> [...]
> > > > >
> > > > > Then we clearly have another member of mm_struct on the same cache line as
> > > > > pcpu_cid which is bouncing all over the place and causing false-sharing. Any
> > > > > idea which field(s) are causing this ?
> > > >
> > > > That's my first reaction too but as I said in an earlier reply:
> > > > https://lore.kernel.org/lkml/20230419080606.GA4247@ziqianlu-desk2/
> > > > I've tried to place pcpu_cid into a dedicate cacheline with no other
> > > > fields sharing a cacheline with it in mm_struct but it didn't help...
> > >
> > > I see two possible culprits there:
> > >
> > > 1) The mm_struct pcpu_cid field is suffering from false-sharing. I would be
> > > interested to look at your attempt to move it to a separate cache line to
> > > try to figure out what is going on.
> >
> > Brain damaged...my mistake, I only made sure its following fields not
> > share the same cacheline but forgot to exclude its preceding fields and
> > turned out it's one(some?) of the preceeding fields that caused false
> > sharing. When I did:
> >
> > diff --git a/include/linux/mm_types.h b/include/linux/mm_types.h
> > index 5eab61156f0e..a6f9d815991c 100644
> > --- a/include/linux/mm_types.h
> > +++ b/include/linux/mm_types.h
> > @@ -606,6 +606,7 @@ struct mm_struct {
> > */
> > atomic_t mm_count;
> > #ifdef CONFIG_SCHED_MM_CID
> > + CACHELINE_PADDING(_pad1_);
> > /**
> > * @pcpu_cid: Per-cpu current cid.
> > *
> > mm_cid_get() dropped to 0.0x% when running hackbench :-)
>
> Now we are talking! :)
>
> >
> > sched_mm_cid_migrate_to() is about 4% with most cycles spent on
> > accessing mm->mm_users:
> >
> > │ dst_cid = READ_ONCE(dst_pcpu_cid->cid);
> > 0.03 │ mov 0x8(%r12),%r15d
> > │ if (!mm_cid_is_unset(dst_cid) &&
> > 0.07 │ cmp $0xffffffff,%r15d
> > │ ↓ je 87
> > │ arch_atomic_read():
> > │ {
> > │ /*
> > │ * Note for KASAN: we deliberately don't use READ_ONCE_NOCHECK() here,
> > │ * it's non-inlined function that increases binary size and stack usage.
> > │ */
> > │ return __READ_ONCE((v)->counter);
> > 76.13 │ mov 0x54(%r13),%eax
> > │ sched_mm_cid_migrate_to():
> > │ cmp %eax,0x410(%rdx)
> > 21.71 │ ↓ jle 1d8
> > │ atomic_read(&mm->mm_users) >= t->nr_cpus_allowed)
> >
> > With this info, it should be mm_users that caused false sharing for
> > pcpu_cid previously. Looks like mm_users is bouncing.
>
> I suspect that the culprit here is mm_count rather than mm_users. mm_users
> just happens to share the same cache line as mm_count.
>
> mm_count is incremented/decremented with mmgrab()/mmdrop() during
> context switch.
>
> This is likely causing other issues, for instance, the
> membarrier_state field is AFAIR read-mostly, used for
> membarrier_mm_sync_core_before_usermode() to issue core
> sync before every return to usermode if needed.
>
> Other things like mm_struct pgd pointer appear to be likely
> read-mostly variables.
>
> I suspect it's mm_count which should be moved to its own cache line
> to eliminate false-sharing with all the other read-mostly fields
> of mm_struct.
>
> Thoughts ?

Makes sesne, I was wondering where the write side of mm_user is. Let me
see how that goes by placing mm_count aside from other read mostly fields.

2023-04-20 14:38:41

by Mathieu Desnoyers

[permalink] [raw]
Subject: Re: [RFC PATCH v9 2/2] sched: Fix performance regression introduced by mm_cid

On 2023-04-20 09:54, Mathieu Desnoyers wrote:
> On 2023-04-20 09:35, Aaron Lu wrote:
> [...]
>>>>>
>>>>> Then we clearly have another member of mm_struct on the same cache
>>>>> line as
>>>>> pcpu_cid which is bouncing all over the place and causing
>>>>> false-sharing. Any
>>>>> idea which field(s) are causing this ?
>>>>
>>>> That's my first reaction too but as I said in an earlier reply:
>>>> https://lore.kernel.org/lkml/20230419080606.GA4247@ziqianlu-desk2/
>>>> I've tried to place pcpu_cid into a dedicate cacheline with no other
>>>> fields sharing a cacheline with it in mm_struct but it didn't help...
>>>
>>> I see two possible culprits there:
>>>
>>> 1) The mm_struct pcpu_cid field is suffering from false-sharing. I
>>> would be
>>>     interested to look at your attempt to move it to a separate cache
>>> line to
>>>     try to figure out what is going on.
>>
>> Brain damaged...my mistake, I only made sure its following fields not
>> share the same cacheline but forgot to exclude its preceding fields and
>> turned out it's one(some?) of the preceeding fields that caused false
>> sharing. When I did:
>>
>> diff --git a/include/linux/mm_types.h b/include/linux/mm_types.h
>> index 5eab61156f0e..a6f9d815991c 100644
>> --- a/include/linux/mm_types.h
>> +++ b/include/linux/mm_types.h
>> @@ -606,6 +606,7 @@ struct mm_struct {
>>                   */
>>                  atomic_t mm_count;
>>   #ifdef CONFIG_SCHED_MM_CID
>> +               CACHELINE_PADDING(_pad1_);
>>                  /**
>>                   * @pcpu_cid: Per-cpu current cid.
>>                   *
>> mm_cid_get() dropped to 0.0x% when running hackbench :-)
>
> Now we are talking! :)
>
>>
>> sched_mm_cid_migrate_to() is about 4% with most cycles spent on
>> accessing mm->mm_users:
>>
>>         │     dst_cid = READ_ONCE(dst_pcpu_cid->cid);
>>    0.03 │       mov     0x8(%r12),%r15d
>>         │     if (!mm_cid_is_unset(dst_cid) &&
>>    0.07 │       cmp     $0xffffffff,%r15d
>>         │     ↓ je      87
>>         │     arch_atomic_read():
>>         │     {
>>         │     /*
>>         │     * Note for KASAN: we deliberately don't use
>> READ_ONCE_NOCHECK() here,
>>         │     * it's non-inlined function that increases binary size
>> and stack usage.
>>         │     */
>>         │     return __READ_ONCE((v)->counter);
>>   76.13 │       mov     0x54(%r13),%eax
>>         │     sched_mm_cid_migrate_to():
>>         │       cmp     %eax,0x410(%rdx)
>>   21.71 │     ↓ jle     1d8
>>         │     atomic_read(&mm->mm_users) >= t->nr_cpus_allowed)
>>
>> With this info, it should be mm_users that caused false sharing for
>> pcpu_cid previously. Looks like mm_users is bouncing.
>
> I suspect that the culprit here is mm_count rather than mm_users.
> mm_users just happens to share the same cache line as mm_count.
>
> mm_count is incremented/decremented with mmgrab()/mmdrop() during
> context switch.
>
> This is likely causing other issues, for instance, the
> membarrier_state field is AFAIR read-mostly, used for
> membarrier_mm_sync_core_before_usermode() to issue core
> sync before every return to usermode if needed.
>
> Other things like mm_struct pgd pointer appear to be likely
> read-mostly variables.
>
> I suspect it's mm_count which should be moved to its own cache line
> to eliminate false-sharing with all the other read-mostly fields
> of mm_struct.

I have prepared a patch which moves the mm_count field into its own
cache line, but after a quick discussion with Peter Zijlstra, it appears
that the work on lazy-tlb refcounting currently in

https://git.kernel.org/pub/scm/linux/kernel/git/peterz/queue.git/log/?h=x86/lazy

will take care of this by entirely removing the reference counting for lazy TLB.

So with this, I suspect we are as good as we can be in terms of near-zero
footprint for the mm_cid feature, right ?

diff --git a/include/linux/mm_types.h b/include/linux/mm_types.h
index a57e6ae78e65..f740fa447df1 100644
--- a/include/linux/mm_types.h
+++ b/include/linux/mm_types.h
@@ -553,6 +553,21 @@ struct vm_area_struct {
struct kioctx_table;
struct mm_struct {
struct {
+ /*
+ * Fields which are often written to are placed in a separate
+ * cache line.
+ */
+ struct {
+ /**
+ * @mm_count: The number of references to &struct mm_struct
+ * (@mm_users count as 1).
+ *
+ * Use mmgrab()/mmdrop() to modify. When this drops to 0, the
+ * &struct mm_struct is freed.
+ */
+ atomic_t mm_count;
+ } ____cacheline_aligned_in_smp;
+
struct maple_tree mm_mt;
#ifdef CONFIG_MMU
unsigned long (*get_unmapped_area) (struct file *filp,
@@ -590,14 +605,6 @@ struct mm_struct {
*/
atomic_t mm_users;

- /**
- * @mm_count: The number of references to &struct mm_struct
- * (@mm_users count as 1).
- *
- * Use mmgrab()/mmdrop() to modify. When this drops to 0, the
- * &struct mm_struct is freed.
- */
- atomic_t mm_count;
#ifdef CONFIG_SCHED_MM_CID
/**
* @cid_lock: Protect cid bitmap updates vs lookups.




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

2023-04-20 14:43:08

by Aaron Lu

[permalink] [raw]
Subject: Re: [RFC PATCH v9 2/2] sched: Fix performance regression introduced by mm_cid

On Thu, Apr 20, 2023 at 10:18:52PM +0800, Aaron Lu wrote:
> On Thu, Apr 20, 2023 at 09:54:29AM -0400, Mathieu Desnoyers wrote:
> > On 2023-04-20 09:35, Aaron Lu wrote:
> > [...]
> > > > > >
> > > > > > Then we clearly have another member of mm_struct on the same cache line as
> > > > > > pcpu_cid which is bouncing all over the place and causing false-sharing. Any
> > > > > > idea which field(s) are causing this ?
> > > > >
> > > > > That's my first reaction too but as I said in an earlier reply:
> > > > > https://lore.kernel.org/lkml/20230419080606.GA4247@ziqianlu-desk2/
> > > > > I've tried to place pcpu_cid into a dedicate cacheline with no other
> > > > > fields sharing a cacheline with it in mm_struct but it didn't help...
> > > >
> > > > I see two possible culprits there:
> > > >
> > > > 1) The mm_struct pcpu_cid field is suffering from false-sharing. I would be
> > > > interested to look at your attempt to move it to a separate cache line to
> > > > try to figure out what is going on.
> > >
> > > Brain damaged...my mistake, I only made sure its following fields not
> > > share the same cacheline but forgot to exclude its preceding fields and
> > > turned out it's one(some?) of the preceeding fields that caused false
> > > sharing. When I did:
> > >
> > > diff --git a/include/linux/mm_types.h b/include/linux/mm_types.h
> > > index 5eab61156f0e..a6f9d815991c 100644
> > > --- a/include/linux/mm_types.h
> > > +++ b/include/linux/mm_types.h
> > > @@ -606,6 +606,7 @@ struct mm_struct {
> > > */
> > > atomic_t mm_count;
> > > #ifdef CONFIG_SCHED_MM_CID
> > > + CACHELINE_PADDING(_pad1_);
> > > /**
> > > * @pcpu_cid: Per-cpu current cid.
> > > *
> > > mm_cid_get() dropped to 0.0x% when running hackbench :-)
> >
> > Now we are talking! :)
> >
> > >
> > > sched_mm_cid_migrate_to() is about 4% with most cycles spent on
> > > accessing mm->mm_users:
> > >
> > > │ dst_cid = READ_ONCE(dst_pcpu_cid->cid);
> > > 0.03 │ mov 0x8(%r12),%r15d
> > > │ if (!mm_cid_is_unset(dst_cid) &&
> > > 0.07 │ cmp $0xffffffff,%r15d
> > > │ ↓ je 87
> > > │ arch_atomic_read():
> > > │ {
> > > │ /*
> > > │ * Note for KASAN: we deliberately don't use READ_ONCE_NOCHECK() here,
> > > │ * it's non-inlined function that increases binary size and stack usage.
> > > │ */
> > > │ return __READ_ONCE((v)->counter);
> > > 76.13 │ mov 0x54(%r13),%eax
> > > │ sched_mm_cid_migrate_to():
> > > │ cmp %eax,0x410(%rdx)
> > > 21.71 │ ↓ jle 1d8
> > > │ atomic_read(&mm->mm_users) >= t->nr_cpus_allowed)
> > >
> > > With this info, it should be mm_users that caused false sharing for
> > > pcpu_cid previously. Looks like mm_users is bouncing.
> >
> > I suspect that the culprit here is mm_count rather than mm_users. mm_users
> > just happens to share the same cache line as mm_count.
> >
> > mm_count is incremented/decremented with mmgrab()/mmdrop() during
> > context switch.
> >
> > This is likely causing other issues, for instance, the
> > membarrier_state field is AFAIR read-mostly, used for
> > membarrier_mm_sync_core_before_usermode() to issue core
> > sync before every return to usermode if needed.
> >
> > Other things like mm_struct pgd pointer appear to be likely
> > read-mostly variables.
> >
> > I suspect it's mm_count which should be moved to its own cache line
> > to eliminate false-sharing with all the other read-mostly fields
> > of mm_struct.
> >
> > Thoughts ?
>
> Makes sesne, I was wondering where the write side of mm_user is. Let me
> see how that goes by placing mm_count aside from other read mostly fields.

With the following naive padding for mm_count:

diff --git a/include/linux/mm_types.h b/include/linux/mm_types.h
index 5eab61156f0e..866696e2d83e 100644
--- a/include/linux/mm_types.h
+++ b/include/linux/mm_types.h
@@ -604,7 +604,9 @@ struct mm_struct {
* Use mmgrab()/mmdrop() to modify. When this drops to 0, the
* &struct mm_struct is freed.
*/
+ CACHELINE_PADDING(_pad1_);
atomic_t mm_count;
+ CACHELINE_PADDING(_pad2_);
#ifdef CONFIG_SCHED_MM_CID
/**
* @pcpu_cid: Per-cpu current cid.

mm_cid_get() is about 0.1% and sched_mm_cid_migrate_to() is about 0.2%
for hackbench on SPR :-)

Thanks,
Aaron

2023-04-20 15:19:07

by Mathieu Desnoyers

[permalink] [raw]
Subject: Re: [RFC PATCH v9 2/2] sched: Fix performance regression introduced by mm_cid

On 2023-04-20 10:39, Aaron Lu wrote:
> On Thu, Apr 20, 2023 at 10:18:52PM +0800, Aaron Lu wrote:
>> On Thu, Apr 20, 2023 at 09:54:29AM -0400, Mathieu Desnoyers wrote:
>>> On 2023-04-20 09:35, Aaron Lu wrote:
>>> [...]
>>>>>>>
>>>>>>> Then we clearly have another member of mm_struct on the same cache line as
>>>>>>> pcpu_cid which is bouncing all over the place and causing false-sharing. Any
>>>>>>> idea which field(s) are causing this ?
>>>>>>
>>>>>> That's my first reaction too but as I said in an earlier reply:
>>>>>> https://lore.kernel.org/lkml/20230419080606.GA4247@ziqianlu-desk2/
>>>>>> I've tried to place pcpu_cid into a dedicate cacheline with no other
>>>>>> fields sharing a cacheline with it in mm_struct but it didn't help...
>>>>>
>>>>> I see two possible culprits there:
>>>>>
>>>>> 1) The mm_struct pcpu_cid field is suffering from false-sharing. I would be
>>>>> interested to look at your attempt to move it to a separate cache line to
>>>>> try to figure out what is going on.
>>>>
>>>> Brain damaged...my mistake, I only made sure its following fields not
>>>> share the same cacheline but forgot to exclude its preceding fields and
>>>> turned out it's one(some?) of the preceeding fields that caused false
>>>> sharing. When I did:
>>>>
>>>> diff --git a/include/linux/mm_types.h b/include/linux/mm_types.h
>>>> index 5eab61156f0e..a6f9d815991c 100644
>>>> --- a/include/linux/mm_types.h
>>>> +++ b/include/linux/mm_types.h
>>>> @@ -606,6 +606,7 @@ struct mm_struct {
>>>> */
>>>> atomic_t mm_count;
>>>> #ifdef CONFIG_SCHED_MM_CID
>>>> + CACHELINE_PADDING(_pad1_);
>>>> /**
>>>> * @pcpu_cid: Per-cpu current cid.
>>>> *
>>>> mm_cid_get() dropped to 0.0x% when running hackbench :-)
>>>
>>> Now we are talking! :)
>>>
>>>>
>>>> sched_mm_cid_migrate_to() is about 4% with most cycles spent on
>>>> accessing mm->mm_users:
>>>>
>>>> │ dst_cid = READ_ONCE(dst_pcpu_cid->cid);
>>>> 0.03 │ mov 0x8(%r12),%r15d
>>>> │ if (!mm_cid_is_unset(dst_cid) &&
>>>> 0.07 │ cmp $0xffffffff,%r15d
>>>> │ ↓ je 87
>>>> │ arch_atomic_read():
>>>> │ {
>>>> │ /*
>>>> │ * Note for KASAN: we deliberately don't use READ_ONCE_NOCHECK() here,
>>>> │ * it's non-inlined function that increases binary size and stack usage.
>>>> │ */
>>>> │ return __READ_ONCE((v)->counter);
>>>> 76.13 │ mov 0x54(%r13),%eax
>>>> │ sched_mm_cid_migrate_to():
>>>> │ cmp %eax,0x410(%rdx)
>>>> 21.71 │ ↓ jle 1d8
>>>> │ atomic_read(&mm->mm_users) >= t->nr_cpus_allowed)
>>>>
>>>> With this info, it should be mm_users that caused false sharing for
>>>> pcpu_cid previously. Looks like mm_users is bouncing.
>>>
>>> I suspect that the culprit here is mm_count rather than mm_users. mm_users
>>> just happens to share the same cache line as mm_count.
>>>
>>> mm_count is incremented/decremented with mmgrab()/mmdrop() during
>>> context switch.
>>>
>>> This is likely causing other issues, for instance, the
>>> membarrier_state field is AFAIR read-mostly, used for
>>> membarrier_mm_sync_core_before_usermode() to issue core
>>> sync before every return to usermode if needed.
>>>
>>> Other things like mm_struct pgd pointer appear to be likely
>>> read-mostly variables.
>>>
>>> I suspect it's mm_count which should be moved to its own cache line
>>> to eliminate false-sharing with all the other read-mostly fields
>>> of mm_struct.
>>>
>>> Thoughts ?
>>
>> Makes sesne, I was wondering where the write side of mm_user is. Let me
>> see how that goes by placing mm_count aside from other read mostly fields.
>
> With the following naive padding for mm_count:
>
> diff --git a/include/linux/mm_types.h b/include/linux/mm_types.h
> index 5eab61156f0e..866696e2d83e 100644
> --- a/include/linux/mm_types.h
> +++ b/include/linux/mm_types.h
> @@ -604,7 +604,9 @@ struct mm_struct {
> * Use mmgrab()/mmdrop() to modify. When this drops to 0, the
> * &struct mm_struct is freed.
> */
> + CACHELINE_PADDING(_pad1_);
> atomic_t mm_count;
> + CACHELINE_PADDING(_pad2_);
> #ifdef CONFIG_SCHED_MM_CID
> /**
> * @pcpu_cid: Per-cpu current cid.
>
> mm_cid_get() is about 0.1% and sched_mm_cid_migrate_to() is about 0.2%
> for hackbench on SPR :-)

Allright, then our work is done here. I'm popping the champagne.

I'm doing some more testing on v10 which includes a fix for the
time-snapshot wrong runqueue, and I'll post it today.

Thanks a lot for your help ! :-)

Mathieu


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