``scx_bpf_destroy_dsq()``.
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
-local DSQ.
+DSQ. A task in a non-local DSQ is "move"d into the target CPU's local DSQ.
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
-global DSQ. If that doesn't yield a runnable task either, ``ops.dispatch()``
-is invoked.
+empty, the first task is picked. Otherwise, the CPU tries to move a task
+from the global DSQ. If that doesn't yield a runnable task either,
+``ops.dispatch()`` is invoked.
Scheduling Cycle
----------------
rather than performing them immediately. There can be up to
``ops.dispatch_max_batch`` pending tasks.
- * ``scx_bpf_consume()`` tranfers a task from the specified non-local DSQ
- to the dispatching DSQ. This function cannot be called with any BPF
- locks held. ``scx_bpf_consume()`` flushes the pending dispatched tasks
- before trying to consume the specified DSQ.
+ * ``scx_bpf_move_to_local()`` moves a task from the specified non-local
+ DSQ to the dispatching DSQ. This function cannot be called with any BPF
+ locks held. ``scx_bpf_move_to_local()`` flushes the pending insertions
+ tasks before trying to move from the specified 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:
- * 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.
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
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_vtime()`` for the priority queue. Internal DSQs such as
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
* @prev: previous task being switched out
*
* 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
- * scx_bpf_dsq_insert() and/or consume user DSQs into the local DSQ
- * using scx_bpf_consume().
+ * scx_bpf_dsq_insert() and/or move from user DSQs into the local DSQ
+ * using scx_bpf_dsq_move_to_local().
*
* 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
* for more details.
*
* @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
* 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);
* @enq_flags: SCX_ENQ_*
*
* 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
- * consumed after the tasks in the FIFO queue. All other aspects are identical
- * to scx_bpf_dsq_insert().
+ * Tasks queued into the priority queue are ordered by @vtime. All other aspects
+ * are identical to scx_bpf_dsq_insert().
*
* @vtime ordering is according to time_before64() which considers wrapping. A
* numerically larger vtime may indicate an earlier position in the ordering and
* 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,
u64 slice, u64 vtime, u64 enq_flags)
}
/**
- * scx_bpf_consume - Transfer a task from a DSQ to the current CPU's local DSQ
- * @dsq_id: DSQ to consume
+ * scx_bpf_dsq_move_to_local - move a task from a DSQ to the current CPU's local DSQ
+ * @dsq_id: DSQ to move task from
*
- * Consume a task from the non-local DSQ identified by @dsq_id and transfer it
- * to the current CPU's local DSQ for execution. Can only be called from
- * ops.dispatch().
+ * Move a task from the non-local DSQ identified by @dsq_id to the current CPU's
+ * local DSQ for execution. Can only be called from ops.dispatch().
*
* 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.
*
- * Returns %true if a task has been consumed, %false if there isn't any task to
- * consume.
+ * Returns %true if a task has been moved, %false if there isn't any task to
+ * 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_dispatch_q *dsq;
}
}
+/* 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
* @it__iter: DSQ iterator in progress
BTF_KFUNCS_START(scx_kfunc_ids_dispatch)
BTF_ID_FLAGS(func, scx_bpf_dispatch_nr_slots)
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_dispatch_from_dsq_set_slice)
BTF_ID_FLAGS(func, scx_bpf_dispatch_from_dsq_set_vtime)
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;
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_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;
*/
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;
+bool scx_bpf_consume___compat(u64 dsq_id) __ksym __weak;
#define scx_bpf_dsq_insert(p, dsq_id, slice, enq_flags) \
(bpf_ksym_exists(scx_bpf_dsq_insert) ? \
scx_bpf_dsq_insert_vtime((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) \
_Static_assert(false, "scx_bpf_dispatch() renamed to scx_bpf_dsq_insert()")
#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()")
+#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
* backward compatibility. See compat.h::SCX_OPS_LOAD/ATTACH().
}
/* 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;
dispatch_to_cpu(central_cpu);
} else {
bool *gimme;
- if (scx_bpf_consume(FALLBACK_DSQ_ID))
+ if (scx_bpf_dsq_move_to_local(FALLBACK_DSQ_ID))
return;
gimme = ARRAY_ELEM_PTR(cpu_gimme_task, cpu, nr_cpu_ids);
goto out_free;
}
- if (!scx_bpf_consume(cgid)) {
+ if (!scx_bpf_dsq_move_to_local(cgid)) {
bpf_cgroup_release(cgrp);
stat_inc(FCG_STAT_PNC_EMPTY);
goto out_stash;
goto pick_next_cgroup;
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);
return;
}
pick_next_cgroup:
cpuc->cur_at = now;
- if (scx_bpf_consume(FALLBACK_DSQ)) {
+ if (scx_bpf_dsq_move_to_local(FALLBACK_DSQ)) {
cpuc->cur_cgid = 0;
return;
}
if (dispatch_highpri(false))
return;
- if (!nr_highpri_queued && scx_bpf_consume(SHARED_DSQ))
+ if (!nr_highpri_queued && scx_bpf_dsq_move_to_local(SHARED_DSQ))
return;
if (dsp_inf_loop_after && nr_dispatched > dsp_inf_loop_after) {
if (!batch || !scx_bpf_dispatch_nr_slots()) {
if (dispatch_highpri(false))
return;
- scx_bpf_consume(SHARED_DSQ);
+ scx_bpf_dsq_move_to_local(SHARED_DSQ);
return;
}
if (!cpuc->dsp_cnt)
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)