summaryrefslogtreecommitdiff
path: root/kernel/sched/sched.h
diff options
context:
space:
mode:
Diffstat (limited to 'kernel/sched/sched.h')
-rw-r--r--kernel/sched/sched.h666
1 files changed, 372 insertions, 294 deletions
diff --git a/kernel/sched/sched.h b/kernel/sched/sched.h
index 1f5d07067f60..8590113e4a60 100644
--- a/kernel/sched/sched.h
+++ b/kernel/sched/sched.h
@@ -5,6 +5,7 @@
#ifndef _KERNEL_SCHED_SCHED_H
#define _KERNEL_SCHED_SCHED_H
+#include <linux/prandom.h>
#include <linux/sched/affinity.h>
#include <linux/sched/autogroup.h>
#include <linux/sched/cpufreq.h>
@@ -20,7 +21,6 @@
#include <linux/sched/task_flags.h>
#include <linux/sched/task.h>
#include <linux/sched/topology.h>
-
#include <linux/atomic.h>
#include <linux/bitmap.h>
#include <linux/bug.h>
@@ -405,6 +405,7 @@ extern s64 dl_scaled_delta_exec(struct rq *rq, struct sched_dl_entity *dl_se, s6
* naturally thottled to once per period, avoiding high context switch
* workloads from spamming the hrtimer program/cancel paths.
*/
+extern void dl_server_update_idle(struct sched_dl_entity *dl_se, s64 delta_exec);
extern void dl_server_update(struct sched_dl_entity *dl_se, s64 delta_exec);
extern void dl_server_start(struct sched_dl_entity *dl_se);
extern void dl_server_stop(struct sched_dl_entity *dl_se);
@@ -412,8 +413,6 @@ extern void dl_server_init(struct sched_dl_entity *dl_se, struct rq *rq,
dl_server_pick_f pick_task);
extern void sched_init_dl_servers(void);
-extern void dl_server_update_idle_time(struct rq *rq,
- struct task_struct *p);
extern void fair_server_init(struct rq *rq);
extern void __dl_server_attach_root(struct sched_dl_entity *dl_se, struct rq *rq);
extern int dl_server_apply_params(struct sched_dl_entity *dl_se,
@@ -682,10 +681,10 @@ struct cfs_rq {
s64 avg_vruntime;
u64 avg_load;
- u64 min_vruntime;
+ u64 zero_vruntime;
#ifdef CONFIG_SCHED_CORE
unsigned int forceidle_seq;
- u64 min_vruntime_fi;
+ u64 zero_vruntime_fi;
#endif
struct rb_root_cached tasks_timeline;
@@ -780,10 +779,10 @@ enum scx_rq_flags {
*/
SCX_RQ_ONLINE = 1 << 0,
SCX_RQ_CAN_STOP_TICK = 1 << 1,
- SCX_RQ_BAL_PENDING = 1 << 2, /* balance hasn't run yet */
SCX_RQ_BAL_KEEP = 1 << 3, /* balance decided to keep current */
SCX_RQ_BYPASSING = 1 << 4,
SCX_RQ_CLK_VALID = 1 << 5, /* RQ clock is fresh and valid */
+ SCX_RQ_BAL_CB_PENDING = 1 << 6, /* must queue a cb after dispatching */
SCX_RQ_IN_WAKEUP = 1 << 16,
SCX_RQ_IN_BALANCE = 1 << 17,
@@ -1119,6 +1118,8 @@ struct rq {
/* runqueue lock: */
raw_spinlock_t __lock;
+ /* Per class runqueue modification mask; bits in class order. */
+ unsigned int queue_mask;
unsigned int nr_running;
#ifdef CONFIG_NUMA_BALANCING
unsigned int nr_numa_running;
@@ -1348,6 +1349,12 @@ static inline bool is_migration_disabled(struct task_struct *p)
}
DECLARE_PER_CPU_SHARED_ALIGNED(struct rq, runqueues);
+DECLARE_PER_CPU(struct rnd_state, sched_rnd_state);
+
+static inline u32 sched_rng(void)
+{
+ return prandom_u32_state(this_cpu_ptr(&sched_rnd_state));
+}
#define cpu_rq(cpu) (&per_cpu(runqueues, (cpu)))
#define this_rq() this_cpu_ptr(&runqueues)
@@ -1431,6 +1438,9 @@ static inline bool sched_core_cookie_match(struct rq *rq, struct task_struct *p)
if (!sched_core_enabled(rq))
return true;
+ if (rq->core->core_cookie == p->core_cookie)
+ return true;
+
for_each_cpu(cpu, cpu_smt_mask(cpu_of(rq))) {
if (!available_idle_cpu(cpu)) {
idle_core = false;
@@ -1442,7 +1452,7 @@ static inline bool sched_core_cookie_match(struct rq *rq, struct task_struct *p)
* A CPU in an idle core is always the best choice for tasks with
* cookies.
*/
- return idle_core || rq->core->core_cookie == p->core_cookie;
+ return idle_core;
}
static inline bool sched_group_cookie_match(struct rq *rq,
@@ -1826,7 +1836,8 @@ struct rq *task_rq_lock(struct task_struct *p, struct rq_flags *rf)
__acquires(p->pi_lock)
__acquires(rq->lock);
-static inline void __task_rq_unlock(struct rq *rq, struct rq_flags *rf)
+static inline void
+__task_rq_unlock(struct rq *rq, struct task_struct *p, struct rq_flags *rf)
__releases(rq->lock)
{
rq_unpin_lock(rq, rf);
@@ -1838,8 +1849,7 @@ task_rq_unlock(struct rq *rq, struct task_struct *p, struct rq_flags *rf)
__releases(rq->lock)
__releases(p->pi_lock)
{
- rq_unpin_lock(rq, rf);
- raw_spin_rq_unlock(rq);
+ __task_rq_unlock(rq, p, rf);
raw_spin_unlock_irqrestore(&p->pi_lock, rf->flags);
}
@@ -1848,6 +1858,11 @@ DEFINE_LOCK_GUARD_1(task_rq_lock, struct task_struct,
task_rq_unlock(_T->rq, _T->lock, &_T->rf),
struct rq *rq; struct rq_flags rf)
+DEFINE_LOCK_GUARD_1(__task_rq_lock, struct task_struct,
+ _T->rq = __task_rq_lock(_T->lock, &_T->rf),
+ __task_rq_unlock(_T->rq, _T->lock, &_T->rf),
+ struct rq *rq; struct rq_flags rf)
+
static inline void rq_lock_irqsave(struct rq *rq, struct rq_flags *rf)
__acquires(rq->lock)
{
@@ -2208,6 +2223,7 @@ static inline void __set_task_cpu(struct task_struct *p, unsigned int cpu)
smp_wmb();
WRITE_ONCE(task_thread_info(p)->cpu, cpu);
p->wake_cpu = cpu;
+ rseq_sched_set_ids_changed(p);
#endif /* CONFIG_SMP */
}
@@ -2341,8 +2357,7 @@ extern const u32 sched_prio_to_wmult[40];
/*
* {de,en}queue flags:
*
- * DEQUEUE_SLEEP - task is no longer runnable
- * ENQUEUE_WAKEUP - task just became runnable
+ * SLEEP/WAKEUP - task is no-longer/just-became runnable
*
* SAVE/RESTORE - an otherwise spurious dequeue/enqueue, done to ensure tasks
* are in a known state which allows modification. Such pairs
@@ -2355,34 +2370,46 @@ extern const u32 sched_prio_to_wmult[40];
*
* MIGRATION - p->on_rq == TASK_ON_RQ_MIGRATING (used for DEADLINE)
*
+ * DELAYED - de/re-queue a sched_delayed task
+ *
+ * CLASS - going to update p->sched_class; makes sched_change call the
+ * various switch methods.
+ *
* ENQUEUE_HEAD - place at front of runqueue (tail if not specified)
* ENQUEUE_REPLENISH - CBS (replenish runtime and postpone deadline)
* ENQUEUE_MIGRATED - the task was migrated during wakeup
* ENQUEUE_RQ_SELECTED - ->select_task_rq() was called
*
+ * XXX SAVE/RESTORE in combination with CLASS doesn't really make sense, but
+ * SCHED_DEADLINE seems to rely on this for now.
*/
-#define DEQUEUE_SLEEP 0x01 /* Matches ENQUEUE_WAKEUP */
-#define DEQUEUE_SAVE 0x02 /* Matches ENQUEUE_RESTORE */
-#define DEQUEUE_MOVE 0x04 /* Matches ENQUEUE_MOVE */
-#define DEQUEUE_NOCLOCK 0x08 /* Matches ENQUEUE_NOCLOCK */
-#define DEQUEUE_SPECIAL 0x10
-#define DEQUEUE_MIGRATING 0x100 /* Matches ENQUEUE_MIGRATING */
-#define DEQUEUE_DELAYED 0x200 /* Matches ENQUEUE_DELAYED */
-#define DEQUEUE_THROTTLE 0x800
-
-#define ENQUEUE_WAKEUP 0x01
-#define ENQUEUE_RESTORE 0x02
-#define ENQUEUE_MOVE 0x04
-#define ENQUEUE_NOCLOCK 0x08
-
-#define ENQUEUE_HEAD 0x10
-#define ENQUEUE_REPLENISH 0x20
-#define ENQUEUE_MIGRATED 0x40
-#define ENQUEUE_INITIAL 0x80
-#define ENQUEUE_MIGRATING 0x100
-#define ENQUEUE_DELAYED 0x200
-#define ENQUEUE_RQ_SELECTED 0x400
+#define DEQUEUE_SLEEP 0x0001 /* Matches ENQUEUE_WAKEUP */
+#define DEQUEUE_SAVE 0x0002 /* Matches ENQUEUE_RESTORE */
+#define DEQUEUE_MOVE 0x0004 /* Matches ENQUEUE_MOVE */
+#define DEQUEUE_NOCLOCK 0x0008 /* Matches ENQUEUE_NOCLOCK */
+
+#define DEQUEUE_MIGRATING 0x0010 /* Matches ENQUEUE_MIGRATING */
+#define DEQUEUE_DELAYED 0x0020 /* Matches ENQUEUE_DELAYED */
+#define DEQUEUE_CLASS 0x0040 /* Matches ENQUEUE_CLASS */
+
+#define DEQUEUE_SPECIAL 0x00010000
+#define DEQUEUE_THROTTLE 0x00020000
+
+#define ENQUEUE_WAKEUP 0x0001
+#define ENQUEUE_RESTORE 0x0002
+#define ENQUEUE_MOVE 0x0004
+#define ENQUEUE_NOCLOCK 0x0008
+
+#define ENQUEUE_MIGRATING 0x0010
+#define ENQUEUE_DELAYED 0x0020
+#define ENQUEUE_CLASS 0x0040
+
+#define ENQUEUE_HEAD 0x00010000
+#define ENQUEUE_REPLENISH 0x00020000
+#define ENQUEUE_MIGRATED 0x00040000
+#define ENQUEUE_INITIAL 0x00080000
+#define ENQUEUE_RQ_SELECTED 0x00100000
#define RETRY_TASK ((void *)-1UL)
@@ -2399,16 +2426,61 @@ struct sched_class {
#ifdef CONFIG_UCLAMP_TASK
int uclamp_enabled;
#endif
+ /*
+ * idle: 0
+ * ext: 1
+ * fair: 2
+ * rt: 4
+ * dl: 8
+ * stop: 16
+ */
+ unsigned int queue_mask;
+ /*
+ * move_queued_task/activate_task/enqueue_task: rq->lock
+ * ttwu_do_activate/activate_task/enqueue_task: rq->lock
+ * wake_up_new_task/activate_task/enqueue_task: task_rq_lock
+ * ttwu_runnable/enqueue_task: task_rq_lock
+ * proxy_task_current: rq->lock
+ * sched_change_end
+ */
void (*enqueue_task) (struct rq *rq, struct task_struct *p, int flags);
+ /*
+ * move_queued_task/deactivate_task/dequeue_task: rq->lock
+ * __schedule/block_task/dequeue_task: rq->lock
+ * proxy_task_current: rq->lock
+ * wait_task_inactive: task_rq_lock
+ * sched_change_begin
+ */
bool (*dequeue_task) (struct rq *rq, struct task_struct *p, int flags);
+
+ /*
+ * do_sched_yield: rq->lock
+ */
void (*yield_task) (struct rq *rq);
+ /*
+ * yield_to: rq->lock (double)
+ */
bool (*yield_to_task)(struct rq *rq, struct task_struct *p);
+ /*
+ * move_queued_task: rq->lock
+ * __migrate_swap_task: rq->lock
+ * ttwu_do_activate: rq->lock
+ * ttwu_runnable: task_rq_lock
+ * wake_up_new_task: task_rq_lock
+ */
void (*wakeup_preempt)(struct rq *rq, struct task_struct *p, int flags);
+ /*
+ * schedule/pick_next_task/prev_balance: rq->lock
+ */
int (*balance)(struct rq *rq, struct task_struct *prev, struct rq_flags *rf);
- struct task_struct *(*pick_task)(struct rq *rq);
+
+ /*
+ * schedule/pick_next_task: rq->lock
+ */
+ struct task_struct *(*pick_task)(struct rq *rq, struct rq_flags *rf);
/*
* Optional! When implemented pick_next_task() should be equivalent to:
*
@@ -2418,55 +2490,123 @@ struct sched_class {
* set_next_task_first(next);
* }
*/
- struct task_struct *(*pick_next_task)(struct rq *rq, struct task_struct *prev);
+ struct task_struct *(*pick_next_task)(struct rq *rq, struct task_struct *prev,
+ struct rq_flags *rf);
+ /*
+ * sched_change:
+ * __schedule: rq->lock
+ */
void (*put_prev_task)(struct rq *rq, struct task_struct *p, struct task_struct *next);
void (*set_next_task)(struct rq *rq, struct task_struct *p, bool first);
+ /*
+ * select_task_rq: p->pi_lock
+ * sched_exec: p->pi_lock
+ */
int (*select_task_rq)(struct task_struct *p, int task_cpu, int flags);
+ /*
+ * set_task_cpu: p->pi_lock || rq->lock (ttwu like)
+ */
void (*migrate_task_rq)(struct task_struct *p, int new_cpu);
+ /*
+ * ttwu_do_activate: rq->lock
+ * wake_up_new_task: task_rq_lock
+ */
void (*task_woken)(struct rq *this_rq, struct task_struct *task);
+ /*
+ * do_set_cpus_allowed: task_rq_lock + sched_change
+ */
void (*set_cpus_allowed)(struct task_struct *p, struct affinity_context *ctx);
+ /*
+ * sched_set_rq_{on,off}line: rq->lock
+ */
void (*rq_online)(struct rq *rq);
void (*rq_offline)(struct rq *rq);
+ /*
+ * push_cpu_stop: p->pi_lock && rq->lock
+ */
struct rq *(*find_lock_rq)(struct task_struct *p, struct rq *rq);
+ /*
+ * hrtick: rq->lock
+ * sched_tick: rq->lock
+ * sched_tick_remote: rq->lock
+ */
void (*task_tick)(struct rq *rq, struct task_struct *p, int queued);
+ /*
+ * sched_cgroup_fork: p->pi_lock
+ */
void (*task_fork)(struct task_struct *p);
+ /*
+ * finish_task_switch: no locks
+ */
void (*task_dead)(struct task_struct *p);
/*
- * The switched_from() call is allowed to drop rq->lock, therefore we
- * cannot assume the switched_from/switched_to pair is serialized by
- * rq->lock. They are however serialized by p->pi_lock.
+ * sched_change
+ */
+ void (*switching_from)(struct rq *this_rq, struct task_struct *task);
+ void (*switched_from) (struct rq *this_rq, struct task_struct *task);
+ void (*switching_to) (struct rq *this_rq, struct task_struct *task);
+ void (*switched_to) (struct rq *this_rq, struct task_struct *task);
+ u64 (*get_prio) (struct rq *this_rq, struct task_struct *task);
+ void (*prio_changed) (struct rq *this_rq, struct task_struct *task,
+ u64 oldprio);
+
+ /*
+ * set_load_weight: task_rq_lock + sched_change
+ * __setscheduler_parms: task_rq_lock + sched_change
*/
- void (*switching_to) (struct rq *this_rq, struct task_struct *task);
- void (*switched_from)(struct rq *this_rq, struct task_struct *task);
- void (*switched_to) (struct rq *this_rq, struct task_struct *task);
void (*reweight_task)(struct rq *this_rq, struct task_struct *task,
const struct load_weight *lw);
- void (*prio_changed) (struct rq *this_rq, struct task_struct *task,
- int oldprio);
+ /*
+ * sched_rr_get_interval: task_rq_lock
+ */
unsigned int (*get_rr_interval)(struct rq *rq,
struct task_struct *task);
+ /*
+ * task_sched_runtime: task_rq_lock
+ */
void (*update_curr)(struct rq *rq);
#ifdef CONFIG_FAIR_GROUP_SCHED
+ /*
+ * sched_change_group: task_rq_lock + sched_change
+ */
void (*task_change_group)(struct task_struct *p);
#endif
#ifdef CONFIG_SCHED_CORE
+ /*
+ * pick_next_task: rq->lock
+ * try_steal_cookie: rq->lock (double)
+ */
int (*task_is_throttled)(struct task_struct *p, int cpu);
#endif
};
+/*
+ * Does not nest; only used around sched_class::pick_task() rq-lock-breaks.
+ */
+static inline void rq_modified_clear(struct rq *rq)
+{
+ rq->queue_mask = 0;
+}
+
+static inline bool rq_modified_above(struct rq *rq, const struct sched_class * class)
+{
+ unsigned int mask = class->queue_mask;
+ return rq->queue_mask & ~((mask << 1) - 1);
+}
+
static inline void put_prev_task(struct rq *rq, struct task_struct *prev)
{
WARN_ON_ONCE(rq->donor != prev);
@@ -2578,8 +2718,9 @@ static inline bool sched_fair_runnable(struct rq *rq)
return rq->cfs.nr_queued > 0;
}
-extern struct task_struct *pick_next_task_fair(struct rq *rq, struct task_struct *prev, struct rq_flags *rf);
-extern struct task_struct *pick_task_idle(struct rq *rq);
+extern struct task_struct *pick_next_task_fair(struct rq *rq, struct task_struct *prev,
+ struct rq_flags *rf);
+extern struct task_struct *pick_task_idle(struct rq *rq, struct rq_flags *rf);
#define SCA_CHECK 0x01
#define SCA_MIGRATE_DISABLE 0x02
@@ -2609,7 +2750,7 @@ static inline bool task_allowed_on_cpu(struct task_struct *p, int cpu)
static inline cpumask_t *alloc_user_cpus_ptr(int node)
{
/*
- * See do_set_cpus_allowed() above for the rcu_head usage.
+ * See set_cpus_allowed_force() above for the rcu_head usage.
*/
int size = max_t(int, cpumask_size(), sizeof(struct rcu_head));
@@ -3539,285 +3680,212 @@ 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 */
+static __always_inline bool cid_on_cpu(unsigned int cid)
+{
+ return cid & MM_CID_ONCPU;
+}
-extern raw_spinlock_t cid_lock;
-extern int use_cid_lock;
+static __always_inline bool cid_in_transit(unsigned int cid)
+{
+ return cid & MM_CID_TRANSIT;
+}
-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 __always_inline unsigned int cpu_cid_to_cid(unsigned int cid)
+{
+ return cid & ~MM_CID_ONCPU;
+}
-static inline void __mm_cid_put(struct mm_struct *mm, int cid)
+static __always_inline unsigned int cid_to_cpu_cid(unsigned int cid)
{
- if (cid < 0)
- return;
- cpumask_clear_cpu(cid, mm_cidmask(mm));
+ return cid | MM_CID_ONCPU;
}
-/*
- * 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 __always_inline unsigned int cid_to_transit_cid(unsigned int cid)
{
- struct mm_struct *mm = t->mm;
- struct mm_cid __percpu *pcpu_cid = mm->pcpu_cid;
- int cid;
+ return cid | MM_CID_TRANSIT;
+}
- 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))
- return;
- __mm_cid_put(mm, mm_cid_clear_lazy_put(cid));
+static __always_inline unsigned int cid_from_transit_cid(unsigned int cid)
+{
+ return cid & ~MM_CID_TRANSIT;
}
-static inline int mm_cid_pcpu_unset(struct mm_struct *mm)
+static __always_inline bool cid_on_task(unsigned int cid)
{
- struct mm_cid __percpu *pcpu_cid = mm->pcpu_cid;
- int cid, res;
+ /* True if none of the MM_CID_ONCPU, MM_CID_TRANSIT, MM_CID_UNSET bits is set */
+ return cid < MM_CID_TRANSIT;
+}
- 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;
+static __always_inline void mm_drop_cid(struct mm_struct *mm, unsigned int cid)
+{
+ clear_bit(cid, mm_cidmask(mm));
}
-static inline void mm_cid_put(struct mm_struct *mm)
+static __always_inline void mm_unset_cid_on_task(struct task_struct *t)
{
- int cid;
+ unsigned int cid = t->mm_cid.cid;
- 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));
+ t->mm_cid.cid = MM_CID_UNSET;
+ if (cid_on_task(cid))
+ mm_drop_cid(t->mm, cid);
}
-static inline int __mm_cid_try_get(struct task_struct *t, struct mm_struct *mm)
+static __always_inline void mm_drop_cid_on_cpu(struct mm_struct *mm, struct mm_cid_pcpu *pcp)
{
- struct cpumask *cidmask = mm_cidmask(mm);
- struct mm_cid __percpu *pcpu_cid = mm->pcpu_cid;
- int cid, max_nr_cid, allowed_max_nr_cid;
+ /* 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);
+}
- /*
- * 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;
+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;
}
-/*
- * 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)
+static inline unsigned int mm_get_cid(struct mm_struct *mm)
{
- struct mm_cid *pcpu_cid = per_cpu_ptr(mm->pcpu_cid, cpu_of(rq));
+ unsigned int cid = __mm_get_cid(mm, READ_ONCE(mm->mm_cid.max_cids));
- lockdep_assert_rq_held(rq);
- WRITE_ONCE(pcpu_cid->time, rq->clock);
+ while (cid == MM_CID_UNSET) {
+ cpu_relax();
+ cid = __mm_get_cid(mm, num_possible_cpus());
+ }
+ return cid;
}
-static inline int __mm_cid_get(struct rq *rq, struct task_struct *t,
- struct mm_struct *mm)
+static inline unsigned int mm_cid_converge(struct mm_struct *mm, unsigned int orig_cid,
+ unsigned int max_cids)
{
- int cid;
+ unsigned int new_cid, cid = cpu_cid_to_cid(orig_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;
+ /* 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;
+}
- /*
- * 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.
- */
- barrier();
- WRITE_ONCE(use_cid_lock, 0);
-unlock:
- raw_spin_unlock(&cid_lock);
-end:
- mm_cid_snapshot_time(rq, mm);
+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);
+ }
+}
- return cid;
+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 inline int mm_cid_get(struct rq *rq, struct task_struct *t,
- struct mm_struct *mm)
+static __always_inline void mm_cid_from_cpu(struct task_struct *t, unsigned int cpu_cid)
{
- struct mm_cid __percpu *pcpu_cid = mm->pcpu_cid;
- struct cpumask *cpumask;
- int cid;
+ unsigned int max_cids, tcid = t->mm_cid.cid;
+ struct mm_struct *mm = t->mm;
- 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(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));
+ 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));
}
- cid = __mm_cid_get(rq, t, mm);
- __this_cpu_write(pcpu_cid->cid, cid);
- __this_cpu_write(pcpu_cid->recent_cid, cid);
-
- return cid;
+ mm_cid_update_pcpu_cid(mm, cpu_cid);
+ mm_cid_update_task_cid(t, cpu_cid);
}
-static inline void switch_mm_cid(struct rq *rq,
- struct task_struct *prev,
- struct task_struct *next)
+static __always_inline void mm_cid_from_task(struct task_struct *t, unsigned int cpu_cid)
{
- /*
- * 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();
+ 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);
}
- if (prev->mm_cid_active) {
- mm_cid_snapshot_time(rq, prev->mm);
- mm_cid_put_lazy(prev);
- prev->mm_cid = -1;
- }
- if (next->mm_cid_active)
- next->last_mm_cid = next->mm_cid = mm_cid_get(rq, next, next->mm);
+ 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);
}
#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_switch_to(struct task_struct *prev, struct task_struct *next) { }
#endif /* !CONFIG_SCHED_MM_CID */
extern u64 avg_vruntime(struct cfs_rq *cfs_rq);
@@ -3876,32 +3944,42 @@ extern void set_load_weight(struct task_struct *p, bool update_load);
extern void enqueue_task(struct rq *rq, struct task_struct *p, int flags);
extern bool dequeue_task(struct rq *rq, struct task_struct *p, int flags);
-extern void check_class_changing(struct rq *rq, struct task_struct *p,
- const struct sched_class *prev_class);
-extern void check_class_changed(struct rq *rq, struct task_struct *p,
- const struct sched_class *prev_class,
- int oldprio);
-
extern struct balance_callback *splice_balance_callbacks(struct rq *rq);
extern void balance_callbacks(struct rq *rq, struct balance_callback *head);
-#ifdef CONFIG_SCHED_CLASS_EXT
/*
- * Used by SCX in the enable/disable paths to move tasks between sched_classes
- * and establish invariants.
+ * The 'sched_change' pattern is the safe, easy and slow way of changing a
+ * task's scheduling properties. It dequeues a task, such that the scheduler
+ * is fully unaware of it; at which point its properties can be modified;
+ * after which it is enqueued again.
+ *
+ * Typically this must be called while holding task_rq_lock, since most/all
+ * properties are serialized under those locks. There is currently one
+ * exception to this rule in sched/ext which only holds rq->lock.
*/
-struct sched_enq_and_set_ctx {
+
+/*
+ * This structure is a temporary, used to preserve/convey the queueing state
+ * of the task between sched_change_begin() and sched_change_end(). Ensuring
+ * the task's queueing state is idempotent across the operation.
+ */
+struct sched_change_ctx {
+ u64 prio;
struct task_struct *p;
- int queue_flags;
+ int flags;
bool queued;
bool running;
};
-void sched_deq_and_put_task(struct task_struct *p, int queue_flags,
- struct sched_enq_and_set_ctx *ctx);
-void sched_enq_and_set_task(struct sched_enq_and_set_ctx *ctx);
+struct sched_change_ctx *sched_change_begin(struct task_struct *p, unsigned int flags);
+void sched_change_end(struct sched_change_ctx *ctx);
-#endif /* CONFIG_SCHED_CLASS_EXT */
+DEFINE_CLASS(sched_change, struct sched_change_ctx *,
+ sched_change_end(_T),
+ sched_change_begin(p, flags),
+ struct task_struct *p, unsigned int flags)
+
+DEFINE_CLASS_IS_UNCONDITIONAL(sched_change)
#include "ext.h"