sched_ext: Rename scx_bpf_consume() to scx_bpf_dsq_move_to_local()

In sched_ext API, a repeatedly reported pain point is the overuse of the
verb "dispatch" and confusion around "consume":

- ops.dispatch()
- scx_bpf_dispatch[_vtime]()
- scx_bpf_consume()
- scx_bpf_dispatch[_vtime]_from_dsq*()

This overloading of the term is historical. Originally, there were only
built-in DSQs and moving a task into a DSQ always dispatched it for
execution. Using the verb "dispatch" for the kfuncs to move tasks into these
DSQs made sense.

Later, user DSQs were added and scx_bpf_dispatch[_vtime]() updated to be
able to insert tasks into any DSQ. The only allowed DSQ to DSQ transfer was
from a non-local DSQ to a local DSQ and this operation was named "consume".
This was already confusing as a task could be dispatched to a user DSQ from
ops.enqueue() and then the DSQ would have to be consumed in ops.dispatch().
Later addition of scx_bpf_dispatch_from_dsq*() made the confusion even worse
as "dispatch" in this context meant moving a task to an arbitrary DSQ from a
user DSQ.

Clean up the API with the following renames:

1. scx_bpf_dispatch[_vtime]()		-> scx_bpf_dsq_insert[_vtime]()
2. scx_bpf_consume()			-> scx_bpf_dsq_move_to_local()
3. scx_bpf_dispatch[_vtime]_from_dsq*()	-> scx_bpf_dsq_move[_vtime]*()

This patch performs the second rename. Compatibility is maintained by:

- The previous kfunc names are still provided by the kernel so that old
  binaries can run. Kernel generates a warning when the old names are used.

- compat.bpf.h provides wrappers for the new names which automatically fall
  back to the old names when running on older kernels. They also trigger
  build error if old names are used for new builds.

The compat features will be dropped after v6.15.

v2: Comment and documentation updates.

Signed-off-by: Tejun Heo <tj@kernel.org>
Acked-by: Andrea Righi <arighi@nvidia.com>
Acked-by: Changwoo Min <changwoo@igalia.com>
Acked-by: Johannes Bechberger <me@mostlynerdless.de>
Acked-by: Giovanni Gherdovich <ggherdovich@suse.com>
Cc: Dan Schatzberg <dschatzberg@meta.com>
Cc: Ming Yang <yougmark94@gmail.com>
This commit is contained in:
Tejun Heo 2024-11-11 07:06:16 -10:00
parent cc26abb1a1
commit 5209c03c8e
8 changed files with 58 additions and 37 deletions

View File

