@@ -49,12 +49,15 @@ enum scx_dsq_id_flags {
};
/*
- * Dispatch queue (dsq) is a simple FIFO which is used to buffer between the
- * scheduler core and the BPF scheduler. See the documentation for more details.
+ * A dispatch queue (DSQ) can be either a FIFO or p->scx.dsq_vtime ordered
+ * queue. A built-in DSQ is always a FIFO. The built-in local DSQs are used to
+ * buffer between the scheduler core and the BPF scheduler. See the
+ * documentation for more details.
*/
struct scx_dispatch_q {
raw_spinlock_t lock;
struct list_head list; /* tasks in dispatch order */
+ struct rb_root priq; /* used to order by p->scx.dsq_vtime */
u32 nr;
u64 id;
struct rhash_head hash_node;
@@ -86,6 +89,11 @@ enum scx_task_state {
SCX_TASK_NR_STATES,
};
+/* scx_entity.dsq_flags */
+enum scx_ent_dsq_flags {
+ SCX_TASK_DSQ_ON_PRIQ = 1 << 0, /* task is queued on the priority queue of a dsq */
+};
+
/*
* Mask bits for scx_entity.kf_mask. Not all kfuncs can be called from
* everywhere and the following bits track which kfunc sets are currently
@@ -111,13 +119,19 @@ enum scx_kf_mask {
__SCX_KF_TERMINAL = SCX_KF_ENQUEUE | SCX_KF_SELECT_CPU | SCX_KF_REST,
};
+struct scx_dsq_node {
+ struct list_head list; /* dispatch order */
+ struct rb_node priq; /* p->scx.dsq_vtime order */
+ u32 flags; /* SCX_TASK_DSQ_* flags */
+};
+
/*
* The following is embedded in task_struct and contains all fields necessary
* for a task to be scheduled by SCX.
*/
struct sched_ext_entity {
struct scx_dispatch_q *dsq;
- struct list_head dsq_node;
+ struct scx_dsq_node dsq_node; /* protected by dsq lock */
u32 flags; /* protected by rq lock */
u32 weight;
s32 sticky_cpu;
@@ -149,6 +163,15 @@ struct sched_ext_entity {
*/
u64 slice;
+ /*
+ * Used to order tasks when dispatching to the vtime-ordered priority
+ * queue of a dsq. This is usually set through scx_bpf_dispatch_vtime()
+ * but can also be modified directly by the BPF scheduler. Modifying it
+ * while a task is queued on a dsq may mangle the ordering and is not
+ * recommended.
+ */
+ u64 dsq_vtime;
+
/*
* If set, reject future sched_setscheduler(2) calls updating the policy
* to %SCHED_EXT with -%EACCES.
@@ -101,7 +101,7 @@ struct task_struct init_task __aligned(L1_CACHE_BYTES) = {
#endif
#ifdef CONFIG_SCHED_CLASS_EXT
.scx = {
- .dsq_node = LIST_HEAD_INIT(init_task.scx.dsq_node),
+ .dsq_node.list = LIST_HEAD_INIT(init_task.scx.dsq_node.list),
.sticky_cpu = -1,
.holding_cpu = -1,
.runnable_node = LIST_HEAD_INIT(init_task.scx.runnable_node),
@@ -685,6 +685,7 @@ enum scx_enq_flags {
__SCX_ENQ_INTERNAL_MASK = 0xffLLU << 56,
SCX_ENQ_CLEAR_OPSS = 1LLU << 56,
+ SCX_ENQ_DSQ_PRIQ = 1LLU << 57,
};
enum scx_deq_flags {
@@ -1369,6 +1370,17 @@ static void update_curr_scx(struct rq *rq)
}
}
+static bool scx_dsq_priq_less(struct rb_node *node_a,
+ const struct rb_node *node_b)
+{
+ const struct task_struct *a =
+ container_of(node_a, struct task_struct, scx.dsq_node.priq);
+ const struct task_struct *b =
+ container_of(node_b, struct task_struct, scx.dsq_node.priq);
+
+ return time_before64(a->scx.dsq_vtime, b->scx.dsq_vtime);
+}
+
static void dsq_mod_nr(struct scx_dispatch_q *dsq, s32 delta)
{
/* scx_bpf_dsq_nr_queued() reads ->nr without locking, use WRITE_ONCE() */
@@ -1380,7 +1392,9 @@ static void dispatch_enqueue(struct scx_dispatch_q *dsq, struct task_struct *p,
{
bool is_local = dsq->id == SCX_DSQ_LOCAL;
- WARN_ON_ONCE(p->scx.dsq || !list_empty(&p->scx.dsq_node));
+ WARN_ON_ONCE(p->scx.dsq || !list_empty(&p->scx.dsq_node.list));
+ WARN_ON_ONCE((p->scx.dsq_node.flags & SCX_TASK_DSQ_ON_PRIQ) ||
+ !RB_EMPTY_NODE(&p->scx.dsq_node.priq));
if (!is_local) {
raw_spin_lock(&dsq->lock);
@@ -1393,10 +1407,59 @@ static void dispatch_enqueue(struct scx_dispatch_q *dsq, struct task_struct *p,
}
}
- if (enq_flags & (SCX_ENQ_HEAD | SCX_ENQ_PREEMPT))
- list_add(&p->scx.dsq_node, &dsq->list);
- else
- list_add_tail(&p->scx.dsq_node, &dsq->list);
+ if (unlikely((dsq->id & SCX_DSQ_FLAG_BUILTIN) &&
+ (enq_flags & SCX_ENQ_DSQ_PRIQ))) {
+ /*
+ * SCX_DSQ_LOCAL and SCX_DSQ_GLOBAL DSQs always consume from
+ * their FIFO queues. To avoid confusion and accidentally
+ * starving vtime-dispatched tasks by FIFO-dispatched tasks, we
+ * disallow any internal DSQ from doing vtime ordering of
+ * tasks.
+ */
+ scx_ops_error("cannot use vtime ordering for built-in DSQs");
+ enq_flags &= ~SCX_ENQ_DSQ_PRIQ;
+ }
+
+ if (enq_flags & SCX_ENQ_DSQ_PRIQ) {
+ struct rb_node *rbp;
+
+ /*
+ * A PRIQ DSQ shouldn't be using FIFO enqueueing. As tasks are
+ * linked to both the rbtree and list on PRIQs, this can only be
+ * tested easily when adding the first task.
+ */
+ if (unlikely(RB_EMPTY_ROOT(&dsq->priq) &&
+ !list_empty(&dsq->list)))
+ scx_ops_error("DSQ ID 0x%016llx already had FIFO-enqueued tasks",
+ dsq->id);
+
+ p->scx.dsq_node.flags |= SCX_TASK_DSQ_ON_PRIQ;
+ rb_add(&p->scx.dsq_node.priq, &dsq->priq, scx_dsq_priq_less);
+
+ /*
+ * Find the previous task and insert after it on the list so
+ * that @dsq->list is vtime ordered.
+ */
+ rbp = rb_prev(&p->scx.dsq_node.priq);
+ if (rbp) {
+ struct task_struct *prev =
+ container_of(rbp, struct task_struct,
+ scx.dsq_node.priq);
+ list_add(&p->scx.dsq_node.list, &prev->scx.dsq_node.list);
+ } else {
+ list_add(&p->scx.dsq_node.list, &dsq->list);
+ }
+ } else {
+ /* a FIFO DSQ shouldn't be using PRIQ enqueuing */
+ if (unlikely(!RB_EMPTY_ROOT(&dsq->priq)))
+ scx_ops_error("DSQ ID 0x%016llx already had PRIQ-enqueued tasks",
+ dsq->id);
+
+ if (enq_flags & (SCX_ENQ_HEAD | SCX_ENQ_PREEMPT))
+ list_add(&p->scx.dsq_node.list, &dsq->list);
+ else
+ list_add_tail(&p->scx.dsq_node.list, &dsq->list);
+ }
dsq_mod_nr(dsq, 1);
p->scx.dsq = dsq;
@@ -1435,13 +1498,30 @@ static void dispatch_enqueue(struct scx_dispatch_q *dsq, struct task_struct *p,
}
}
+static void task_unlink_from_dsq(struct task_struct *p,
+ struct scx_dispatch_q *dsq)
+{
+ if (p->scx.dsq_node.flags & SCX_TASK_DSQ_ON_PRIQ) {
+ rb_erase(&p->scx.dsq_node.priq, &dsq->priq);
+ RB_CLEAR_NODE(&p->scx.dsq_node.priq);
+ p->scx.dsq_node.flags &= ~SCX_TASK_DSQ_ON_PRIQ;
+ }
+
+ list_del_init(&p->scx.dsq_node.list);
+}
+
+static bool task_linked_on_dsq(struct task_struct *p)
+{
+ return !list_empty(&p->scx.dsq_node.list);
+}
+
static void dispatch_dequeue(struct scx_rq *scx_rq, struct task_struct *p)
{
struct scx_dispatch_q *dsq = p->scx.dsq;
bool is_local = dsq == &scx_rq->local_dsq;
if (!dsq) {
- WARN_ON_ONCE(!list_empty(&p->scx.dsq_node));
+ WARN_ON_ONCE(task_linked_on_dsq(p));
/*
* When dispatching directly from the BPF scheduler to a local
* DSQ, the task isn't associated with any DSQ but
@@ -1462,8 +1542,8 @@ static void dispatch_dequeue(struct scx_rq *scx_rq, struct task_struct *p)
*/
if (p->scx.holding_cpu < 0) {
/* @p must still be on @dsq, dequeue */
- WARN_ON_ONCE(list_empty(&p->scx.dsq_node));
- list_del_init(&p->scx.dsq_node);
+ WARN_ON_ONCE(!task_linked_on_dsq(p));
+ task_unlink_from_dsq(p, dsq);
dsq_mod_nr(dsq, -1);
} else {
/*
@@ -1472,7 +1552,7 @@ static void dispatch_dequeue(struct scx_rq *scx_rq, struct task_struct *p)
* holding_cpu which tells dispatch_to_local_dsq() that it lost
* the race.
*/
- WARN_ON_ONCE(!list_empty(&p->scx.dsq_node));
+ WARN_ON_ONCE(task_linked_on_dsq(p));
p->scx.holding_cpu = -1;
}
p->scx.dsq = NULL;
@@ -1975,7 +2055,8 @@ static void consume_local_task(struct rq *rq, struct scx_dispatch_q *dsq,
/* @dsq is locked and @p is on this rq */
WARN_ON_ONCE(p->scx.holding_cpu >= 0);
- list_move_tail(&p->scx.dsq_node, &scx_rq->local_dsq.list);
+ task_unlink_from_dsq(p, dsq);
+ list_add_tail(&p->scx.dsq_node.list, &scx_rq->local_dsq.list);
dsq_mod_nr(dsq, -1);
dsq_mod_nr(&scx_rq->local_dsq, 1);
p->scx.dsq = &scx_rq->local_dsq;
@@ -2018,7 +2099,7 @@ static bool consume_remote_task(struct rq *rq, struct rq_flags *rf,
* move_task_to_local_dsq().
*/
WARN_ON_ONCE(p->scx.holding_cpu >= 0);
- list_del_init(&p->scx.dsq_node);
+ task_unlink_from_dsq(p, dsq);
dsq_mod_nr(dsq, -1);
p->scx.holding_cpu = raw_smp_processor_id();
raw_spin_unlock(&dsq->lock);
@@ -2050,7 +2131,7 @@ static bool consume_dispatch_q(struct rq *rq, struct rq_flags *rf,
raw_spin_lock(&dsq->lock);
- list_for_each_entry(p, &dsq->list, scx.dsq_node) {
+ list_for_each_entry(p, &dsq->list, scx.dsq_node.list) {
struct rq *task_rq = task_rq(p);
if (rq == task_rq) {
@@ -2570,7 +2651,7 @@ static void put_prev_task_scx(struct rq *rq, struct task_struct *p)
static struct task_struct *first_local_task(struct rq *rq)
{
return list_first_entry_or_null(&rq->scx.local_dsq.list,
- struct task_struct, scx.dsq_node);
+ struct task_struct, scx.dsq_node.list);
}
static struct task_struct *pick_next_task_scx(struct rq *rq)
@@ -3267,7 +3348,8 @@ void init_scx_entity(struct sched_ext_entity *scx)
*/
memset(scx, 0, offsetof(struct sched_ext_entity, tasks_node));
- INIT_LIST_HEAD(&scx->dsq_node);
+ INIT_LIST_HEAD(&scx->dsq_node.list);
+ RB_CLEAR_NODE(&scx->dsq_node.priq);
scx->sticky_cpu = -1;
scx->holding_cpu = -1;
INIT_LIST_HEAD(&scx->runnable_node);
@@ -4294,9 +4376,10 @@ static void scx_dump_task(struct seq_buf *s, struct task_struct *p, char marker,
seq_buf_printf(s, "\n %c%c %s[%d] %+ldms\n",
marker, task_state_to_char(p), p->comm, p->pid,
jiffies_delta_msecs(p->scx.runnable_at, now));
- seq_buf_printf(s, " scx_state/flags=%u/0x%x ops_state/qseq=%lu/%lu\n",
+ seq_buf_printf(s, " scx_state/flags=%u/0x%x dsq_flags=0x%x ops_state/qseq=%lu/%lu\n",
scx_get_task_state(p),
p->scx.flags & ~SCX_TASK_STATE_MASK,
+ p->scx.dsq_node.flags,
ops_state & SCX_OPSS_STATE_MASK,
ops_state >> SCX_OPSS_QSEQ_SHIFT);
seq_buf_printf(s, " sticky/holding_cpu=%d/%d dsq_id=%s\n",
@@ -4844,6 +4927,9 @@ static int bpf_scx_btf_struct_access(struct bpf_verifier_log *log,
if (off >= offsetof(struct task_struct, scx.slice) &&
off + size <= offsetofend(struct task_struct, scx.slice))
return SCALAR_VALUE;
+ if (off >= offsetof(struct task_struct, scx.dsq_vtime) &&
+ off + size <= offsetofend(struct task_struct, scx.dsq_vtime))
+ return SCALAR_VALUE;
if (off >= offsetof(struct task_struct, scx.disallow) &&
off + size <= offsetofend(struct task_struct, scx.disallow))
return SCALAR_VALUE;
@@ -5483,10 +5569,44 @@ __bpf_kfunc void scx_bpf_dispatch(struct task_struct *p, u64 dsq_id, u64 slice,
scx_dispatch_commit(p, dsq_id, enq_flags);
}
+/**
+ * scx_bpf_dispatch_vtime - Dispatch a task into the vtime priority queue of a DSQ
+ * @p: task_struct to dispatch
+ * @dsq_id: DSQ to dispatch to
+ * @slice: duration @p can run for in nsecs
+ * @vtime: @p's ordering inside the vtime-sorted queue of the target DSQ
+ * @enq_flags: SCX_ENQ_*
+ *
+ * Dispatch @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_dispatch().
+ *
+ * @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.
+ */
+__bpf_kfunc void scx_bpf_dispatch_vtime(struct task_struct *p, u64 dsq_id,
+ u64 slice, u64 vtime, u64 enq_flags)
+{
+ if (!scx_dispatch_preamble(p, enq_flags))
+ return;
+
+ if (slice)
+ p->scx.slice = slice;
+ else
+ p->scx.slice = p->scx.slice ?: 1;
+
+ p->scx.dsq_vtime = vtime;
+
+ scx_dispatch_commit(p, dsq_id, enq_flags | SCX_ENQ_DSQ_PRIQ);
+}
+
__bpf_kfunc_end_defs();
BTF_KFUNCS_START(scx_kfunc_ids_enqueue_dispatch)
BTF_ID_FLAGS(func, scx_bpf_dispatch, KF_RCU)
+BTF_ID_FLAGS(func, scx_bpf_dispatch_vtime, KF_RCU)
BTF_KFUNCS_END(scx_kfunc_ids_enqueue_dispatch)
static const struct btf_kfunc_id_set scx_kfunc_set_enqueue_dispatch = {
@@ -31,6 +31,7 @@ static inline void ___vmlinux_h_sanity_check___(void)
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;
void scx_bpf_dispatch(struct task_struct *p, u64 dsq_id, u64 slice, u64 enq_flags) __ksym;
+void scx_bpf_dispatch_vtime(struct task_struct *p, u64 dsq_id, u64 slice, u64 vtime, u64 enq_flags) __ksym;
u32 scx_bpf_dispatch_nr_slots(void) __ksym;
void scx_bpf_dispatch_cancel(void) __ksym;
bool scx_bpf_consume(u64 dsq_id) __ksym;
@@ -38,6 +38,10 @@
* this isn't a real concern especially given the performance gain. Also, there
* are ways to mitigate the problem further by e.g. introducing an extra
* scheduling layer on cgroup delegation boundaries.
+ *
+ * The scheduler first picks the cgroup to run and then schedule the tasks
+ * within by using nested weighted vtime scheduling by default. The
+ * cgroup-internal scheduling can be switched to FIFO with the -f option.
*/
#include <scx/common.bpf.h>
#include "scx_flatcg.h"
@@ -51,6 +55,7 @@ char _license[] SEC("license") = "GPL";
const volatile u32 nr_cpus = 32; /* !0 for veristat, set during init */
const volatile u64 cgrp_slice_ns = SCX_SLICE_DFL;
+const volatile bool fifo_sched;
u64 cvtime_now;
UEI_DEFINE(uei);
@@ -387,7 +392,21 @@ void BPF_STRUCT_OPS(fcg_enqueue, struct task_struct *p, u64 enq_flags)
if (!cgc)
goto out_release;
- scx_bpf_dispatch(p, cgrp->kn->id, SCX_SLICE_DFL, enq_flags);
+ if (fifo_sched) {
+ scx_bpf_dispatch(p, cgrp->kn->id, SCX_SLICE_DFL, enq_flags);
+ } else {
+ u64 tvtime = p->scx.dsq_vtime;
+
+ /*
+ * Limit the amount of budget that an idling task can accumulate
+ * to one slice.
+ */
+ if (vtime_before(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,
+ tvtime, enq_flags);
+ }
cgrp_enqueued(cgrp, cgc);
out_release:
@@ -499,12 +518,48 @@ void BPF_STRUCT_OPS(fcg_runnable, struct task_struct *p, u64 enq_flags)
bpf_cgroup_release(cgrp);
}
+void BPF_STRUCT_OPS(fcg_running, struct task_struct *p)
+{
+ struct cgroup *cgrp;
+ struct fcg_cgrp_ctx *cgc;
+
+ if (fifo_sched)
+ return;
+
+ cgrp = scx_bpf_task_cgroup(p);
+ cgc = find_cgrp_ctx(cgrp);
+ if (cgc) {
+ /*
+ * @cgc->tvtime_now always progresses forward as tasks start
+ * executing. The test and update can be performed concurrently
+ * from multiple CPUs and thus racy. Any error should be
+ * contained and temporary. Let's just live with it.
+ */
+ if (vtime_before(cgc->tvtime_now, p->scx.dsq_vtime))
+ cgc->tvtime_now = p->scx.dsq_vtime;
+ }
+ bpf_cgroup_release(cgrp);
+}
+
void BPF_STRUCT_OPS(fcg_stopping, struct task_struct *p, bool runnable)
{
struct fcg_task_ctx *taskc;
struct cgroup *cgrp;
struct fcg_cgrp_ctx *cgc;
+ /*
+ * Scale the execution time by the inverse of the weight and charge.
+ *
+ * Note that the default yield implementation yields by setting
+ * @p->scx.slice to zero and the following would treat the yielding task
+ * as if it has consumed all its slice. If this penalizes yielding tasks
+ * too much, determine the execution time by taking explicit timestamps
+ * instead of depending on @p->scx.slice.
+ */
+ if (!fifo_sched)
+ p->scx.dsq_vtime +=
+ (SCX_SLICE_DFL - p->scx.slice) * 100 / p->scx.weight;
+
taskc = bpf_task_storage_get(&task_ctx, p, 0, 0);
if (!taskc) {
scx_bpf_error("task_ctx lookup failed");
@@ -742,6 +797,7 @@ s32 BPF_STRUCT_OPS(fcg_init_task, struct task_struct *p,
struct scx_init_task_args *args)
{
struct fcg_task_ctx *taskc;
+ struct fcg_cgrp_ctx *cgc;
/*
* @p is new. Let's ensure that its task_ctx is available. We can sleep
@@ -753,6 +809,12 @@ s32 BPF_STRUCT_OPS(fcg_init_task, struct task_struct *p,
return -ENOMEM;
taskc->bypassed_at = 0;
+
+ if (!(cgc = find_cgrp_ctx(args->cgroup)))
+ return -ENOENT;
+
+ p->scx.dsq_vtime = cgc->tvtime_now;
+
return 0;
}
@@ -840,6 +902,20 @@ void BPF_STRUCT_OPS(fcg_cgroup_exit, struct cgroup *cgrp)
scx_bpf_destroy_dsq(cgid);
}
+void BPF_STRUCT_OPS(fcg_cgroup_move, struct task_struct *p,
+ struct cgroup *from, struct cgroup *to)
+{
+ struct fcg_cgrp_ctx *from_cgc, *to_cgc;
+ s64 vtime_delta;
+
+ /* find_cgrp_ctx() triggers scx_ops_error() on lookup failures */
+ if (!(from_cgc = find_cgrp_ctx(from)) || !(to_cgc = find_cgrp_ctx(to)))
+ return;
+
+ vtime_delta = p->scx.dsq_vtime - from_cgc->tvtime_now;
+ p->scx.dsq_vtime = to_cgc->tvtime_now + vtime_delta;
+}
+
void BPF_STRUCT_OPS(fcg_exit, struct scx_exit_info *ei)
{
UEI_RECORD(uei, ei);
@@ -850,12 +926,14 @@ SCX_OPS_DEFINE(flatcg_ops,
.enqueue = (void *)fcg_enqueue,
.dispatch = (void *)fcg_dispatch,
.runnable = (void *)fcg_runnable,
+ .running = (void *)fcg_running,
.stopping = (void *)fcg_stopping,
.quiescent = (void *)fcg_quiescent,
.init_task = (void *)fcg_init_task,
.cgroup_set_weight = (void *)fcg_cgroup_set_weight,
.cgroup_init = (void *)fcg_cgroup_init,
.cgroup_exit = (void *)fcg_cgroup_exit,
+ .cgroup_move = (void *)fcg_cgroup_move,
.exit = (void *)fcg_exit,
.flags = SCX_OPS_CGROUP_KNOB_WEIGHT | SCX_OPS_ENQ_EXITING,
.name = "flatcg");
@@ -26,10 +26,11 @@ const char help_fmt[] =
"\n"
"See the top-level comment in .bpf.c for more details.\n"
"\n"
-"Usage: %s [-s SLICE_US] [-i INTERVAL] [-v]\n"
+"Usage: %s [-s SLICE_US] [-i INTERVAL] [-f] [-v]\n"
"\n"
" -s SLICE_US Override slice duration\n"
" -i INTERVAL Report interval\n"
+" -f Use FIFO scheduling instead of weighted vtime scheduling\n"
" -v Print libbpf debug messages\n"
" -h Display this help and exit\n";
@@ -137,7 +138,7 @@ int main(int argc, char **argv)
skel->rodata->nr_cpus = libbpf_num_possible_cpus();
- while ((opt = getopt(argc, argv, "s:i:dvh")) != -1) {
+ while ((opt = getopt(argc, argv, "s:i:dfvh")) != -1) {
double v;
switch (opt) {
@@ -153,6 +154,9 @@ int main(int argc, char **argv)
case 'd':
dump_cgrps = true;
break;
+ case 'f':
+ skel->rodata->fifo_sched = true;
+ break;
case 'v':
verbose = true;
break;
@@ -2,11 +2,20 @@
/*
* A simple scheduler.
*
- * A simple global FIFO scheduler. It also demonstrates the following niceties.
+ * By default, it operates as a simple global weighted vtime scheduler and can
+ * be switched to FIFO scheduling. It also demonstrates the following niceties.
*
* - Statistics tracking how many tasks are queued to local and global dsq's.
* - Termination notification for userspace.
*
+ * While very simple, this scheduler should work reasonably well on CPUs with a
+ * uniform L3 cache topology. While preemption is not implemented, the fact that
+ * the scheduling queue is shared across all CPUs means that whatever is at the
+ * front of the queue is likely to be executed fairly quickly given enough
+ * number of CPUs. The FIFO scheduling mode may be beneficial to some workloads
+ * but comes with the usual problems with FIFO scheduling where saturating
+ * threads can easily drown out interactive ones.
+ *
* Copyright (c) 2022 Meta Platforms, Inc. and affiliates.
* Copyright (c) 2022 Tejun Heo <tj@kernel.org>
* Copyright (c) 2022 David Vernet <dvernet@meta.com>
@@ -15,8 +24,13 @@
char _license[] SEC("license") = "GPL";
+const volatile bool fifo_sched;
+
+static u64 vtime_now;
UEI_DEFINE(uei);
+#define SHARED_DSQ 0
+
struct {
__uint(type, BPF_MAP_TYPE_PERCPU_ARRAY);
__uint(key_size, sizeof(u32));
@@ -31,6 +45,11 @@ static void stat_inc(u32 idx)
(*cnt_p)++;
}
+static inline bool vtime_before(u64 a, u64 b)
+{
+ return (s64)(a - b) < 0;
+}
+
s32 BPF_STRUCT_OPS(simple_select_cpu, struct task_struct *p, s32 prev_cpu, u64 wake_flags)
{
bool is_idle = false;
@@ -48,7 +67,69 @@ s32 BPF_STRUCT_OPS(simple_select_cpu, struct task_struct *p, s32 prev_cpu, u64 w
void BPF_STRUCT_OPS(simple_enqueue, struct task_struct *p, u64 enq_flags)
{
stat_inc(1); /* count global queueing */
- scx_bpf_dispatch(p, SCX_DSQ_GLOBAL, SCX_SLICE_DFL, enq_flags);
+
+ if (fifo_sched) {
+ scx_bpf_dispatch(p, SHARED_DSQ, SCX_SLICE_DFL, enq_flags);
+ } else {
+ u64 vtime = p->scx.dsq_vtime;
+
+ /*
+ * Limit the amount of budget that an idling task can accumulate
+ * to one slice.
+ */
+ if (vtime_before(vtime, vtime_now - SCX_SLICE_DFL))
+ vtime = vtime_now - SCX_SLICE_DFL;
+
+ scx_bpf_dispatch_vtime(p, SHARED_DSQ, SCX_SLICE_DFL, vtime,
+ enq_flags);
+ }
+}
+
+void BPF_STRUCT_OPS(simple_dispatch, s32 cpu, struct task_struct *prev)
+{
+ scx_bpf_consume(SHARED_DSQ);
+}
+
+void BPF_STRUCT_OPS(simple_running, struct task_struct *p)
+{
+ if (fifo_sched)
+ return;
+
+ /*
+ * Global vtime always progresses forward as tasks start executing. The
+ * test and update can be performed concurrently from multiple CPUs and
+ * thus racy. Any error should be contained and temporary. Let's just
+ * live with it.
+ */
+ if (vtime_before(vtime_now, p->scx.dsq_vtime))
+ vtime_now = p->scx.dsq_vtime;
+}
+
+void BPF_STRUCT_OPS(simple_stopping, struct task_struct *p, bool runnable)
+{
+ if (fifo_sched)
+ return;
+
+ /*
+ * Scale the execution time by the inverse of the weight and charge.
+ *
+ * Note that the default yield implementation yields by setting
+ * @p->scx.slice to zero and the following would treat the yielding task
+ * as if it has consumed all its slice. If this penalizes yielding tasks
+ * too much, determine the execution time by taking explicit timestamps
+ * instead of depending on @p->scx.slice.
+ */
+ p->scx.dsq_vtime += (SCX_SLICE_DFL - p->scx.slice) * 100 / p->scx.weight;
+}
+
+void BPF_STRUCT_OPS(simple_enable, struct task_struct *p)
+{
+ p->scx.dsq_vtime = vtime_now;
+}
+
+s32 BPF_STRUCT_OPS_SLEEPABLE(simple_init)
+{
+ return scx_bpf_create_dsq(SHARED_DSQ, -1);
}
void BPF_STRUCT_OPS(simple_exit, struct scx_exit_info *ei)
@@ -59,5 +140,10 @@ void BPF_STRUCT_OPS(simple_exit, struct scx_exit_info *ei)
SCX_OPS_DEFINE(simple_ops,
.select_cpu = (void *)simple_select_cpu,
.enqueue = (void *)simple_enqueue,
+ .dispatch = (void *)simple_dispatch,
+ .running = (void *)simple_running,
+ .stopping = (void *)simple_stopping,
+ .enable = (void *)simple_enable,
+ .init = (void *)simple_init,
.exit = (void *)simple_exit,
.name = "simple");
@@ -17,8 +17,9 @@ const char help_fmt[] =
"\n"
"See the top-level comment in .bpf.c for more details.\n"
"\n"
-"Usage: %s [-v]\n"
+"Usage: %s [-f] [-v]\n"
"\n"
+" -f Use FIFO scheduling instead of weighted vtime scheduling\n"
" -v Print libbpf debug messages\n"
" -h Display this help and exit\n";
@@ -70,8 +71,11 @@ int main(int argc, char **argv)
restart:
skel = SCX_OPS_OPEN(simple_ops, scx_simple);
- while ((opt = getopt(argc, argv, "vh")) != -1) {
+ while ((opt = getopt(argc, argv, "fvh")) != -1) {
switch (opt) {
+ case 'f':
+ skel->rodata->fifo_sched = true;
+ break;
case 'v':
verbose = true;
break;