summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--kernel/sched/ext.c320
-rw-r--r--tools/sched_ext/include/scx/common.bpf.h15
-rw-r--r--tools/sched_ext/include/scx/compat.bpf.h19
-rw-r--r--tools/sched_ext/include/scx/user_exit_info.h4
-rw-r--r--tools/sched_ext/scx_flatcg.bpf.c32
-rw-r--r--tools/sched_ext/scx_qmap.bpf.c12
6 files changed, 236 insertions, 166 deletions
diff --git a/kernel/sched/ext.c b/kernel/sched/ext.c
index c09e3dc38c34..3cd7c50a51c5 100644
--- a/kernel/sched/ext.c
+++ b/kernel/sched/ext.c
@@ -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 @@ local_norefill:
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 @@ retry:
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 @@ err:
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.
- */
- preempt_disable();
-
- /*
- * 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.
+ * All tasks are READY. It's safe to turn on scx_enabled() and switch
+ * all eligible tasks.
*/
- 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;
- }
+ WRITE_ONCE(scx_switching_all, !(ops->flags & SCX_OPS_SWITCH_PARTIAL));
+ static_branch_enable(&__scx_ops_enabled);
/*
- * 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 @@ err_unlock:
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;
diff --git a/tools/sched_ext/include/scx/common.bpf.h b/tools/sched_ext/include/scx/common.bpf.h
index f538c75db183..225f61f9bfca 100644
--- a/tools/sched_ext/include/scx/common.bpf.h
+++ b/tools/sched_ext/include/scx/common.bpf.h
@@ -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;
diff --git a/tools/sched_ext/include/scx/compat.bpf.h b/tools/sched_ext/include/scx/compat.bpf.h
index 3d2fe1208900..e5afe9efd3f3 100644
--- a/tools/sched_ext/include/scx/compat.bpf.h
+++ b/tools/sched_ext/include/scx/compat.bpf.h
@@ -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().
diff --git a/tools/sched_ext/include/scx/user_exit_info.h b/tools/sched_ext/include/scx/user_exit_info.h
index 891693ee604e..8ce2734402e1 100644
--- a/tools/sched_ext/include/scx/user_exit_info.h
+++ b/tools/sched_ext/include/scx/user_exit_info.h
@@ -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) \
diff --git a/tools/sched_ext/scx_flatcg.bpf.c b/tools/sched_ext/scx_flatcg.bpf.c
index 3ab2b60781a0..b722baf6da4b 100644
--- a/tools/sched_ext/scx_flatcg.bpf.c
+++ b/tools/sched_ext/scx_flatcg.bpf.c
@@ -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");
diff --git a/tools/sched_ext/scx_qmap.bpf.c b/tools/sched_ext/scx_qmap.bpf.c
index 83c8f54c1e31..67e2a7968cc9 100644
--- a/tools/sched_ext/scx_qmap.bpf.c
+++ b/tools/sched_ext/scx_qmap.bpf.c
@@ -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);