@ -203,13 +203,12 @@ an arbitrary number of dsq's using ``scx_bpf_create_dsq()`` and
``scx_bpf_destroy_dsq()``. ``scx_bpf_destroy_dsq()``.
A CPU always executes a task from its local DSQ. A task is "inserted" into a A CPU always executes a task from its local DSQ. A task is "inserted" into a
DSQ. A non-local DSQ is "consumed" to transfer a task to the consuming CPU's DSQ. A task in a non-local DSQ is "move"d into the target CPU's local DSQ.
local DSQ.
When a CPU is looking for the next task to run, if the local DSQ is not When a CPU is looking for the next task to run, if the local DSQ is not
empty, the first task is picked. Otherwise, the CPU tries to consume the empty, the first task is picked. Otherwise, the CPU tries to move a task
global DSQ. If that doesn't yield a runnable task either, ``ops.dispatch()`` from the global DSQ. If that doesn't yield a runnable task either,
is invoked. ``ops.dispatch()`` is invoked.
Scheduling Cycle Scheduling Cycle
---------------- ----------------
@ -265,15 +264,15 @@ The following briefly shows how a waking task is scheduled and executed.
rather than performing them immediately. There can be up to rather than performing them immediately. There can be up to
``ops.dispatch_max_batch`` pending tasks. ``ops.dispatch_max_batch`` pending tasks.
* ``scx_bpf_consume()`` tranfers a task from the specified non-local DSQ * ``scx_bpf_move_to_local()`` moves a task from the specified non-local
to the dispatching DSQ. This function cannot be called with any BPF DSQ to the dispatching DSQ. This function cannot be called with any BPF
locks held. ``scx_bpf_consume()`` flushes the pending dispatched tasks locks held. ``scx_bpf_move_to_local()`` flushes the pending insertions
before trying to consume the specified DSQ. tasks before trying to move from the specified DSQ.
4. After ``ops.dispatch()`` returns, if there are tasks in the local DSQ, 4. After ``ops.dispatch()`` returns, if there are tasks in the local DSQ,
the CPU runs the first one. If empty, the following steps are taken: the CPU runs the first one. If empty, the following steps are taken:
* Try to consume the global DSQ. If successful, run the task. * Try to move from the global DSQ. If successful, run the task.
* If ``ops.dispatch()`` has dispatched any tasks, retry #3. * If ``ops.dispatch()`` has dispatched any tasks, retry #3.
@ -286,7 +285,7 @@ Note that the BPF scheduler can always choose to dispatch tasks immediately
in ``ops.enqueue()`` as illustrated in the above simple example. If only the in ``ops.enqueue()`` as illustrated in the above simple example. If only the
built-in DSQs are used, there is no need to implement ``ops.dispatch()`` as built-in DSQs are used, there is no need to implement ``ops.dispatch()`` as
a task is never queued on the BPF scheduler and both the local and global a task is never queued on the BPF scheduler and both the local and global
DSQs are consumed automatically. DSQs are executed automatically.
``scx_bpf_dsq_insert()`` inserts the task on the FIFO of the target DSQ. Use ``scx_bpf_dsq_insert()`` inserts the task on the FIFO of the target DSQ. Use
``scx_bpf_dsq_insert_vtime()`` for the priority queue. Internal DSQs such as ``scx_bpf_dsq_insert_vtime()`` for the priority queue. Internal DSQs such as

View File

