sched_ext: A second set of fixes for v6.12-rc1

- When sched_ext is in bypass mode (e.g. while disabling the BPF scheduler),
   it was using one DSQ to implement global FIFO scheduling as all it has to
   do is guaranteeing reasonable forward progress. On multi-socket machines,
   this can lead to live-lock conditions under certain workloads. Fixed by
   splitting the queue used for FIFO scheduling per NUMA node. This required
   several preparation patches.
 
 - Hotplug tests on powerpc could reliably trigger deadlock while enabling a
   BPF scheduler. This was caused by cpu_hotplug_lock nesting inside
   scx_fork_rwsem and then CPU hotplug path trying to fork a new thread while
   holding cpu_hotplug_lock. Fixed by restructuring locking in enable and
   disable paths so that the two locks are not coupled. This required several
   preparation patches which also fixed a couple other issues in the enable
   path.
 
 - A build fix for !CONFIG_SMP.
 
 - Userspace tooling sync and updates.
 -----BEGIN PGP SIGNATURE-----
 
 iIQEABYKACwWIQTfIjM1kS57o3GsC/uxYfJx3gVYGQUCZvsB8A4cdGpAa2VybmVs
 Lm9yZwAKCRCxYfJx3gVYGW0cAP9LWESwNvszp0jDglg8kvS6tyHDLNQYLT99Dy4Q
 NPE1OgD+KUKXuRBOhT3dc9dbwodmPtX2rq36QIgHhoGiPB9hew8=
 =NH+o
 -----END PGP SIGNATURE-----

Merge tag 'sched_ext-for-6.12-rc1-fixes-1' of git://git.kernel.org/pub/scm/linux/kernel/git/tj/sched_ext

Pull sched_ext fixes from Tejun Heo:

 - When sched_ext is in bypass mode (e.g. while disabling the BPF
   scheduler), it was using one DSQ to implement global FIFO scheduling
   as all it has to do is guaranteeing reasonable forward progress.

   On multi-socket machines, this can lead to live-lock conditions under
   certain workloads. Fixed by splitting the queue used for FIFO
   scheduling per NUMA node. This required several preparation patches.

 - Hotplug tests on powerpc could reliably trigger deadlock while
   enabling a BPF scheduler.

   This was caused by cpu_hotplug_lock nesting inside scx_fork_rwsem and
   then CPU hotplug path trying to fork a new thread while holding
   cpu_hotplug_lock.

   Fixed by restructuring locking in enable and disable paths so that
   the two locks are not coupled. This required several preparation
   patches which also fixed a couple other issues in the enable path.

 - A build fix for !CONFIG_SMP

 - Userspace tooling sync and updates

* tag 'sched_ext-for-6.12-rc1-fixes-1' of git://git.kernel.org/pub/scm/linux/kernel/git/tj/sched_ext:
  sched_ext: Remove redundant p->nr_cpus_allowed checker
  sched_ext: Decouple locks in scx_ops_enable()
  sched_ext: Decouple locks in scx_ops_disable_workfn()
  sched_ext: Add scx_cgroup_enabled to gate cgroup operations and fix scx_tg_online()
  sched_ext: Enable scx_ops_init_task() separately
  sched_ext: Fix SCX_TASK_INIT -> SCX_TASK_READY transitions in scx_ops_enable()
  sched_ext: Initialize in bypass mode
  sched_ext: Remove SCX_OPS_PREPPING
  sched_ext: Relocate check_hotplug_seq() call in scx_ops_enable()
  sched_ext: Use shorter slice while bypassing
  sched_ext: Split the global DSQ per NUMA node
  sched_ext: Relocate find_user_dsq()
  sched_ext: Allow only user DSQs for scx_bpf_consume(), scx_bpf_dsq_nr_queued() and bpf_iter_scx_dsq_new()
  scx_flatcg: Use a user DSQ for fallback instead of SCX_DSQ_GLOBAL
  tools/sched_ext: Receive misc updates from SCX repo
  sched_ext: Add __COMPAT helpers for features added during v6.12 devel cycle
  sched_ext: Build fix for !CONFIG_SMP
This commit is contained in:
Linus Torvalds 2024-09-30 12:58:17 -07:00
commit e32cde8d2b
6 changed files with 236 additions and 166 deletions

View file

