mirror of
git://git.kernel.org/pub/scm/linux/kernel/git/torvalds/linux.git
synced 2025-09-04 20:19:47 +08:00
sched_ext: Rename scx_bpf_dispatch[_vtime]() to scx_bpf_dsq_insert[_vtime]()
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 first set of renames. 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: 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:
parent
72b85bf6a7
commit
cc26abb1a1
@ -130,7 +130,7 @@ optional. The following modified excerpt is from
|
|||||||
* Decide which CPU a task should be migrated to before being
|
* Decide which CPU a task should be migrated to before being
|
||||||
* enqueued (either at wakeup, fork time, or exec time). If an
|
* enqueued (either at wakeup, fork time, or exec time). If an
|
||||||
* idle core is found by the default ops.select_cpu() implementation,
|
* idle core is found by the default ops.select_cpu() implementation,
|
||||||
* then dispatch the task directly to SCX_DSQ_LOCAL and skip the
|
* then insert the task directly into SCX_DSQ_LOCAL and skip the
|
||||||
* ops.enqueue() callback.
|
* ops.enqueue() callback.
|
||||||
*
|
*
|
||||||
* Note that this implementation has exactly the same behavior as the
|
* Note that this implementation has exactly the same behavior as the
|
||||||
@ -148,15 +148,15 @@ optional. The following modified excerpt is from
|
|||||||
cpu = scx_bpf_select_cpu_dfl(p, prev_cpu, wake_flags, &direct);
|
cpu = scx_bpf_select_cpu_dfl(p, prev_cpu, wake_flags, &direct);
|
||||||
|
|
||||||
if (direct)
|
if (direct)
|
||||||
scx_bpf_dispatch(p, SCX_DSQ_LOCAL, SCX_SLICE_DFL, 0);
|
scx_bpf_dsq_insert(p, SCX_DSQ_LOCAL, SCX_SLICE_DFL, 0);
|
||||||
|
|
||||||
return cpu;
|
return cpu;
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Do a direct dispatch of a task to the global DSQ. This ops.enqueue()
|
* Do a direct insertion of a task to the global DSQ. This ops.enqueue()
|
||||||
* callback will only be invoked if we failed to find a core to dispatch
|
* callback will only be invoked if we failed to find a core to insert
|
||||||
* to in ops.select_cpu() above.
|
* into in ops.select_cpu() above.
|
||||||
*
|
*
|
||||||
* Note that this implementation has exactly the same behavior as the
|
* Note that this implementation has exactly the same behavior as the
|
||||||
* default ops.enqueue implementation, which just dispatches the task
|
* default ops.enqueue implementation, which just dispatches the task
|
||||||
@ -166,7 +166,7 @@ optional. The following modified excerpt is from
|
|||||||
*/
|
*/
|
||||||
void BPF_STRUCT_OPS(simple_enqueue, struct task_struct *p, u64 enq_flags)
|
void BPF_STRUCT_OPS(simple_enqueue, struct task_struct *p, u64 enq_flags)
|
||||||
{
|
{
|
||||||
scx_bpf_dispatch(p, SCX_DSQ_GLOBAL, SCX_SLICE_DFL, enq_flags);
|
scx_bpf_dsq_insert(p, SCX_DSQ_GLOBAL, SCX_SLICE_DFL, enq_flags);
|
||||||
}
|
}
|
||||||
|
|
||||||
s32 BPF_STRUCT_OPS_SLEEPABLE(simple_init)
|
s32 BPF_STRUCT_OPS_SLEEPABLE(simple_init)
|
||||||
@ -202,7 +202,7 @@ and one local dsq per CPU (``SCX_DSQ_LOCAL``). The BPF scheduler can manage
|
|||||||
an arbitrary number of dsq's using ``scx_bpf_create_dsq()`` and
|
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 "dispatched" to 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 non-local DSQ is "consumed" to transfer a task to the consuming CPU's
|
||||||
local DSQ.
|
local DSQ.
|
||||||
|
|
||||||
@ -229,26 +229,26 @@ The following briefly shows how a waking task is scheduled and executed.
|
|||||||
scheduler can wake up any cpu using the ``scx_bpf_kick_cpu()`` helper,
|
scheduler can wake up any cpu using the ``scx_bpf_kick_cpu()`` helper,
|
||||||
using ``ops.select_cpu()`` judiciously can be simpler and more efficient.
|
using ``ops.select_cpu()`` judiciously can be simpler and more efficient.
|
||||||
|
|
||||||
A task can be immediately dispatched to a DSQ from ``ops.select_cpu()`` by
|
A task can be immediately inserted into a DSQ from ``ops.select_cpu()``
|
||||||
calling ``scx_bpf_dispatch()``. If the task is dispatched to
|
by calling ``scx_bpf_dsq_insert()``. If the task is inserted into
|
||||||
``SCX_DSQ_LOCAL`` from ``ops.select_cpu()``, it will be dispatched to the
|
``SCX_DSQ_LOCAL`` from ``ops.select_cpu()``, it will be inserted into the
|
||||||
local DSQ of whichever CPU is returned from ``ops.select_cpu()``.
|
local DSQ of whichever CPU is returned from ``ops.select_cpu()``.
|
||||||
Additionally, dispatching directly from ``ops.select_cpu()`` will cause the
|
Additionally, inserting directly from ``ops.select_cpu()`` will cause the
|
||||||
``ops.enqueue()`` callback to be skipped.
|
``ops.enqueue()`` callback to be skipped.
|
||||||
|
|
||||||
Note that the scheduler core will ignore an invalid CPU selection, for
|
Note that the scheduler core will ignore an invalid CPU selection, for
|
||||||
example, if it's outside the allowed cpumask of the task.
|
example, if it's outside the allowed cpumask of the task.
|
||||||
|
|
||||||
2. Once the target CPU is selected, ``ops.enqueue()`` is invoked (unless the
|
2. Once the target CPU is selected, ``ops.enqueue()`` is invoked (unless the
|
||||||
task was dispatched directly from ``ops.select_cpu()``). ``ops.enqueue()``
|
task was inserted directly from ``ops.select_cpu()``). ``ops.enqueue()``
|
||||||
can make one of the following decisions:
|
can make one of the following decisions:
|
||||||
|
|
||||||
* Immediately dispatch the task to either the global or local DSQ by
|
* Immediately insert the task into either the global or local DSQ by
|
||||||
calling ``scx_bpf_dispatch()`` with ``SCX_DSQ_GLOBAL`` or
|
calling ``scx_bpf_dsq_insert()`` with ``SCX_DSQ_GLOBAL`` or
|
||||||
``SCX_DSQ_LOCAL``, respectively.
|
``SCX_DSQ_LOCAL``, respectively.
|
||||||
|
|
||||||
* Immediately dispatch the task to a custom DSQ by calling
|
* Immediately insert the task into a custom DSQ by calling
|
||||||
``scx_bpf_dispatch()`` with a DSQ ID which is smaller than 2^63.
|
``scx_bpf_dsq_insert()`` with a DSQ ID which is smaller than 2^63.
|
||||||
|
|
||||||
* Queue the task on the BPF side.
|
* Queue the task on the BPF side.
|
||||||
|
|
||||||
@ -257,11 +257,11 @@ The following briefly shows how a waking task is scheduled and executed.
|
|||||||
run, ``ops.dispatch()`` is invoked which can use the following two
|
run, ``ops.dispatch()`` is invoked which can use the following two
|
||||||
functions to populate the local DSQ.
|
functions to populate the local DSQ.
|
||||||
|
|
||||||
* ``scx_bpf_dispatch()`` dispatches a task to a DSQ. Any target DSQ can
|
* ``scx_bpf_dsq_insert()`` inserts a task to a DSQ. Any target DSQ can be
|
||||||
be used - ``SCX_DSQ_LOCAL``, ``SCX_DSQ_LOCAL_ON | cpu``,
|
used - ``SCX_DSQ_LOCAL``, ``SCX_DSQ_LOCAL_ON | cpu``,
|
||||||
``SCX_DSQ_GLOBAL`` or a custom DSQ. While ``scx_bpf_dispatch()``
|
``SCX_DSQ_GLOBAL`` or a custom DSQ. While ``scx_bpf_dsq_insert()``
|
||||||
currently can't be called with BPF locks held, this is being worked on
|
currently can't be called with BPF locks held, this is being worked on
|
||||||
and will be supported. ``scx_bpf_dispatch()`` schedules dispatching
|
and will be supported. ``scx_bpf_dsq_insert()`` schedules insertion
|
||||||
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.
|
||||||
|
|
||||||
@ -288,12 +288,12 @@ 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 consumed automatically.
|
||||||
|
|
||||||
``scx_bpf_dispatch()`` queues 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_dispatch_vtime()`` for the priority queue. Internal DSQs such as
|
``scx_bpf_dsq_insert_vtime()`` for the priority queue. Internal DSQs such as
|
||||||
``SCX_DSQ_LOCAL`` and ``SCX_DSQ_GLOBAL`` do not support priority-queue
|
``SCX_DSQ_LOCAL`` and ``SCX_DSQ_GLOBAL`` do not support priority-queue
|
||||||
dispatching, and must be dispatched to with ``scx_bpf_dispatch()``. See the
|
dispatching, and must be dispatched to with ``scx_bpf_dsq_insert()``. See
|
||||||
function documentation and usage in ``tools/sched_ext/scx_simple.bpf.c`` for
|
the function documentation and usage in ``tools/sched_ext/scx_simple.bpf.c``
|
||||||
more information.
|
for more information.
|
||||||
|
|
||||||
Where to Look
|
Where to Look
|
||||||
=============
|
=============
|
||||||
|
@ -220,10 +220,10 @@ struct sched_ext_ops {
|
|||||||
* dispatch. While an explicit custom mechanism can be added,
|
* dispatch. While an explicit custom mechanism can be added,
|
||||||
* select_cpu() serves as the default way to wake up idle CPUs.
|
* select_cpu() serves as the default way to wake up idle CPUs.
|
||||||
*
|
*
|
||||||
* @p may be dispatched directly by calling scx_bpf_dispatch(). If @p
|
* @p may be inserted into a DSQ directly by calling
|
||||||
* is dispatched, the ops.enqueue() callback will be skipped. Finally,
|
* scx_bpf_dsq_insert(). If so, the ops.enqueue() will be skipped.
|
||||||
* if @p is dispatched to SCX_DSQ_LOCAL, it will be dispatched to the
|
* Directly inserting into %SCX_DSQ_LOCAL will put @p in the local DSQ
|
||||||
* local DSQ of whatever CPU is returned by this callback.
|
* of the CPU returned by this operation.
|
||||||
*
|
*
|
||||||
* Note that select_cpu() is never called for tasks that can only run
|
* Note that select_cpu() is never called for tasks that can only run
|
||||||
* on a single CPU or tasks with migration disabled, as they don't have
|
* on a single CPU or tasks with migration disabled, as they don't have
|
||||||
@ -237,12 +237,12 @@ struct sched_ext_ops {
|
|||||||
* @p: task being enqueued
|
* @p: task being enqueued
|
||||||
* @enq_flags: %SCX_ENQ_*
|
* @enq_flags: %SCX_ENQ_*
|
||||||
*
|
*
|
||||||
* @p is ready to run. Dispatch directly by calling scx_bpf_dispatch()
|
* @p is ready to run. Insert directly into a DSQ by calling
|
||||||
* or enqueue on the BPF scheduler. If not directly dispatched, the bpf
|
* scx_bpf_dsq_insert() or enqueue on the BPF scheduler. If not directly
|
||||||
* scheduler owns @p and if it fails to dispatch @p, the task will
|
* inserted, the bpf scheduler owns @p and if it fails to dispatch @p,
|
||||||
* stall.
|
* the task will stall.
|
||||||
*
|
*
|
||||||
* If @p was dispatched from ops.select_cpu(), this callback is
|
* If @p was inserted into a DSQ from ops.select_cpu(), this callback is
|
||||||
* skipped.
|
* skipped.
|
||||||
*/
|
*/
|
||||||
void (*enqueue)(struct task_struct *p, u64 enq_flags);
|
void (*enqueue)(struct task_struct *p, u64 enq_flags);
|
||||||
@ -270,11 +270,11 @@ struct sched_ext_ops {
|
|||||||
*
|
*
|
||||||
* 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_dispatch() and/or consume user DSQs into the local DSQ using
|
* scx_bpf_dsq_insert() and/or consume user DSQs into the local DSQ
|
||||||
* scx_bpf_consume().
|
* using scx_bpf_consume().
|
||||||
*
|
*
|
||||||
* The maximum number of times scx_bpf_dispatch() can be called without
|
* The maximum number of times scx_bpf_dsq_insert() can be called
|
||||||
* an intervening scx_bpf_consume() is specified by
|
* without an intervening scx_bpf_consume() 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.
|
||||||
*
|
*
|
||||||
@ -714,7 +714,7 @@ enum scx_enq_flags {
|
|||||||
|
|
||||||
/*
|
/*
|
||||||
* Set the following to trigger preemption when calling
|
* Set the following to trigger preemption when calling
|
||||||
* scx_bpf_dispatch() with a local dsq as the target. The slice of the
|
* scx_bpf_dsq_insert() with a local dsq as the target. The slice of the
|
||||||
* current task is cleared to zero and the CPU is kicked into the
|
* current task is cleared to zero and the CPU is kicked into the
|
||||||
* scheduling path. Implies %SCX_ENQ_HEAD.
|
* scheduling path. Implies %SCX_ENQ_HEAD.
|
||||||
*/
|
*/
|
||||||
@ -2322,7 +2322,7 @@ static bool task_can_run_on_remote_rq(struct task_struct *p, struct rq *rq,
|
|||||||
/*
|
/*
|
||||||
* We don't require the BPF scheduler to avoid dispatching to offline
|
* We don't require the BPF scheduler to avoid dispatching to offline
|
||||||
* CPUs mostly for convenience but also because CPUs can go offline
|
* CPUs mostly for convenience but also because CPUs can go offline
|
||||||
* between scx_bpf_dispatch() calls and here. Trigger error iff the
|
* between scx_bpf_dsq_insert() calls and here. Trigger error iff the
|
||||||
* picked CPU is outside the allowed mask.
|
* picked CPU is outside the allowed mask.
|
||||||
*/
|
*/
|
||||||
if (!task_allowed_on_cpu(p, cpu)) {
|
if (!task_allowed_on_cpu(p, cpu)) {
|
||||||
@ -2658,7 +2658,7 @@ static void dispatch_to_local_dsq(struct rq *rq, struct scx_dispatch_q *dst_dsq,
|
|||||||
* Dispatching to local DSQs may need to wait for queueing to complete or
|
* Dispatching to local DSQs may need to wait for queueing to complete or
|
||||||
* require rq lock dancing. As we don't wanna do either while inside
|
* require rq lock dancing. As we don't wanna do either while inside
|
||||||
* ops.dispatch() to avoid locking order inversion, we split dispatching into
|
* ops.dispatch() to avoid locking order inversion, we split dispatching into
|
||||||
* two parts. scx_bpf_dispatch() which is called by ops.dispatch() records the
|
* two parts. scx_bpf_dsq_insert() which is called by ops.dispatch() records the
|
||||||
* task and its qseq. Once ops.dispatch() returns, this function is called to
|
* task and its qseq. Once ops.dispatch() returns, this function is called to
|
||||||
* finish up.
|
* finish up.
|
||||||
*
|
*
|
||||||
@ -2690,7 +2690,7 @@ retry:
|
|||||||
/*
|
/*
|
||||||
* If qseq doesn't match, @p has gone through at least one
|
* If qseq doesn't match, @p has gone through at least one
|
||||||
* dispatch/dequeue and re-enqueue cycle between
|
* dispatch/dequeue and re-enqueue cycle between
|
||||||
* scx_bpf_dispatch() and here and we have no claim on it.
|
* scx_bpf_dsq_insert() and here and we have no claim on it.
|
||||||
*/
|
*/
|
||||||
if ((opss & SCX_OPSS_QSEQ_MASK) != qseq_at_dispatch)
|
if ((opss & SCX_OPSS_QSEQ_MASK) != qseq_at_dispatch)
|
||||||
return;
|
return;
|
||||||
@ -6258,7 +6258,7 @@ static const struct btf_kfunc_id_set scx_kfunc_set_select_cpu = {
|
|||||||
.set = &scx_kfunc_ids_select_cpu,
|
.set = &scx_kfunc_ids_select_cpu,
|
||||||
};
|
};
|
||||||
|
|
||||||
static bool scx_dispatch_preamble(struct task_struct *p, u64 enq_flags)
|
static bool scx_dsq_insert_preamble(struct task_struct *p, u64 enq_flags)
|
||||||
{
|
{
|
||||||
if (!scx_kf_allowed(SCX_KF_ENQUEUE | SCX_KF_DISPATCH))
|
if (!scx_kf_allowed(SCX_KF_ENQUEUE | SCX_KF_DISPATCH))
|
||||||
return false;
|
return false;
|
||||||
@ -6278,7 +6278,8 @@ static bool scx_dispatch_preamble(struct task_struct *p, u64 enq_flags)
|
|||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
static void scx_dispatch_commit(struct task_struct *p, u64 dsq_id, u64 enq_flags)
|
static void scx_dsq_insert_commit(struct task_struct *p, u64 dsq_id,
|
||||||
|
u64 enq_flags)
|
||||||
{
|
{
|
||||||
struct scx_dsp_ctx *dspc = this_cpu_ptr(scx_dsp_ctx);
|
struct scx_dsp_ctx *dspc = this_cpu_ptr(scx_dsp_ctx);
|
||||||
struct task_struct *ddsp_task;
|
struct task_struct *ddsp_task;
|
||||||
@ -6305,14 +6306,14 @@ static void scx_dispatch_commit(struct task_struct *p, u64 dsq_id, u64 enq_flags
|
|||||||
__bpf_kfunc_start_defs();
|
__bpf_kfunc_start_defs();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* scx_bpf_dispatch - Dispatch a task into the FIFO queue of a DSQ
|
* scx_bpf_dsq_insert - Insert a task into the FIFO queue of a DSQ
|
||||||
* @p: task_struct to dispatch
|
* @p: task_struct to insert
|
||||||
* @dsq_id: DSQ to dispatch to
|
* @dsq_id: DSQ to insert into
|
||||||
* @slice: duration @p can run for in nsecs, 0 to keep the current value
|
* @slice: duration @p can run for in nsecs, 0 to keep the current value
|
||||||
* @enq_flags: SCX_ENQ_*
|
* @enq_flags: SCX_ENQ_*
|
||||||
*
|
*
|
||||||
* Dispatch @p into the FIFO queue of the DSQ identified by @dsq_id. It is safe
|
* Insert @p into the FIFO queue of the DSQ identified by @dsq_id. It is safe to
|
||||||
* to call this function spuriously. Can be called from ops.enqueue(),
|
* call this function spuriously. Can be called from ops.enqueue(),
|
||||||
* ops.select_cpu(), and ops.dispatch().
|
* ops.select_cpu(), and ops.dispatch().
|
||||||
*
|
*
|
||||||
* When called from ops.select_cpu() or ops.enqueue(), it's for direct dispatch
|
* When called from ops.select_cpu() or ops.enqueue(), it's for direct dispatch
|
||||||
@ -6321,14 +6322,14 @@ __bpf_kfunc_start_defs();
|
|||||||
* ops.select_cpu() to be on the target CPU in the first place.
|
* ops.select_cpu() to be on the target CPU in the first place.
|
||||||
*
|
*
|
||||||
* When called from ops.select_cpu(), @enq_flags and @dsp_id are stored, and @p
|
* When called from ops.select_cpu(), @enq_flags and @dsp_id are stored, and @p
|
||||||
* will be directly dispatched to the corresponding dispatch queue after
|
* will be directly inserted into the corresponding dispatch queue after
|
||||||
* ops.select_cpu() returns. If @p is dispatched to SCX_DSQ_LOCAL, it will be
|
* ops.select_cpu() returns. If @p is inserted into SCX_DSQ_LOCAL, it will be
|
||||||
* dispatched to the local DSQ of the CPU returned by ops.select_cpu().
|
* inserted into the local DSQ of the CPU returned by ops.select_cpu().
|
||||||
* @enq_flags are OR'd with the enqueue flags on the enqueue path before the
|
* @enq_flags are OR'd with the enqueue flags on the enqueue path before the
|
||||||
* task is dispatched.
|
* task is inserted.
|
||||||
*
|
*
|
||||||
* When called from ops.dispatch(), there are no restrictions on @p or @dsq_id
|
* When called from ops.dispatch(), there are no restrictions on @p or @dsq_id
|
||||||
* and this function can be called upto ops.dispatch_max_batch times to dispatch
|
* and this function can be called upto ops.dispatch_max_batch times to insert
|
||||||
* multiple tasks. scx_bpf_dispatch_nr_slots() returns the number of the
|
* multiple tasks. scx_bpf_dispatch_nr_slots() returns the number of the
|
||||||
* remaining slots. scx_bpf_consume() flushes the batch and resets the counter.
|
* remaining slots. scx_bpf_consume() flushes the batch and resets the counter.
|
||||||
*
|
*
|
||||||
@ -6340,10 +6341,10 @@ __bpf_kfunc_start_defs();
|
|||||||
* %SCX_SLICE_INF, @p never expires and the BPF scheduler must kick the CPU with
|
* %SCX_SLICE_INF, @p never expires and the BPF scheduler must kick the CPU with
|
||||||
* scx_bpf_kick_cpu() to trigger scheduling.
|
* scx_bpf_kick_cpu() to trigger scheduling.
|
||||||
*/
|
*/
|
||||||
__bpf_kfunc void scx_bpf_dispatch(struct task_struct *p, u64 dsq_id, u64 slice,
|
__bpf_kfunc void scx_bpf_dsq_insert(struct task_struct *p, u64 dsq_id, u64 slice,
|
||||||
u64 enq_flags)
|
u64 enq_flags)
|
||||||
{
|
{
|
||||||
if (!scx_dispatch_preamble(p, enq_flags))
|
if (!scx_dsq_insert_preamble(p, enq_flags))
|
||||||
return;
|
return;
|
||||||
|
|
||||||
if (slice)
|
if (slice)
|
||||||
@ -6351,30 +6352,38 @@ __bpf_kfunc void scx_bpf_dispatch(struct task_struct *p, u64 dsq_id, u64 slice,
|
|||||||
else
|
else
|
||||||
p->scx.slice = p->scx.slice ?: 1;
|
p->scx.slice = p->scx.slice ?: 1;
|
||||||
|
|
||||||
scx_dispatch_commit(p, dsq_id, enq_flags);
|
scx_dsq_insert_commit(p, dsq_id, enq_flags);
|
||||||
|
}
|
||||||
|
|
||||||
|
/* for backward compatibility, will be removed in v6.15 */
|
||||||
|
__bpf_kfunc void scx_bpf_dispatch(struct task_struct *p, u64 dsq_id, u64 slice,
|
||||||
|
u64 enq_flags)
|
||||||
|
{
|
||||||
|
printk_deferred_once(KERN_WARNING "sched_ext: scx_bpf_dispatch() renamed to scx_bpf_dsq_insert()");
|
||||||
|
scx_bpf_dsq_insert(p, dsq_id, slice, enq_flags);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* scx_bpf_dispatch_vtime - Dispatch a task into the vtime priority queue of a DSQ
|
* scx_bpf_dsq_insert_vtime - Insert a task into the vtime priority queue of a DSQ
|
||||||
* @p: task_struct to dispatch
|
* @p: task_struct to insert
|
||||||
* @dsq_id: DSQ to dispatch to
|
* @dsq_id: DSQ to insert into
|
||||||
* @slice: duration @p can run for in nsecs, 0 to keep the current value
|
* @slice: duration @p can run for in nsecs, 0 to keep the current value
|
||||||
* @vtime: @p's ordering inside the vtime-sorted queue of the target DSQ
|
* @vtime: @p's ordering inside the vtime-sorted queue of the target DSQ
|
||||||
* @enq_flags: SCX_ENQ_*
|
* @enq_flags: SCX_ENQ_*
|
||||||
*
|
*
|
||||||
* Dispatch @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 and always
|
||||||
* consumed after the tasks in the FIFO queue. All other aspects are identical
|
* consumed after the tasks in the FIFO queue. All other aspects are identical
|
||||||
* to scx_bpf_dispatch().
|
* 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.
|
||||||
*/
|
*/
|
||||||
__bpf_kfunc void scx_bpf_dispatch_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)
|
||||||
{
|
{
|
||||||
if (!scx_dispatch_preamble(p, enq_flags))
|
if (!scx_dsq_insert_preamble(p, enq_flags))
|
||||||
return;
|
return;
|
||||||
|
|
||||||
if (slice)
|
if (slice)
|
||||||
@ -6384,12 +6393,22 @@ __bpf_kfunc void scx_bpf_dispatch_vtime(struct task_struct *p, u64 dsq_id,
|
|||||||
|
|
||||||
p->scx.dsq_vtime = vtime;
|
p->scx.dsq_vtime = vtime;
|
||||||
|
|
||||||
scx_dispatch_commit(p, dsq_id, enq_flags | SCX_ENQ_DSQ_PRIQ);
|
scx_dsq_insert_commit(p, dsq_id, enq_flags | SCX_ENQ_DSQ_PRIQ);
|
||||||
|
}
|
||||||
|
|
||||||
|
/* for backward compatibility, will be removed in v6.15 */
|
||||||
|
__bpf_kfunc void scx_bpf_dispatch_vtime(struct task_struct *p, u64 dsq_id,
|
||||||
|
u64 slice, u64 vtime, u64 enq_flags)
|
||||||
|
{
|
||||||
|
printk_deferred_once(KERN_WARNING "sched_ext: scx_bpf_dispatch_vtime() renamed to scx_bpf_dsq_insert_vtime()");
|
||||||
|
scx_bpf_dsq_insert_vtime(p, dsq_id, slice, vtime, enq_flags);
|
||||||
}
|
}
|
||||||
|
|
||||||
__bpf_kfunc_end_defs();
|
__bpf_kfunc_end_defs();
|
||||||
|
|
||||||
BTF_KFUNCS_START(scx_kfunc_ids_enqueue_dispatch)
|
BTF_KFUNCS_START(scx_kfunc_ids_enqueue_dispatch)
|
||||||
|
BTF_ID_FLAGS(func, scx_bpf_dsq_insert, KF_RCU)
|
||||||
|
BTF_ID_FLAGS(func, scx_bpf_dsq_insert_vtime, KF_RCU)
|
||||||
BTF_ID_FLAGS(func, scx_bpf_dispatch, KF_RCU)
|
BTF_ID_FLAGS(func, scx_bpf_dispatch, KF_RCU)
|
||||||
BTF_ID_FLAGS(func, scx_bpf_dispatch_vtime, KF_RCU)
|
BTF_ID_FLAGS(func, scx_bpf_dispatch_vtime, KF_RCU)
|
||||||
BTF_KFUNCS_END(scx_kfunc_ids_enqueue_dispatch)
|
BTF_KFUNCS_END(scx_kfunc_ids_enqueue_dispatch)
|
||||||
@ -6527,9 +6546,9 @@ __bpf_kfunc void scx_bpf_dispatch_cancel(void)
|
|||||||
* to the current CPU's local DSQ for execution. Can only be called from
|
* to the current CPU's local DSQ for execution. Can only be called from
|
||||||
* ops.dispatch().
|
* ops.dispatch().
|
||||||
*
|
*
|
||||||
* This function flushes the in-flight dispatches from scx_bpf_dispatch() before
|
* This function flushes the in-flight dispatches from scx_bpf_dsq_insert()
|
||||||
* trying to consume the specified DSQ. It may also grab rq locks and thus can't
|
* before trying to consume the specified DSQ. It may also grab rq locks and
|
||||||
* 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 consumed, %false if there isn't any task to
|
||||||
* consume.
|
* consume.
|
||||||
@ -6650,7 +6669,7 @@ __bpf_kfunc bool scx_bpf_dispatch_from_dsq(struct bpf_iter_scx_dsq *it__iter,
|
|||||||
* scx_bpf_dispatch_from_dsq_set_vtime() to update.
|
* scx_bpf_dispatch_from_dsq_set_vtime() to update.
|
||||||
*
|
*
|
||||||
* All other aspects are identical to scx_bpf_dispatch_from_dsq(). See
|
* All other aspects are identical to scx_bpf_dispatch_from_dsq(). See
|
||||||
* scx_bpf_dispatch_vtime() for more information on @vtime.
|
* scx_bpf_dsq_insert_vtime() for more information on @vtime.
|
||||||
*/
|
*/
|
||||||
__bpf_kfunc bool scx_bpf_dispatch_vtime_from_dsq(struct bpf_iter_scx_dsq *it__iter,
|
__bpf_kfunc bool scx_bpf_dispatch_vtime_from_dsq(struct bpf_iter_scx_dsq *it__iter,
|
||||||
struct task_struct *p, u64 dsq_id,
|
struct task_struct *p, u64 dsq_id,
|
||||||
|
@ -36,8 +36,8 @@ static inline void ___vmlinux_h_sanity_check___(void)
|
|||||||
|
|
||||||
s32 scx_bpf_create_dsq(u64 dsq_id, s32 node) __ksym;
|
s32 scx_bpf_create_dsq(u64 dsq_id, s32 node) __ksym;
|
||||||
s32 scx_bpf_select_cpu_dfl(struct task_struct *p, s32 prev_cpu, u64 wake_flags, bool *is_idle) __ksym;
|
s32 scx_bpf_select_cpu_dfl(struct task_struct *p, s32 prev_cpu, u64 wake_flags, bool *is_idle) __ksym;
|
||||||
void scx_bpf_dispatch(struct task_struct *p, u64 dsq_id, u64 slice, u64 enq_flags) __ksym;
|
void scx_bpf_dsq_insert(struct task_struct *p, u64 dsq_id, u64 slice, u64 enq_flags) __ksym __weak;
|
||||||
void scx_bpf_dispatch_vtime(struct task_struct *p, u64 dsq_id, u64 slice, u64 vtime, u64 enq_flags) __ksym;
|
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_consume(u64 dsq_id) __ksym;
|
||||||
|
@ -34,6 +34,32 @@
|
|||||||
(bpf_ksym_exists(scx_bpf_dispatch_vtime_from_dsq) ? \
|
(bpf_ksym_exists(scx_bpf_dispatch_vtime_from_dsq) ? \
|
||||||
scx_bpf_dispatch_vtime_from_dsq((it), (p), (dsq_id), (enq_flags)) : false)
|
scx_bpf_dispatch_vtime_from_dsq((it), (p), (dsq_id), (enq_flags)) : false)
|
||||||
|
|
||||||
|
/*
|
||||||
|
* v6.13: The verb `dispatch` was too overloaded and confusing. kfuncs are
|
||||||
|
* renamed to unload the verb.
|
||||||
|
*
|
||||||
|
* Build error is triggered if old names are used. New binaries work with both
|
||||||
|
* new and old names. The compat macros will be removed on v6.15 release.
|
||||||
|
*/
|
||||||
|
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;
|
||||||
|
|
||||||
|
#define scx_bpf_dsq_insert(p, dsq_id, slice, enq_flags) \
|
||||||
|
(bpf_ksym_exists(scx_bpf_dsq_insert) ? \
|
||||||
|
scx_bpf_dsq_insert((p), (dsq_id), (slice), (enq_flags)) : \
|
||||||
|
scx_bpf_dispatch___compat((p), (dsq_id), (slice), (enq_flags)))
|
||||||
|
|
||||||
|
#define scx_bpf_dsq_insert_vtime(p, dsq_id, slice, vtime, enq_flags) \
|
||||||
|
(bpf_ksym_exists(scx_bpf_dsq_insert_vtime) ? \
|
||||||
|
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_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 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().
|
||||||
|
@ -118,14 +118,14 @@ void BPF_STRUCT_OPS(central_enqueue, struct task_struct *p, u64 enq_flags)
|
|||||||
*/
|
*/
|
||||||
if ((p->flags & PF_KTHREAD) && p->nr_cpus_allowed == 1) {
|
if ((p->flags & PF_KTHREAD) && p->nr_cpus_allowed == 1) {
|
||||||
__sync_fetch_and_add(&nr_locals, 1);
|
__sync_fetch_and_add(&nr_locals, 1);
|
||||||
scx_bpf_dispatch(p, SCX_DSQ_LOCAL, SCX_SLICE_INF,
|
scx_bpf_dsq_insert(p, SCX_DSQ_LOCAL, SCX_SLICE_INF,
|
||||||
enq_flags | SCX_ENQ_PREEMPT);
|
enq_flags | SCX_ENQ_PREEMPT);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (bpf_map_push_elem(¢ral_q, &pid, 0)) {
|
if (bpf_map_push_elem(¢ral_q, &pid, 0)) {
|
||||||
__sync_fetch_and_add(&nr_overflows, 1);
|
__sync_fetch_and_add(&nr_overflows, 1);
|
||||||
scx_bpf_dispatch(p, FALLBACK_DSQ_ID, SCX_SLICE_INF, enq_flags);
|
scx_bpf_dsq_insert(p, FALLBACK_DSQ_ID, SCX_SLICE_INF, enq_flags);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -158,7 +158,7 @@ static bool dispatch_to_cpu(s32 cpu)
|
|||||||
*/
|
*/
|
||||||
if (!bpf_cpumask_test_cpu(cpu, p->cpus_ptr)) {
|
if (!bpf_cpumask_test_cpu(cpu, p->cpus_ptr)) {
|
||||||
__sync_fetch_and_add(&nr_mismatches, 1);
|
__sync_fetch_and_add(&nr_mismatches, 1);
|
||||||
scx_bpf_dispatch(p, FALLBACK_DSQ_ID, SCX_SLICE_INF, 0);
|
scx_bpf_dsq_insert(p, FALLBACK_DSQ_ID, SCX_SLICE_INF, 0);
|
||||||
bpf_task_release(p);
|
bpf_task_release(p);
|
||||||
/*
|
/*
|
||||||
* We might run out of dispatch buffer slots if we continue dispatching
|
* We might run out of dispatch buffer slots if we continue dispatching
|
||||||
@ -172,7 +172,7 @@ static bool dispatch_to_cpu(s32 cpu)
|
|||||||
}
|
}
|
||||||
|
|
||||||
/* dispatch to local and mark that @cpu doesn't need more */
|
/* dispatch to local and mark that @cpu doesn't need more */
|
||||||
scx_bpf_dispatch(p, SCX_DSQ_LOCAL_ON | cpu, SCX_SLICE_INF, 0);
|
scx_bpf_dsq_insert(p, SCX_DSQ_LOCAL_ON | cpu, SCX_SLICE_INF, 0);
|
||||||
|
|
||||||
if (cpu != central_cpu)
|
if (cpu != central_cpu)
|
||||||
scx_bpf_kick_cpu(cpu, SCX_KICK_IDLE);
|
scx_bpf_kick_cpu(cpu, SCX_KICK_IDLE);
|
||||||
|
@ -341,7 +341,7 @@ s32 BPF_STRUCT_OPS(fcg_select_cpu, struct task_struct *p, s32 prev_cpu, u64 wake
|
|||||||
if (is_idle) {
|
if (is_idle) {
|
||||||
set_bypassed_at(p, taskc);
|
set_bypassed_at(p, taskc);
|
||||||
stat_inc(FCG_STAT_LOCAL);
|
stat_inc(FCG_STAT_LOCAL);
|
||||||
scx_bpf_dispatch(p, SCX_DSQ_LOCAL, SCX_SLICE_DFL, 0);
|
scx_bpf_dsq_insert(p, SCX_DSQ_LOCAL, SCX_SLICE_DFL, 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
return cpu;
|
return cpu;
|
||||||
@ -377,10 +377,12 @@ void BPF_STRUCT_OPS(fcg_enqueue, struct task_struct *p, u64 enq_flags)
|
|||||||
*/
|
*/
|
||||||
if (p->nr_cpus_allowed == 1 && (p->flags & PF_KTHREAD)) {
|
if (p->nr_cpus_allowed == 1 && (p->flags & PF_KTHREAD)) {
|
||||||
stat_inc(FCG_STAT_LOCAL);
|
stat_inc(FCG_STAT_LOCAL);
|
||||||
scx_bpf_dispatch(p, SCX_DSQ_LOCAL, SCX_SLICE_DFL, enq_flags);
|
scx_bpf_dsq_insert(p, SCX_DSQ_LOCAL, SCX_SLICE_DFL,
|
||||||
|
enq_flags);
|
||||||
} else {
|
} else {
|
||||||
stat_inc(FCG_STAT_GLOBAL);
|
stat_inc(FCG_STAT_GLOBAL);
|
||||||
scx_bpf_dispatch(p, FALLBACK_DSQ, SCX_SLICE_DFL, enq_flags);
|
scx_bpf_dsq_insert(p, FALLBACK_DSQ, SCX_SLICE_DFL,
|
||||||
|
enq_flags);
|
||||||
}
|
}
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
@ -391,7 +393,7 @@ void BPF_STRUCT_OPS(fcg_enqueue, struct task_struct *p, u64 enq_flags)
|
|||||||
goto out_release;
|
goto out_release;
|
||||||
|
|
||||||
if (fifo_sched) {
|
if (fifo_sched) {
|
||||||
scx_bpf_dispatch(p, cgrp->kn->id, SCX_SLICE_DFL, enq_flags);
|
scx_bpf_dsq_insert(p, cgrp->kn->id, SCX_SLICE_DFL, enq_flags);
|
||||||
} else {
|
} else {
|
||||||
u64 tvtime = p->scx.dsq_vtime;
|
u64 tvtime = p->scx.dsq_vtime;
|
||||||
|
|
||||||
@ -402,7 +404,7 @@ void BPF_STRUCT_OPS(fcg_enqueue, struct task_struct *p, u64 enq_flags)
|
|||||||
if (vtime_before(tvtime, cgc->tvtime_now - SCX_SLICE_DFL))
|
if (vtime_before(tvtime, cgc->tvtime_now - SCX_SLICE_DFL))
|
||||||
tvtime = cgc->tvtime_now - SCX_SLICE_DFL;
|
tvtime = cgc->tvtime_now - SCX_SLICE_DFL;
|
||||||
|
|
||||||
scx_bpf_dispatch_vtime(p, cgrp->kn->id, SCX_SLICE_DFL,
|
scx_bpf_dsq_insert_vtime(p, cgrp->kn->id, SCX_SLICE_DFL,
|
||||||
tvtime, enq_flags);
|
tvtime, enq_flags);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -226,7 +226,7 @@ void BPF_STRUCT_OPS(qmap_enqueue, struct task_struct *p, u64 enq_flags)
|
|||||||
*/
|
*/
|
||||||
if (tctx->force_local) {
|
if (tctx->force_local) {
|
||||||
tctx->force_local = false;
|
tctx->force_local = false;
|
||||||
scx_bpf_dispatch(p, SCX_DSQ_LOCAL, slice_ns, enq_flags);
|
scx_bpf_dsq_insert(p, SCX_DSQ_LOCAL, slice_ns, enq_flags);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -234,7 +234,7 @@ void BPF_STRUCT_OPS(qmap_enqueue, struct task_struct *p, u64 enq_flags)
|
|||||||
if (!(enq_flags & SCX_ENQ_CPU_SELECTED) &&
|
if (!(enq_flags & SCX_ENQ_CPU_SELECTED) &&
|
||||||
(cpu = pick_direct_dispatch_cpu(p, scx_bpf_task_cpu(p))) >= 0) {
|
(cpu = pick_direct_dispatch_cpu(p, scx_bpf_task_cpu(p))) >= 0) {
|
||||||
__sync_fetch_and_add(&nr_ddsp_from_enq, 1);
|
__sync_fetch_and_add(&nr_ddsp_from_enq, 1);
|
||||||
scx_bpf_dispatch(p, SCX_DSQ_LOCAL_ON | cpu, slice_ns, enq_flags);
|
scx_bpf_dsq_insert(p, SCX_DSQ_LOCAL_ON | cpu, slice_ns, enq_flags);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -247,7 +247,7 @@ void BPF_STRUCT_OPS(qmap_enqueue, struct task_struct *p, u64 enq_flags)
|
|||||||
if (enq_flags & SCX_ENQ_REENQ) {
|
if (enq_flags & SCX_ENQ_REENQ) {
|
||||||
s32 cpu;
|
s32 cpu;
|
||||||
|
|
||||||
scx_bpf_dispatch(p, SHARED_DSQ, 0, enq_flags);
|
scx_bpf_dsq_insert(p, SHARED_DSQ, 0, enq_flags);
|
||||||
cpu = scx_bpf_pick_idle_cpu(p->cpus_ptr, 0);
|
cpu = scx_bpf_pick_idle_cpu(p->cpus_ptr, 0);
|
||||||
if (cpu >= 0)
|
if (cpu >= 0)
|
||||||
scx_bpf_kick_cpu(cpu, SCX_KICK_IDLE);
|
scx_bpf_kick_cpu(cpu, SCX_KICK_IDLE);
|
||||||
@ -262,7 +262,7 @@ void BPF_STRUCT_OPS(qmap_enqueue, struct task_struct *p, u64 enq_flags)
|
|||||||
|
|
||||||
/* Queue on the selected FIFO. If the FIFO overflows, punt to global. */
|
/* Queue on the selected FIFO. If the FIFO overflows, punt to global. */
|
||||||
if (bpf_map_push_elem(ring, &pid, 0)) {
|
if (bpf_map_push_elem(ring, &pid, 0)) {
|
||||||
scx_bpf_dispatch(p, SHARED_DSQ, slice_ns, enq_flags);
|
scx_bpf_dsq_insert(p, SHARED_DSQ, slice_ns, enq_flags);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -385,7 +385,7 @@ void BPF_STRUCT_OPS(qmap_dispatch, s32 cpu, struct task_struct *prev)
|
|||||||
*/
|
*/
|
||||||
p = bpf_task_from_pid(2);
|
p = bpf_task_from_pid(2);
|
||||||
if (p) {
|
if (p) {
|
||||||
scx_bpf_dispatch(p, SCX_DSQ_LOCAL, slice_ns, 0);
|
scx_bpf_dsq_insert(p, SCX_DSQ_LOCAL, slice_ns, 0);
|
||||||
bpf_task_release(p);
|
bpf_task_release(p);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
@ -431,7 +431,7 @@ void BPF_STRUCT_OPS(qmap_dispatch, s32 cpu, struct task_struct *prev)
|
|||||||
update_core_sched_head_seq(p);
|
update_core_sched_head_seq(p);
|
||||||
__sync_fetch_and_add(&nr_dispatched, 1);
|
__sync_fetch_and_add(&nr_dispatched, 1);
|
||||||
|
|
||||||
scx_bpf_dispatch(p, SHARED_DSQ, slice_ns, 0);
|
scx_bpf_dsq_insert(p, SHARED_DSQ, slice_ns, 0);
|
||||||
bpf_task_release(p);
|
bpf_task_release(p);
|
||||||
|
|
||||||
batch--;
|
batch--;
|
||||||
|
@ -31,10 +31,10 @@ UEI_DEFINE(uei);
|
|||||||
|
|
||||||
/*
|
/*
|
||||||
* Built-in DSQs such as SCX_DSQ_GLOBAL cannot be used as priority queues
|
* Built-in DSQs such as SCX_DSQ_GLOBAL cannot be used as priority queues
|
||||||
* (meaning, cannot be dispatched to with scx_bpf_dispatch_vtime()). We
|
* (meaning, cannot be dispatched to with scx_bpf_dsq_insert_vtime()). We
|
||||||
* therefore create a separate DSQ with ID 0 that we dispatch to and consume
|
* therefore create a separate DSQ with ID 0 that we dispatch to and consume
|
||||||
* from. If scx_simple only supported global FIFO scheduling, then we could
|
* from. If scx_simple only supported global FIFO scheduling, then we could just
|
||||||
* just use SCX_DSQ_GLOBAL.
|
* use SCX_DSQ_GLOBAL.
|
||||||
*/
|
*/
|
||||||
#define SHARED_DSQ 0
|
#define SHARED_DSQ 0
|
||||||
|
|
||||||
@ -65,7 +65,7 @@ s32 BPF_STRUCT_OPS(simple_select_cpu, struct task_struct *p, s32 prev_cpu, u64 w
|
|||||||
cpu = scx_bpf_select_cpu_dfl(p, prev_cpu, wake_flags, &is_idle);
|
cpu = scx_bpf_select_cpu_dfl(p, prev_cpu, wake_flags, &is_idle);
|
||||||
if (is_idle) {
|
if (is_idle) {
|
||||||
stat_inc(0); /* count local queueing */
|
stat_inc(0); /* count local queueing */
|
||||||
scx_bpf_dispatch(p, SCX_DSQ_LOCAL, SCX_SLICE_DFL, 0);
|
scx_bpf_dsq_insert(p, SCX_DSQ_LOCAL, SCX_SLICE_DFL, 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
return cpu;
|
return cpu;
|
||||||
@ -76,7 +76,7 @@ void BPF_STRUCT_OPS(simple_enqueue, struct task_struct *p, u64 enq_flags)
|
|||||||
stat_inc(1); /* count global queueing */
|
stat_inc(1); /* count global queueing */
|
||||||
|
|
||||||
if (fifo_sched) {
|
if (fifo_sched) {
|
||||||
scx_bpf_dispatch(p, SHARED_DSQ, SCX_SLICE_DFL, enq_flags);
|
scx_bpf_dsq_insert(p, SHARED_DSQ, SCX_SLICE_DFL, enq_flags);
|
||||||
} else {
|
} else {
|
||||||
u64 vtime = p->scx.dsq_vtime;
|
u64 vtime = p->scx.dsq_vtime;
|
||||||
|
|
||||||
@ -87,7 +87,7 @@ void BPF_STRUCT_OPS(simple_enqueue, struct task_struct *p, u64 enq_flags)
|
|||||||
if (vtime_before(vtime, vtime_now - SCX_SLICE_DFL))
|
if (vtime_before(vtime, vtime_now - SCX_SLICE_DFL))
|
||||||
vtime = vtime_now - SCX_SLICE_DFL;
|
vtime = vtime_now - SCX_SLICE_DFL;
|
||||||
|
|
||||||
scx_bpf_dispatch_vtime(p, SHARED_DSQ, SCX_SLICE_DFL, vtime,
|
scx_bpf_dsq_insert_vtime(p, SHARED_DSQ, SCX_SLICE_DFL, vtime,
|
||||||
enq_flags);
|
enq_flags);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user