@ -264,17 +264,17 @@ struct sched_ext_ops {
void (*dequeue)(struct task_struct *p, u64 deq_flags); void (*dequeue)(struct task_struct *p, u64 deq_flags);
/** /**
* dispatch - Dispatch tasks from the BPF scheduler and/or consume DSQs * dispatch - Dispatch tasks from the BPF scheduler and/or user DSQs
* @cpu: CPU to dispatch tasks for * @cpu: CPU to dispatch tasks for
* @prev: previous task being switched out * @prev: previous task being switched out
* *
* Called when a CPU's local dsq is empty. The operation should dispatch * Called when a CPU's local dsq is empty. The operation should dispatch
* one or more tasks from the BPF scheduler into the DSQs using * one or more tasks from the BPF scheduler into the DSQs using
* scx_bpf_dsq_insert() and/or consume user DSQs into the local DSQ * scx_bpf_dsq_insert() and/or move from user DSQs into the local DSQ
* using scx_bpf_consume(). * using scx_bpf_dsq_move_to_local().
* *
* The maximum number of times scx_bpf_dsq_insert() can be called * The maximum number of times scx_bpf_dsq_insert() can be called
* without an intervening scx_bpf_consume() is specified by * without an intervening scx_bpf_dsq_move_to_local() is specified by
* ops.dispatch_max_batch. See the comments on top of the two functions * ops.dispatch_max_batch. See the comments on top of the two functions
* for more details. * for more details.
* *
@ -282,7 +282,7 @@ struct sched_ext_ops {
* @prev is still runnable as indicated by set %SCX_TASK_QUEUED in * @prev is still runnable as indicated by set %SCX_TASK_QUEUED in
* @prev->scx.flags, it is not enqueued yet and will be enqueued after * @prev->scx.flags, it is not enqueued yet and will be enqueued after
* ops.dispatch() returns. To keep executing @prev, return without * ops.dispatch() returns. To keep executing @prev, return without
* dispatching or consuming any tasks. Also see %SCX_OPS_ENQ_LAST. * dispatching or moving any tasks. Also see %SCX_OPS_ENQ_LAST.
*/ */
void (*dispatch)(s32 cpu, struct task_struct *prev); void (*dispatch)(s32 cpu, struct task_struct *prev);
@ -6372,13 +6372,17 @@ __bpf_kfunc void scx_bpf_dispatch(struct task_struct *p, u64 dsq_id, u64 slice,
* @enq_flags: SCX_ENQ_* * @enq_flags: SCX_ENQ_*
* *
* Insert @p into the vtime priority queue of the DSQ identified by @dsq_id. * Insert @p into the vtime priority queue of the DSQ identified by @dsq_id.
* Tasks queued into the priority queue are ordered by @vtime and always * Tasks queued into the priority queue are ordered by @vtime. All other aspects
* consumed after the tasks in the FIFO queue. All other aspects are identical * are identical to scx_bpf_dsq_insert().
* to scx_bpf_dsq_insert().
* *
* @vtime ordering is according to time_before64() which considers wrapping. A * @vtime ordering is according to time_before64() which considers wrapping. A
* numerically larger vtime may indicate an earlier position in the ordering and * numerically larger vtime may indicate an earlier position in the ordering and
* vice-versa. * vice-versa.
*
* A DSQ can only be used as a FIFO or priority queue at any given time and this
* function must not be called on a DSQ which already has one or more FIFO tasks
* queued and vice-versa. Also, the built-in DSQs (SCX_DSQ_LOCAL and
* SCX_DSQ_GLOBAL) cannot be used as priority queues.
*/ */
__bpf_kfunc void scx_bpf_dsq_insert_vtime(struct task_struct *p, u64 dsq_id, __bpf_kfunc void scx_bpf_dsq_insert_vtime(struct task_struct *p, u64 dsq_id,
u64 slice, u64 vtime, u64 enq_flags) u64 slice, u64 vtime, u64 enq_flags)
@ -6539,21 +6543,20 @@ __bpf_kfunc void scx_bpf_dispatch_cancel(void)
} }
/** /**
* scx_bpf_consume - Transfer a task from a DSQ to the current CPU's local DSQ * scx_bpf_dsq_move_to_local - move a task from a DSQ to the current CPU's local DSQ
* @dsq_id: DSQ to consume * @dsq_id: DSQ to move task from
* *
* Consume a task from the non-local DSQ identified by @dsq_id and transfer it * Move a task from the non-local DSQ identified by @dsq_id to the current CPU's
* to the current CPU's local DSQ for execution. Can only be called from * local DSQ for execution. Can only be called from ops.dispatch().
* ops.dispatch().
* *
* This function flushes the in-flight dispatches from scx_bpf_dsq_insert() * This function flushes the in-flight dispatches from scx_bpf_dsq_insert()
* before trying to consume the specified DSQ. It may also grab rq locks and * before trying to move from the specified DSQ. It may also grab rq locks and
* thus can't be called under any BPF locks. * thus can't be called under any BPF locks.
* *
* Returns %true if a task has been consumed, %false if there isn't any task to * Returns %true if a task has been moved, %false if there isn't any task to
* consume. * move.
*/ */
__bpf_kfunc bool scx_bpf_consume(u64 dsq_id) __bpf_kfunc bool scx_bpf_dsq_move_to_local(u64 dsq_id)
{ {
struct scx_dsp_ctx *dspc = this_cpu_ptr(scx_dsp_ctx); struct scx_dsp_ctx *dspc = this_cpu_ptr(scx_dsp_ctx);
struct scx_dispatch_q *dsq; struct scx_dispatch_q *dsq;
@ -6583,6 +6586,13 @@ __bpf_kfunc bool scx_bpf_consume(u64 dsq_id)
} }
} }
/* for backward compatibility, will be removed in v6.15 */
__bpf_kfunc bool scx_bpf_consume(u64 dsq_id)
{
printk_deferred_once(KERN_WARNING "sched_ext: scx_bpf_consume() renamed to scx_bpf_dsq_move_to_local()");
return scx_bpf_dsq_move_to_local(dsq_id);
}
/** /**
* scx_bpf_dispatch_from_dsq_set_slice - Override slice when dispatching from DSQ * scx_bpf_dispatch_from_dsq_set_slice - Override slice when dispatching from DSQ
* @it__iter: DSQ iterator in progress * @it__iter: DSQ iterator in progress
@ -6684,6 +6694,7 @@ __bpf_kfunc_end_defs();
BTF_KFUNCS_START(scx_kfunc_ids_dispatch) BTF_KFUNCS_START(scx_kfunc_ids_dispatch)
BTF_ID_FLAGS(func, scx_bpf_dispatch_nr_slots) BTF_ID_FLAGS(func, scx_bpf_dispatch_nr_slots)
BTF_ID_FLAGS(func, scx_bpf_dispatch_cancel) BTF_ID_FLAGS(func, scx_bpf_dispatch_cancel)
BTF_ID_FLAGS(func, scx_bpf_dsq_move_to_local)
BTF_ID_FLAGS(func, scx_bpf_consume) BTF_ID_FLAGS(func, scx_bpf_consume)
BTF_ID_FLAGS(func, scx_bpf_dispatch_from_dsq_set_slice) BTF_ID_FLAGS(func, scx_bpf_dispatch_from_dsq_set_slice)
BTF_ID_FLAGS(func, scx_bpf_dispatch_from_dsq_set_vtime) BTF_ID_FLAGS(func, scx_bpf_dispatch_from_dsq_set_vtime)

View File

@ -40,7 +40,7 @@ void scx_bpf_dsq_insert(struct task_struct *p, u64 dsq_id, u64 slice, u64 enq_fl
void scx_bpf_dsq_insert_vtime(struct task_struct *p, u64 dsq_id, u64 slice, u64 vtime, u64 enq_flags) __ksym __weak; void scx_bpf_dsq_insert_vtime(struct task_struct *p, u64 dsq_id, u64 slice, u64 vtime, u64 enq_flags) __ksym __weak;
u32 scx_bpf_dispatch_nr_slots(void) __ksym; u32 scx_bpf_dispatch_nr_slots(void) __ksym;
void scx_bpf_dispatch_cancel(void) __ksym; void scx_bpf_dispatch_cancel(void) __ksym;
bool scx_bpf_consume(u64 dsq_id) __ksym; bool scx_bpf_dsq_move_to_local(u64 dsq_id) __ksym;
void scx_bpf_dispatch_from_dsq_set_slice(struct bpf_iter_scx_dsq *it__iter, u64 slice) __ksym __weak; void scx_bpf_dispatch_from_dsq_set_slice(struct bpf_iter_scx_dsq *it__iter, u64 slice) __ksym __weak;
void scx_bpf_dispatch_from_dsq_set_vtime(struct bpf_iter_scx_dsq *it__iter, u64 vtime) __ksym __weak; void scx_bpf_dispatch_from_dsq_set_vtime(struct bpf_iter_scx_dsq *it__iter, u64 vtime) __ksym __weak;
bool scx_bpf_dispatch_from_dsq(struct bpf_iter_scx_dsq *it__iter, struct task_struct *p, u64 dsq_id, u64 enq_flags) __ksym __weak; bool scx_bpf_dispatch_from_dsq(struct bpf_iter_scx_dsq *it__iter, struct task_struct *p, u64 dsq_id, u64 enq_flags) __ksym __weak;

View File

@ -43,6 +43,7 @@
*/ */
void scx_bpf_dispatch___compat(struct task_struct *p, u64 dsq_id, u64 slice, u64 enq_flags) __ksym __weak; void scx_bpf_dispatch___compat(struct task_struct *p, u64 dsq_id, u64 slice, u64 enq_flags) __ksym __weak;
void scx_bpf_dispatch_vtime___compat(struct task_struct *p, u64 dsq_id, u64 slice, u64 vtime, u64 enq_flags) __ksym __weak; void scx_bpf_dispatch_vtime___compat(struct task_struct *p, u64 dsq_id, u64 slice, u64 vtime, u64 enq_flags) __ksym __weak;
bool scx_bpf_consume___compat(u64 dsq_id) __ksym __weak;
#define scx_bpf_dsq_insert(p, dsq_id, slice, enq_flags) \ #define scx_bpf_dsq_insert(p, dsq_id, slice, enq_flags) \
(bpf_ksym_exists(scx_bpf_dsq_insert) ? \ (bpf_ksym_exists(scx_bpf_dsq_insert) ? \
@ -54,12 +55,22 @@ void scx_bpf_dispatch_vtime___compat(struct task_struct *p, u64 dsq_id, u64 slic
scx_bpf_dsq_insert_vtime((p), (dsq_id), (slice), (vtime), (enq_flags)) : \ scx_bpf_dsq_insert_vtime((p), (dsq_id), (slice), (vtime), (enq_flags)) : \
scx_bpf_dispatch_vtime___compat((p), (dsq_id), (slice), (vtime), (enq_flags))) scx_bpf_dispatch_vtime___compat((p), (dsq_id), (slice), (vtime), (enq_flags)))
#define scx_bpf_dsq_move_to_local(dsq_id) \
(bpf_ksym_exists(scx_bpf_dsq_move_to_local) ? \
scx_bpf_dsq_move_to_local((dsq_id)) : \
scx_bpf_consume___compat((dsq_id)))
#define scx_bpf_dispatch(p, dsq_id, slice, enq_flags) \ #define scx_bpf_dispatch(p, dsq_id, slice, enq_flags) \
_Static_assert(false, "scx_bpf_dispatch() renamed to scx_bpf_dsq_insert()") _Static_assert(false, "scx_bpf_dispatch() renamed to scx_bpf_dsq_insert()")
#define scx_bpf_dispatch_vtime(p, dsq_id, slice, vtime, enq_flags) \ #define scx_bpf_dispatch_vtime(p, dsq_id, slice, vtime, enq_flags) \
_Static_assert(false, "scx_bpf_dispatch_vtime() renamed to scx_bpf_dsq_insert_vtime()") _Static_assert(false, "scx_bpf_dispatch_vtime() renamed to scx_bpf_dsq_insert_vtime()")
#define scx_bpf_consume(dsq_id) ({ \
_Static_assert(false, "scx_bpf_consume() renamed to scx_bpf_dsq_move_to_local()"); \
false; \
})
/* /*
* Define sched_ext_ops. This may be expanded to define multiple variants for * Define sched_ext_ops. This may be expanded to define multiple variants for
* backward compatibility. See compat.h::SCX_OPS_LOAD/ATTACH(). * backward compatibility. See compat.h::SCX_OPS_LOAD/ATTACH().

View File

@ -219,13 +219,13 @@ void BPF_STRUCT_OPS(central_dispatch, s32 cpu, struct task_struct *prev)
} }
/* look for a task to run on the central CPU */ /* look for a task to run on the central CPU */
if (scx_bpf_consume(FALLBACK_DSQ_ID)) if (scx_bpf_dsq_move_to_local(FALLBACK_DSQ_ID))
return; return;
dispatch_to_cpu(central_cpu); dispatch_to_cpu(central_cpu);
} else { } else {
bool *gimme; bool *gimme;
if (scx_bpf_consume(FALLBACK_DSQ_ID)) if (scx_bpf_dsq_move_to_local(FALLBACK_DSQ_ID))
return; return;
gimme = ARRAY_ELEM_PTR(cpu_gimme_task, cpu, nr_cpu_ids); gimme = ARRAY_ELEM_PTR(cpu_gimme_task, cpu, nr_cpu_ids);

View File

@ -665,7 +665,7 @@ static bool try_pick_next_cgroup(u64 *cgidp)
goto out_free; goto out_free;
} }
if (!scx_bpf_consume(cgid)) { if (!scx_bpf_dsq_move_to_local(cgid)) {
bpf_cgroup_release(cgrp); bpf_cgroup_release(cgrp);
stat_inc(FCG_STAT_PNC_EMPTY); stat_inc(FCG_STAT_PNC_EMPTY);
goto out_stash; goto out_stash;
@ -745,7 +745,7 @@ void BPF_STRUCT_OPS(fcg_dispatch, s32 cpu, struct task_struct *prev)
goto pick_next_cgroup; goto pick_next_cgroup;
if (vtime_before(now, cpuc->cur_at + cgrp_slice_ns)) { if (vtime_before(now, cpuc->cur_at + cgrp_slice_ns)) {
if (scx_bpf_consume(cpuc->cur_cgid)) { if (scx_bpf_dsq_move_to_local(cpuc->cur_cgid)) {
stat_inc(FCG_STAT_CNS_KEEP); stat_inc(FCG_STAT_CNS_KEEP);
return; return;
} }
@ -785,7 +785,7 @@ void BPF_STRUCT_OPS(fcg_dispatch, s32 cpu, struct task_struct *prev)
pick_next_cgroup: pick_next_cgroup:
cpuc->cur_at = now; cpuc->cur_at = now;
if (scx_bpf_consume(FALLBACK_DSQ)) { if (scx_bpf_dsq_move_to_local(FALLBACK_DSQ)) {
cpuc->cur_cgid = 0; cpuc->cur_cgid = 0;
return; return;
} }

View File

@ -374,7 +374,7 @@ void BPF_STRUCT_OPS(qmap_dispatch, s32 cpu, struct task_struct *prev)
if (dispatch_highpri(false)) if (dispatch_highpri(false))
return; return;
if (!nr_highpri_queued && scx_bpf_consume(SHARED_DSQ)) if (!nr_highpri_queued && scx_bpf_dsq_move_to_local(SHARED_DSQ))
return; return;
if (dsp_inf_loop_after && nr_dispatched > dsp_inf_loop_after) { if (dsp_inf_loop_after && nr_dispatched > dsp_inf_loop_after) {
@ -439,7 +439,7 @@ void BPF_STRUCT_OPS(qmap_dispatch, s32 cpu, struct task_struct *prev)
if (!batch || !scx_bpf_dispatch_nr_slots()) { if (!batch || !scx_bpf_dispatch_nr_slots()) {
if (dispatch_highpri(false)) if (dispatch_highpri(false))
return; return;
scx_bpf_consume(SHARED_DSQ); scx_bpf_dsq_move_to_local(SHARED_DSQ);
return; return;
} }
if (!cpuc->dsp_cnt) if (!cpuc->dsp_cnt)

View File

@ -94,7 +94,7 @@ void BPF_STRUCT_OPS(simple_enqueue, struct task_struct *p, u64 enq_flags)
void BPF_STRUCT_OPS(simple_dispatch, s32 cpu, struct task_struct *prev) void BPF_STRUCT_OPS(simple_dispatch, s32 cpu, struct task_struct *prev)
{ {
scx_bpf_consume(SHARED_DSQ); scx_bpf_dsq_move_to_local(SHARED_DSQ);
} }
void BPF_STRUCT_OPS(simple_running, struct task_struct *p) void BPF_STRUCT_OPS(simple_running, struct task_struct *p)