@ -9,6 +9,7 @@
#define SCX_OP_IDX(op) (offsetof(struct sched_ext_ops, op) / sizeof(void (*)(void)))
enum scx_consts {
SCX_SLICE_BYPASS = SCX_SLICE_DFL / 4,
SCX_DSP_DFL_MAX_BATCH = 32,
SCX_DSP_MAX_LOOPS = 32,
SCX_WATCHDOG_MAX_TIMEOUT = 30 * HZ,
@ -778,7 +779,6 @@ enum scx_tg_flags {
};
enum scx_ops_enable_state {
SCX_OPS_PREPPING,
SCX_OPS_ENABLING,
SCX_OPS_ENABLED,
SCX_OPS_DISABLING,
@ -786,7 +786,6 @@ enum scx_ops_enable_state {
};
static const char *scx_ops_enable_state_str[] = {
[SCX_OPS_PREPPING] = "prepping",
[SCX_OPS_ENABLING] = "enabling",
[SCX_OPS_ENABLED] = "enabled",
[SCX_OPS_DISABLING] = "disabling",
@ -854,6 +853,7 @@ DEFINE_STATIC_KEY_FALSE(__scx_ops_enabled);
DEFINE_STATIC_PERCPU_RWSEM(scx_fork_rwsem);
static atomic_t scx_ops_enable_state_var = ATOMIC_INIT(SCX_OPS_DISABLED);
static atomic_t scx_ops_bypass_depth = ATOMIC_INIT(0);
static bool scx_ops_init_task_enabled;
static bool scx_switching_all;
DEFINE_STATIC_KEY_FALSE(__scx_switched_all);
@ -925,8 +925,15 @@ static unsigned long __percpu *scx_kick_cpus_pnt_seqs;
*/
static DEFINE_PER_CPU(struct task_struct *, direct_dispatch_task);
/* dispatch queues */
static struct scx_dispatch_q __cacheline_aligned_in_smp scx_dsq_global;
/*
* Dispatch queues.
*
* The global DSQ (%SCX_DSQ_GLOBAL) is split per-node for scalability. This is
* to avoid live-locking in bypass mode where all tasks are dispatched to
* %SCX_DSQ_GLOBAL and all CPUs consume from it. If per-node split isn't
* sufficient, it can be further split.
*/
static struct scx_dispatch_q **global_dsqs;
static const struct rhashtable_params dsq_hash_params = {
.key_len = 8,
@ -1029,6 +1036,16 @@ static bool u32_before(u32 a, u32 b)
return (s32)(a - b) < 0;
}
static struct scx_dispatch_q *find_global_dsq(struct task_struct *p)
{
return global_dsqs[cpu_to_node(task_cpu(p))];
}
static struct scx_dispatch_q *find_user_dsq(u64 dsq_id)
{
return rhashtable_lookup_fast(&dsq_hash, &dsq_id, dsq_hash_params);
}
/*
* scx_kf_mask enforcement. Some kfuncs can only be called from specific SCX
* ops. When invoking SCX ops, SCX_CALL_OP[_RET]() should be used to indicate
@ -1637,7 +1654,7 @@ static void dispatch_enqueue(struct scx_dispatch_q *dsq, struct task_struct *p,
scx_ops_error("attempting to dispatch to a destroyed dsq");
/* fall back to the global dsq */
raw_spin_unlock(&dsq->lock);
dsq = &scx_dsq_global;
dsq = find_global_dsq(p);
raw_spin_lock(&dsq->lock);
}
}
@ -1803,21 +1820,6 @@ static void dispatch_dequeue(struct rq *rq, struct task_struct *p)
raw_spin_unlock(&dsq->lock);
}
static struct scx_dispatch_q *find_user_dsq(u64 dsq_id)
{
return rhashtable_lookup_fast(&dsq_hash, &dsq_id, dsq_hash_params);
}
static struct scx_dispatch_q *find_non_local_dsq(u64 dsq_id)
{
lockdep_assert(rcu_read_lock_any_held());
if (dsq_id == SCX_DSQ_GLOBAL)
return &scx_dsq_global;
else
return find_user_dsq(dsq_id);
}
static struct scx_dispatch_q *find_dsq_for_dispatch(struct rq *rq, u64 dsq_id,
struct task_struct *p)
{
@ -1830,16 +1832,20 @@ static struct scx_dispatch_q *find_dsq_for_dispatch(struct rq *rq, u64 dsq_id,
s32 cpu = dsq_id & SCX_DSQ_LOCAL_CPU_MASK;
if (!ops_cpu_valid(cpu, "in SCX_DSQ_LOCAL_ON dispatch verdict"))
return &scx_dsq_global;
return find_global_dsq(p);
return &cpu_rq(cpu)->scx.local_dsq;
}
dsq = find_non_local_dsq(dsq_id);
if (dsq_id == SCX_DSQ_GLOBAL)
dsq = find_global_dsq(p);
else
dsq = find_user_dsq(dsq_id);
if (unlikely(!dsq)) {
scx_ops_error("non-existent DSQ 0x%llx for %s[%d]",
dsq_id, p->comm, p->pid);
return &scx_dsq_global;
return find_global_dsq(p);
}
return dsq;
@ -1938,6 +1944,7 @@ static bool scx_rq_online(struct rq *rq)
static void do_enqueue_task(struct rq *rq, struct task_struct *p, u64 enq_flags,
int sticky_cpu)
{
bool bypassing = scx_rq_bypassing(rq);
struct task_struct **ddsp_taskp;
unsigned long qseq;
@ -1955,7 +1962,7 @@ static void do_enqueue_task(struct rq *rq, struct task_struct *p, u64 enq_flags,
if (!scx_rq_online(rq))
goto local;
if (scx_rq_bypassing(rq))
if (bypassing)
goto global;
if (p->scx.ddsp_dsq_id != SCX_DSQ_INVALID)
@ -2010,8 +2017,8 @@ static void do_enqueue_task(struct rq *rq, struct task_struct *p, u64 enq_flags,
global:
touch_core_sched(rq, p); /* see the comment in local: */
p->scx.slice = SCX_SLICE_DFL;
dispatch_enqueue(&scx_dsq_global, p, enq_flags);
p->scx.slice = bypassing ? SCX_SLICE_BYPASS : SCX_SLICE_DFL;
dispatch_enqueue(find_global_dsq(p), p, enq_flags);
}
static bool task_runnable(const struct task_struct *p)
@ -2357,6 +2364,7 @@ static bool consume_remote_task(struct rq *this_rq, struct task_struct *p,
}
}
#else /* CONFIG_SMP */
static inline void move_remote_task_to_local_dsq(struct task_struct *p, u64 enq_flags, struct rq *src_rq, struct rq *dst_rq) { WARN_ON_ONCE(1); }
static inline bool task_can_run_on_remote_rq(struct task_struct *p, struct rq *rq, bool trigger_error) { return false; }
static inline bool consume_remote_task(struct rq *this_rq, struct task_struct *p, struct scx_dispatch_q *dsq, struct rq *task_rq) { return false; }
#endif /* CONFIG_SMP */
@ -2396,6 +2404,13 @@ static bool consume_dispatch_q(struct rq *rq, struct scx_dispatch_q *dsq)
return false;
}
static bool consume_global_dsq(struct rq *rq)
{
int node = cpu_to_node(cpu_of(rq));
return consume_dispatch_q(rq, global_dsqs[node]);
}
/**
* dispatch_to_local_dsq - Dispatch a task to a local dsq
* @rq: current rq which is locked
@ -2429,7 +2444,8 @@ static void dispatch_to_local_dsq(struct rq *rq, struct scx_dispatch_q *dst_dsq,
#ifdef CONFIG_SMP
if (unlikely(!task_can_run_on_remote_rq(p, dst_rq, true))) {
dispatch_enqueue(&scx_dsq_global, p, enq_flags | SCX_ENQ_CLEAR_OPSS);
dispatch_enqueue(find_global_dsq(p), p,
enq_flags | SCX_ENQ_CLEAR_OPSS);
return;
}
@ -2629,7 +2645,7 @@ static int balance_one(struct rq *rq, struct task_struct *prev)
if (rq->scx.local_dsq.nr)
goto has_tasks;
if (consume_dispatch_q(rq, &scx_dsq_global))
if (consume_global_dsq(rq))
goto has_tasks;
if (!SCX_HAS_OP(dispatch) || scx_rq_bypassing(rq) || !scx_rq_online(rq))
@ -2654,7 +2670,7 @@ static int balance_one(struct rq *rq, struct task_struct *prev)
if (rq->scx.local_dsq.nr)
goto has_tasks;
if (consume_dispatch_q(rq, &scx_dsq_global))
if (consume_global_dsq(rq))
goto has_tasks;
/*
@ -3058,22 +3074,13 @@ static s32 scx_select_cpu_dfl(struct task_struct *p, s32 prev_cpu,
* there is an idle core elsewhere on the system.
*/
cpu = smp_processor_id();
if ((wake_flags & SCX_WAKE_SYNC) && p->nr_cpus_allowed > 1 &&
if ((wake_flags & SCX_WAKE_SYNC) &&
!cpumask_empty(idle_masks.cpu) && !(current->flags & PF_EXITING) &&
cpu_rq(cpu)->scx.local_dsq.nr == 0) {
if (cpumask_test_cpu(cpu, p->cpus_ptr))
goto cpu_found;
}
if (p->nr_cpus_allowed == 1) {
if (test_and_clear_cpu_idle(prev_cpu)) {
cpu = prev_cpu;
goto cpu_found;
} else {
return prev_cpu;
}
}
/*
* If CPU has SMT, any wholly idle CPU is likely a better pick than
* partially idle @prev_cpu.
@ -3550,7 +3557,7 @@ int scx_fork(struct task_struct *p)
{
percpu_rwsem_assert_held(&scx_fork_rwsem);
if (scx_enabled())
if (scx_ops_init_task_enabled)
return scx_ops_init_task(p, task_group(p), true);
else
return 0;
@ -3558,7 +3565,7 @@ int scx_fork(struct task_struct *p)
void scx_post_fork(struct task_struct *p)
{
if (scx_enabled()) {
if (scx_ops_init_task_enabled) {
scx_set_task_state(p, SCX_TASK_READY);
/*
@ -3690,6 +3697,7 @@ bool scx_can_stop_tick(struct rq *rq)
#ifdef CONFIG_EXT_GROUP_SCHED
DEFINE_STATIC_PERCPU_RWSEM(scx_cgroup_rwsem);
static bool scx_cgroup_enabled;
static bool cgroup_warned_missing_weight;
static bool cgroup_warned_missing_idle;
@ -3709,8 +3717,7 @@ static void scx_cgroup_warn_missing_weight(struct task_group *tg)
static void scx_cgroup_warn_missing_idle(struct task_group *tg)
{
if (scx_ops_enable_state() == SCX_OPS_DISABLED ||
cgroup_warned_missing_idle)
if (!scx_cgroup_enabled || cgroup_warned_missing_idle)
return;
if (!tg->idle)
@ -3731,15 +3738,18 @@ int scx_tg_online(struct task_group *tg)
scx_cgroup_warn_missing_weight(tg);
if (SCX_HAS_OP(cgroup_init)) {
struct scx_cgroup_init_args args = { .weight = tg->scx_weight };
if (scx_cgroup_enabled) {
if (SCX_HAS_OP(cgroup_init)) {
struct scx_cgroup_init_args args =
{ .weight = tg->scx_weight };
ret = SCX_CALL_OP_RET(SCX_KF_UNLOCKED, cgroup_init,
tg->css.cgroup, &args);
if (!ret)
ret = SCX_CALL_OP_RET(SCX_KF_UNLOCKED, cgroup_init,
tg->css.cgroup, &args);
if (ret)
ret = ops_sanitize_err("cgroup_init", ret);
}
if (ret == 0)
tg->scx_flags |= SCX_TG_ONLINE | SCX_TG_INITED;
else
ret = ops_sanitize_err("cgroup_init", ret);
} else {
tg->scx_flags |= SCX_TG_ONLINE;
}
@ -3770,7 +3780,7 @@ int scx_cgroup_can_attach(struct cgroup_taskset *tset)
/* released in scx_finish/cancel_attach() */
percpu_down_read(&scx_cgroup_rwsem);
if (!scx_enabled())
if (!scx_cgroup_enabled)
return 0;
cgroup_taskset_for_each(p, css, tset) {
@ -3813,7 +3823,7 @@ int scx_cgroup_can_attach(struct cgroup_taskset *tset)
void scx_move_task(struct task_struct *p)
{
if (!scx_enabled())
if (!scx_cgroup_enabled)
return;
/*
@ -3849,7 +3859,7 @@ void scx_cgroup_cancel_attach(struct cgroup_taskset *tset)
struct cgroup_subsys_state *css;
struct task_struct *p;
if (!scx_enabled())
if (!scx_cgroup_enabled)
goto out_unlock;
cgroup_taskset_for_each(p, css, tset) {
@ -3866,7 +3876,7 @@ void scx_group_set_weight(struct task_group *tg, unsigned long weight)
{
percpu_down_read(&scx_cgroup_rwsem);
if (tg->scx_weight != weight) {
if (scx_cgroup_enabled && tg->scx_weight != weight) {
if (SCX_HAS_OP(cgroup_set_weight))
SCX_CALL_OP(SCX_KF_UNLOCKED, cgroup_set_weight,
tg_cgrp(tg), weight);
@ -4038,6 +4048,9 @@ static void scx_cgroup_exit(void)
percpu_rwsem_assert_held(&scx_cgroup_rwsem);
WARN_ON_ONCE(!scx_cgroup_enabled);
scx_cgroup_enabled = false;
/*
* scx_tg_on/offline() are excluded through scx_cgroup_rwsem. If we walk
* cgroups and exit all the inited ones, all online cgroups are exited.
@ -4113,6 +4126,9 @@ static int scx_cgroup_init(void)
}
rcu_read_unlock();
WARN_ON_ONCE(scx_cgroup_enabled);
scx_cgroup_enabled = true;
return 0;
}
@ -4431,19 +4447,23 @@ static void scx_ops_disable_workfn(struct kthread_work *work)
WRITE_ONCE(scx_switching_all, false);
/*
* Avoid racing against fork and cgroup changes. See scx_ops_enable()
* for explanation on the locking order.
* Shut down cgroup support before tasks so that the cgroup attach path
* doesn't race against scx_ops_exit_task().
*/
percpu_down_write(&scx_fork_rwsem);
cpus_read_lock();
scx_cgroup_lock();
scx_cgroup_exit();
scx_cgroup_unlock();
spin_lock_irq(&scx_tasks_lock);
scx_task_iter_init(&sti);
/*
* The BPF scheduler is going away. All tasks including %TASK_DEAD ones
* must be switched out and exited synchronously.
*/
percpu_down_write(&scx_fork_rwsem);
scx_ops_init_task_enabled = false;
spin_lock_irq(&scx_tasks_lock);
scx_task_iter_init(&sti);
while ((p = scx_task_iter_next_locked(&sti))) {
const struct sched_class *old_class = p->sched_class;
struct sched_enq_and_set_ctx ctx;
@ -4461,23 +4481,18 @@ static void scx_ops_disable_workfn(struct kthread_work *work)
}
scx_task_iter_exit(&sti);
spin_unlock_irq(&scx_tasks_lock);
percpu_up_write(&scx_fork_rwsem);
/* no task is on scx, turn off all the switches and flush in-progress calls */
static_branch_disable_cpuslocked(&__scx_ops_enabled);
static_branch_disable(&__scx_ops_enabled);
for (i = SCX_OPI_BEGIN; i < SCX_OPI_END; i++)
static_branch_disable_cpuslocked(&scx_has_op[i]);
static_branch_disable_cpuslocked(&scx_ops_enq_last);
static_branch_disable_cpuslocked(&scx_ops_enq_exiting);
static_branch_disable_cpuslocked(&scx_ops_cpu_preempt);
static_branch_disable_cpuslocked(&scx_builtin_idle_enabled);
static_branch_disable(&scx_has_op[i]);
static_branch_disable(&scx_ops_enq_last);
static_branch_disable(&scx_ops_enq_exiting);
static_branch_disable(&scx_ops_cpu_preempt);
static_branch_disable(&scx_builtin_idle_enabled);
synchronize_rcu();
scx_cgroup_exit();
scx_cgroup_unlock();
cpus_read_unlock();
percpu_up_write(&scx_fork_rwsem);
if (ei->kind >= SCX_EXIT_ERROR) {
pr_err("sched_ext: BPF scheduler \"%s\" disabled (%s)\n",
scx_ops.name, ei->reason);
@ -4929,7 +4944,7 @@ static int scx_ops_enable(struct sched_ext_ops *ops, struct bpf_link *link)
struct scx_task_iter sti;
struct task_struct *p;
unsigned long timeout;
int i, cpu, ret;
int i, cpu, node, ret;
if (!cpumask_equal(housekeeping_cpumask(HK_TYPE_DOMAIN),
cpu_possible_mask)) {
@ -4948,6 +4963,34 @@ static int scx_ops_enable(struct sched_ext_ops *ops, struct bpf_link *link)
}
}
if (!global_dsqs) {
struct scx_dispatch_q **dsqs;
dsqs = kcalloc(nr_node_ids, sizeof(dsqs[0]), GFP_KERNEL);
if (!dsqs) {
ret = -ENOMEM;
goto err_unlock;
}
for_each_node_state(node, N_POSSIBLE) {
struct scx_dispatch_q *dsq;
dsq = kzalloc_node(sizeof(*dsq), GFP_KERNEL, node);
if (!dsq) {
for_each_node_state(node, N_POSSIBLE)
kfree(dsqs[node]);
kfree(dsqs);
ret = -ENOMEM;
goto err_unlock;
}
init_dsq(dsq, SCX_DSQ_GLOBAL);
dsqs[node] = dsq;
}
global_dsqs = dsqs;
}
if (scx_ops_enable_state() != SCX_OPS_DISABLED) {
ret = -EBUSY;
goto err_unlock;
@ -4971,12 +5014,12 @@ static int scx_ops_enable(struct sched_ext_ops *ops, struct bpf_link *link)
}
/*
* Set scx_ops, transition to PREPPING and clear exit info to arm the
* Set scx_ops, transition to ENABLING and clear exit info to arm the
* disable path. Failure triggers full disabling from here on.
*/
scx_ops = *ops;
WARN_ON_ONCE(scx_ops_set_enable_state(SCX_OPS_PREPPING) !=
WARN_ON_ONCE(scx_ops_set_enable_state(SCX_OPS_ENABLING) !=
SCX_OPS_DISABLED);
atomic_set(&scx_exit_kind, SCX_EXIT_NONE);
@ -4997,7 +5040,8 @@ static int scx_ops_enable(struct sched_ext_ops *ops, struct bpf_link *link)
ret = SCX_CALL_OP_RET(SCX_KF_UNLOCKED, init);
if (ret) {
ret = ops_sanitize_err("init", ret);
goto err_disable_unlock_cpus;
cpus_read_unlock();
goto err_disable;
}
}
@ -5005,6 +5049,7 @@ static int scx_ops_enable(struct sched_ext_ops *ops, struct bpf_link *link)
if (((void (**)(void))ops)[i])
static_branch_enable_cpuslocked(&scx_has_op[i]);
check_hotplug_seq(ops);
cpus_read_unlock();
ret = validate_ops(ops);
@ -5032,57 +5077,40 @@ static int scx_ops_enable(struct sched_ext_ops *ops, struct bpf_link *link)
scx_watchdog_timeout / 2);
/*
* Lock out forks, cgroup on/offlining and moves before opening the
* floodgate so that they don't wander into the operations prematurely.
*
* We don't need to keep the CPUs stable but static_branch_*() requires
* cpus_read_lock() and scx_cgroup_rwsem must nest inside
* cpu_hotplug_lock because of the following dependency chain:
*
* cpu_hotplug_lock --> cgroup_threadgroup_rwsem --> scx_cgroup_rwsem
*
* So, we need to do cpus_read_lock() before scx_cgroup_lock() and use
* static_branch_*_cpuslocked().
*
* Note that cpu_hotplug_lock must nest inside scx_fork_rwsem due to the
* following dependency chain:
*
* scx_fork_rwsem --> pernet_ops_rwsem --> cpu_hotplug_lock
* Once __scx_ops_enabled is set, %current can be switched to SCX
* anytime. This can lead to stalls as some BPF schedulers (e.g.
* userspace scheduling) may not function correctly before all tasks are
* switched. Init in bypass mode to guarantee forward progress.
*/
percpu_down_write(&scx_fork_rwsem);
cpus_read_lock();
scx_cgroup_lock();
check_hotplug_seq(ops);
scx_ops_bypass(true);
for (i = SCX_OPI_NORMAL_BEGIN; i < SCX_OPI_NORMAL_END; i++)
if (((void (**)(void))ops)[i])
static_branch_enable_cpuslocked(&scx_has_op[i]);
static_branch_enable(&scx_has_op[i]);
if (ops->flags & SCX_OPS_ENQ_LAST)
static_branch_enable_cpuslocked(&scx_ops_enq_last);
static_branch_enable(&scx_ops_enq_last);
if (ops->flags & SCX_OPS_ENQ_EXITING)
static_branch_enable_cpuslocked(&scx_ops_enq_exiting);
static_branch_enable(&scx_ops_enq_exiting);
if (scx_ops.cpu_acquire || scx_ops.cpu_release)
static_branch_enable_cpuslocked(&scx_ops_cpu_preempt);
static_branch_enable(&scx_ops_cpu_preempt);
if (!ops->update_idle || (ops->flags & SCX_OPS_KEEP_BUILTIN_IDLE)) {
reset_idle_masks();
static_branch_enable_cpuslocked(&scx_builtin_idle_enabled);
static_branch_enable(&scx_builtin_idle_enabled);
} else {
static_branch_disable_cpuslocked(&scx_builtin_idle_enabled);
static_branch_disable(&scx_builtin_idle_enabled);
}
/*
* All cgroups should be initialized before letting in tasks. cgroup
* on/offlining and task migrations are already locked out.
* Lock out forks, cgroup on/offlining and moves before opening the
* floodgate so that they don't wander into the operations prematurely.
*/
ret = scx_cgroup_init();
if (ret)
goto err_disable_unlock_all;
percpu_down_write(&scx_fork_rwsem);
static_branch_enable_cpuslocked(&__scx_ops_enabled);
WARN_ON_ONCE(scx_ops_init_task_enabled);
scx_ops_init_task_enabled = true;
/*
* Enable ops for every task. Fork is excluded by scx_fork_rwsem
@ -5090,9 +5118,19 @@ static int scx_ops_enable(struct sched_ext_ops *ops, struct bpf_link *link)
* leaving as sched_ext_free() can handle both prepped and enabled
* tasks. Prep all tasks first and then enable them with preemption
* disabled.
*
* All cgroups should be initialized before scx_ops_init_task() so that
* the BPF scheduler can reliably track each task's cgroup membership
* from scx_ops_init_task(). Lock out cgroup on/offlining and task
* migrations while tasks are being initialized so that
* scx_cgroup_can_attach() never sees uninitialized tasks.
*/
spin_lock_irq(&scx_tasks_lock);
scx_cgroup_lock();
ret = scx_cgroup_init();
if (ret)
goto err_disable_unlock_all;
spin_lock_irq(&scx_tasks_lock);
scx_task_iter_init(&sti);
while ((p = scx_task_iter_next_locked(&sti))) {
/*
@ -5117,43 +5155,30 @@ static int scx_ops_enable(struct sched_ext_ops *ops, struct bpf_link *link)
goto err_disable_unlock_all;
}
scx_set_task_state(p, SCX_TASK_READY);
put_task_struct(p);
spin_lock_irq(&scx_tasks_lock);
}
scx_task_iter_exit(&sti);
spin_unlock_irq(&scx_tasks_lock);
scx_cgroup_unlock();
percpu_up_write(&scx_fork_rwsem);
/*
* All tasks are prepped but are still ops-disabled. Ensure that
* %current can't be scheduled out and switch everyone.
* preempt_disable() is necessary because we can't guarantee that
* %current won't be starved if scheduled out while switching.
* All tasks are READY. It's safe to turn on scx_enabled() and switch
* all eligible tasks.
*/
preempt_disable();
WRITE_ONCE(scx_switching_all, !(ops->flags & SCX_OPS_SWITCH_PARTIAL));
static_branch_enable(&__scx_ops_enabled);
/*
* From here on, the disable path must assume that tasks have ops
* enabled and need to be recovered.
*
* Transition to ENABLING fails iff the BPF scheduler has already
* triggered scx_bpf_error(). Returning an error code here would lose
* the recorded error information. Exit indicating success so that the
* error is notified through ops.exit() with all the details.
*/
if (!scx_ops_tryset_enable_state(SCX_OPS_ENABLING, SCX_OPS_PREPPING)) {
preempt_enable();
spin_unlock_irq(&scx_tasks_lock);
WARN_ON_ONCE(atomic_read(&scx_exit_kind) == SCX_EXIT_NONE);
ret = 0;
goto err_disable_unlock_all;
}
/*
* We're fully committed and can't fail. The PREPPED -> ENABLED
* We're fully committed and can't fail. The task READY -> ENABLED
* transitions here are synchronized against sched_ext_free() through
* scx_tasks_lock.
*/
WRITE_ONCE(scx_switching_all, !(ops->flags & SCX_OPS_SWITCH_PARTIAL));
percpu_down_write(&scx_fork_rwsem);
spin_lock_irq(&scx_tasks_lock);
scx_task_iter_init(&sti);
while ((p = scx_task_iter_next_locked(&sti))) {
const struct sched_class *old_class = p->sched_class;
@ -5161,7 +5186,6 @@ static int scx_ops_enable(struct sched_ext_ops *ops, struct bpf_link *link)
sched_deq_and_put_task(p, DEQUEUE_SAVE | DEQUEUE_MOVE, &ctx);
scx_set_task_state(p, SCX_TASK_READY);
__setscheduler_prio(p, p->prio);
check_class_changing(task_rq(p), p, old_class);
@ -5170,14 +5194,16 @@ static int scx_ops_enable(struct sched_ext_ops *ops, struct bpf_link *link)
check_class_changed(task_rq(p), p, old_class, p->prio);
}
scx_task_iter_exit(&sti);
spin_unlock_irq(&scx_tasks_lock);
preempt_enable();
scx_cgroup_unlock();
cpus_read_unlock();
percpu_up_write(&scx_fork_rwsem);
/* see above ENABLING transition for the explanation on exiting with 0 */
scx_ops_bypass(false);
/*
* Returning an error code here would lose the recorded error
* information. Exit indicating success so that the error is notified
* through ops.exit() with all the details.
*/
if (!scx_ops_tryset_enable_state(SCX_OPS_ENABLED, SCX_OPS_ENABLING)) {
WARN_ON_ONCE(atomic_read(&scx_exit_kind) == SCX_EXIT_NONE);
ret = 0;
@ -5212,8 +5238,7 @@ static int scx_ops_enable(struct sched_ext_ops *ops, struct bpf_link *link)
err_disable_unlock_all:
scx_cgroup_unlock();
percpu_up_write(&scx_fork_rwsem);
err_disable_unlock_cpus:
cpus_read_unlock();
scx_ops_bypass(false);
err_disable:
mutex_unlock(&scx_ops_enable_mutex);
/* must be fully disabled before returning */
@ -5782,7 +5807,6 @@ void __init init_sched_ext_class(void)
SCX_TG_ONLINE);
BUG_ON(rhashtable_init(&dsq_hash, &dsq_hash_params));
init_dsq(&scx_dsq_global, SCX_DSQ_GLOBAL);
#ifdef CONFIG_SMP
BUG_ON(!alloc_cpumask_var(&idle_masks.cpu, GFP_KERNEL));
BUG_ON(!alloc_cpumask_var(&idle_masks.smt, GFP_KERNEL));
@ -6058,7 +6082,7 @@ static bool scx_dispatch_from_dsq(struct bpf_iter_scx_dsq_kern *kit,
if (dst_dsq->id == SCX_DSQ_LOCAL) {
dst_rq = container_of(dst_dsq, struct rq, scx.local_dsq);
if (!task_can_run_on_remote_rq(p, dst_rq, true)) {
dst_dsq = &scx_dsq_global;
dst_dsq = find_global_dsq(p);
dst_rq = src_rq;
}
} else {
@ -6175,7 +6199,7 @@ __bpf_kfunc bool scx_bpf_consume(u64 dsq_id)
flush_dispatch_buf(dspc->rq);
dsq = find_non_local_dsq(dsq_id);
dsq = find_user_dsq(dsq_id);
if (unlikely(!dsq)) {
scx_ops_error("invalid DSQ ID 0x%016llx", dsq_id);
return false;
@ -6496,7 +6520,7 @@ __bpf_kfunc s32 scx_bpf_dsq_nr_queued(u64 dsq_id)
goto out;
}
} else {
dsq = find_non_local_dsq(dsq_id);
dsq = find_user_dsq(dsq_id);
if (dsq) {
ret = READ_ONCE(dsq->nr);
goto out;
@ -6545,7 +6569,7 @@ __bpf_kfunc int bpf_iter_scx_dsq_new(struct bpf_iter_scx_dsq *it, u64 dsq_id,
if (flags & ~__SCX_DSQ_ITER_USER_FLAGS)
return -EINVAL;
kit->dsq = find_non_local_dsq(dsq_id);
kit->dsq = find_user_dsq(dsq_id);
if (!kit->dsq)
return -ENOENT;

View file

@ -7,7 +7,13 @@
#ifndef __SCX_COMMON_BPF_H
#define __SCX_COMMON_BPF_H
#ifdef LSP
#define __bpf__
#include "../vmlinux/vmlinux.h"
#else
#include "vmlinux.h"
#endif
#include <bpf/bpf_helpers.h>
#include <bpf/bpf_tracing.h>
#include <asm-generic/errno.h>
@ -309,6 +315,15 @@ void bpf_cpumask_copy(struct bpf_cpumask *dst, const struct cpumask *src) __ksym
u32 bpf_cpumask_any_distribute(const struct cpumask *cpumask) __ksym;
u32 bpf_cpumask_any_and_distribute(const struct cpumask *src1,
const struct cpumask *src2) __ksym;
u32 bpf_cpumask_weight(const struct cpumask *cpumask) __ksym;
/*
* Access a cpumask in read-only mode (typically to check bits).
*/
const struct cpumask *cast_mask(struct bpf_cpumask *mask)
{
return (const struct cpumask *)mask;
}
/* rcu */
void bpf_rcu_read_lock(void) __ksym;

View file

@ -15,6 +15,25 @@
__ret; \
})
/* v6.12: 819513666966 ("sched_ext: Add cgroup support") */
#define __COMPAT_scx_bpf_task_cgroup(p) \
(bpf_ksym_exists(scx_bpf_task_cgroup) ? \
scx_bpf_task_cgroup((p)) : NULL)
/* v6.12: 4c30f5ce4f7a ("sched_ext: Implement scx_bpf_dispatch[_vtime]_from_dsq()") */
#define __COMPAT_scx_bpf_dispatch_from_dsq_set_slice(it, slice) \
(bpf_ksym_exists(scx_bpf_dispatch_from_dsq_set_slice) ? \
scx_bpf_dispatch_from_dsq_set_slice((it), (slice)) : (void)0)
#define __COMPAT_scx_bpf_dispatch_from_dsq_set_vtime(it, vtime) \
(bpf_ksym_exists(scx_bpf_dispatch_from_dsq_set_vtime) ? \
scx_bpf_dispatch_from_dsq_set_vtime((it), (vtime)) : (void)0)
#define __COMPAT_scx_bpf_dispatch_from_dsq(it, p, dsq_id, enq_flags) \
(bpf_ksym_exists(scx_bpf_dispatch_from_dsq) ? \
scx_bpf_dispatch_from_dsq((it), (p), (dsq_id), (enq_flags)) : false)
#define __COMPAT_scx_bpf_dispatch_vtime_from_dsq(it, p, dsq_id, enq_flags) \
(bpf_ksym_exists(scx_bpf_dispatch_vtime_from_dsq) ? \
scx_bpf_dispatch_vtime_from_dsq((it), (p), (dsq_id), (enq_flags)) : false)
/*
* Define sched_ext_ops. This may be expanded to define multiple variants for
* backward compatibility. See compat.h::SCX_OPS_LOAD/ATTACH().

View file

@ -25,7 +25,11 @@ struct user_exit_info {
#ifdef __bpf__
#ifdef LSP
#include "../vmlinux/vmlinux.h"
#else
#include "vmlinux.h"
#endif
#include <bpf/bpf_core_read.h>
#define UEI_DEFINE(__name) \

View file

@ -49,7 +49,10 @@
/*
* Maximum amount of retries to find a valid cgroup.
*/
#define CGROUP_MAX_RETRIES 1024
enum {
FALLBACK_DSQ = 0,
CGROUP_MAX_RETRIES = 1024,
};
char _license[] SEC("license") = "GPL";
@ -225,7 +228,7 @@ static void cgrp_refresh_hweight(struct cgroup *cgrp, struct fcg_cgrp_ctx *cgc)
break;
/*
* We can be oppotunistic here and not grab the
* We can be opportunistic here and not grab the
* cgv_tree_lock and deal with the occasional races.
* However, hweight updates are already cached and
* relatively low-frequency. Let's just do the
@ -258,8 +261,7 @@ static void cgrp_cap_budget(struct cgv_node *cgv_node, struct fcg_cgrp_ctx *cgc)
* and thus can't be updated and repositioned. Instead, we collect the
* vtime deltas separately and apply it asynchronously here.
*/
delta = cgc->cvtime_delta;
__sync_fetch_and_sub(&cgc->cvtime_delta, delta);
delta = __sync_fetch_and_sub(&cgc->cvtime_delta, cgc->cvtime_delta);
cvtime = cgv_node->cvtime + delta;
/*
@ -378,12 +380,12 @@ void BPF_STRUCT_OPS(fcg_enqueue, struct task_struct *p, u64 enq_flags)
scx_bpf_dispatch(p, SCX_DSQ_LOCAL, SCX_SLICE_DFL, enq_flags);
} else {
stat_inc(FCG_STAT_GLOBAL);
scx_bpf_dispatch(p, SCX_DSQ_GLOBAL, SCX_SLICE_DFL, enq_flags);
scx_bpf_dispatch(p, FALLBACK_DSQ, SCX_SLICE_DFL, enq_flags);
}
return;
}
cgrp = scx_bpf_task_cgroup(p);
cgrp = __COMPAT_scx_bpf_task_cgroup(p);
cgc = find_cgrp_ctx(cgrp);
if (!cgc)
goto out_release;
@ -509,7 +511,7 @@ void BPF_STRUCT_OPS(fcg_runnable, struct task_struct *p, u64 enq_flags)
{
struct cgroup *cgrp;
cgrp = scx_bpf_task_cgroup(p);
cgrp = __COMPAT_scx_bpf_task_cgroup(p);
update_active_weight_sums(cgrp, true);
bpf_cgroup_release(cgrp);
}
@ -522,7 +524,7 @@ void BPF_STRUCT_OPS(fcg_running, struct task_struct *p)
if (fifo_sched)
return;
cgrp = scx_bpf_task_cgroup(p);
cgrp = __COMPAT_scx_bpf_task_cgroup(p);
cgc = find_cgrp_ctx(cgrp);
if (cgc) {
/*
@ -565,7 +567,7 @@ void BPF_STRUCT_OPS(fcg_stopping, struct task_struct *p, bool runnable)
if (!taskc->bypassed_at)
return;
cgrp = scx_bpf_task_cgroup(p);
cgrp = __COMPAT_scx_bpf_task_cgroup(p);
cgc = find_cgrp_ctx(cgrp);
if (cgc) {
__sync_fetch_and_add(&cgc->cvtime_delta,
@ -579,7 +581,7 @@ void BPF_STRUCT_OPS(fcg_quiescent, struct task_struct *p, u64 deq_flags)
{
struct cgroup *cgrp;
cgrp = scx_bpf_task_cgroup(p);
cgrp = __COMPAT_scx_bpf_task_cgroup(p);
update_active_weight_sums(cgrp, false);
bpf_cgroup_release(cgrp);
}
@ -781,7 +783,7 @@ void BPF_STRUCT_OPS(fcg_dispatch, s32 cpu, struct task_struct *prev)
pick_next_cgroup:
cpuc->cur_at = now;
if (scx_bpf_consume(SCX_DSQ_GLOBAL)) {
if (scx_bpf_consume(FALLBACK_DSQ)) {
cpuc->cur_cgid = 0;
return;
}
@ -838,7 +840,7 @@ int BPF_STRUCT_OPS_SLEEPABLE(fcg_cgroup_init, struct cgroup *cgrp,
int ret;
/*
* Technically incorrect as cgroup ID is full 64bit while dq ID is
* Technically incorrect as cgroup ID is full 64bit while dsq ID is
* 63bit. Should not be a problem in practice and easy to spot in the
* unlikely case that it breaks.
*/
@ -926,6 +928,11 @@ void BPF_STRUCT_OPS(fcg_cgroup_move, struct task_struct *p,
p->scx.dsq_vtime = to_cgc->tvtime_now + vtime_delta;
}
s32 BPF_STRUCT_OPS_SLEEPABLE(fcg_init)
{
return scx_bpf_create_dsq(FALLBACK_DSQ, -1);
}
void BPF_STRUCT_OPS(fcg_exit, struct scx_exit_info *ei)
{
UEI_RECORD(uei, ei);
@ -944,6 +951,7 @@ SCX_OPS_DEFINE(flatcg_ops,
.cgroup_init = (void *)fcg_cgroup_init,
.cgroup_exit = (void *)fcg_cgroup_exit,
.cgroup_move = (void *)fcg_cgroup_move,
.init = (void *)fcg_init,
.exit = (void *)fcg_exit,
.flags = SCX_OPS_HAS_CGROUP_WEIGHT | SCX_OPS_ENQ_EXITING,
.name = "flatcg");

View file

@ -318,11 +318,11 @@ static bool dispatch_highpri(bool from_timer)
if (tctx->highpri) {
/* exercise the set_*() and vtime interface too */
scx_bpf_dispatch_from_dsq_set_slice(
__COMPAT_scx_bpf_dispatch_from_dsq_set_slice(
BPF_FOR_EACH_ITER, slice_ns * 2);
scx_bpf_dispatch_from_dsq_set_vtime(
__COMPAT_scx_bpf_dispatch_from_dsq_set_vtime(
BPF_FOR_EACH_ITER, highpri_seq++);
scx_bpf_dispatch_vtime_from_dsq(
__COMPAT_scx_bpf_dispatch_vtime_from_dsq(
BPF_FOR_EACH_ITER, p, HIGHPRI_DSQ, 0);
}
}
@ -340,9 +340,9 @@ static bool dispatch_highpri(bool from_timer)
else
cpu = scx_bpf_pick_any_cpu(p->cpus_ptr, 0);
if (scx_bpf_dispatch_from_dsq(BPF_FOR_EACH_ITER, p,
SCX_DSQ_LOCAL_ON | cpu,
SCX_ENQ_PREEMPT)) {
if (__COMPAT_scx_bpf_dispatch_from_dsq(BPF_FOR_EACH_ITER, p,
SCX_DSQ_LOCAL_ON | cpu,
SCX_ENQ_PREEMPT)) {
if (cpu == this_cpu) {
dispatched = true;
__sync_fetch_and_add(&nr_expedited_local, 1);