diff options
Diffstat (limited to 'kernel/sched/ext.c')
| -rw-r--r-- | kernel/sched/ext.c | 977 | 
1 files changed, 676 insertions, 301 deletions
| diff --git a/kernel/sched/ext.c b/kernel/sched/ext.c index 51b7e04879d7..7fff1d045477 100644 --- a/kernel/sched/ext.c +++ b/kernel/sched/ext.c @@ -199,8 +199,10 @@ struct scx_dump_ctx {  /**   * struct sched_ext_ops - Operation table for BPF scheduler implementation   * - * Userland can implement an arbitrary scheduling policy by implementing and - * loading operations in this table. + * A BPF scheduler can implement an arbitrary scheduling policy by + * implementing and loading operations in this table. Note that a userland + * scheduling policy can also be implemented using the BPF scheduler + * as a shim layer.   */  struct sched_ext_ops {  	/** @@ -218,10 +220,15 @@ struct sched_ext_ops {  	 * dispatch. While an explicit custom mechanism can be added,  	 * select_cpu() serves as the default way to wake up idle CPUs.  	 * -	 * @p may be dispatched directly by calling scx_bpf_dispatch(). If @p -	 * is dispatched, the ops.enqueue() callback will be skipped. Finally, -	 * if @p is dispatched to SCX_DSQ_LOCAL, it will be dispatched to the -	 * local DSQ of whatever CPU is returned by this callback. +	 * @p may be inserted into a DSQ directly by calling +	 * scx_bpf_dsq_insert(). If so, the ops.enqueue() will be skipped. +	 * Directly inserting into %SCX_DSQ_LOCAL will put @p in the local DSQ +	 * of the CPU returned by this operation. +	 * +	 * 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 +	 * the option to select a different CPU. See select_task_rq() for +	 * details.  	 */  	s32 (*select_cpu)(struct task_struct *p, s32 prev_cpu, u64 wake_flags); @@ -230,12 +237,12 @@ struct sched_ext_ops {  	 * @p: task being enqueued  	 * @enq_flags: %SCX_ENQ_*  	 * -	 * @p is ready to run. Dispatch directly by calling scx_bpf_dispatch() -	 * or enqueue on the BPF scheduler. If not directly dispatched, the bpf -	 * scheduler owns @p and if it fails to dispatch @p, the task will -	 * stall. +	 * @p is ready to run. Insert directly into a DSQ by calling +	 * scx_bpf_dsq_insert() or enqueue on the BPF scheduler. If not directly +	 * inserted, the bpf scheduler owns @p and if it fails to dispatch @p, +	 * 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.  	 */  	void (*enqueue)(struct task_struct *p, u64 enq_flags); @@ -257,17 +264,17 @@ struct sched_ext_ops {  	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_dispatch() 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_dispatch() can be called without -	 * an intervening scx_bpf_consume() is specified by +	 * The maximum number of times scx_bpf_dsq_insert() can be called +	 * 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.  	 * @@ -275,7 +282,7 @@ struct sched_ext_ops {  	 * @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); @@ -594,7 +601,7 @@ struct sched_ext_ops {  	 * Update @tg's weight to @weight.  	 */  	void (*cgroup_set_weight)(struct cgroup *cgrp, u32 weight); -#endif	/* CONFIG_CGROUPS */ +#endif	/* CONFIG_EXT_GROUP_SCHED */  	/*  	 * All online ops must come before ops.cpu_online(). @@ -707,7 +714,7 @@ enum scx_enq_flags {  	/*  	 * 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  	 * scheduling path. Implies %SCX_ENQ_HEAD.  	 */ @@ -862,8 +869,9 @@ static DEFINE_MUTEX(scx_ops_enable_mutex);  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 unsigned long scx_in_softlockup; +static atomic_t scx_ops_breather_depth = ATOMIC_INIT(0);  static int scx_ops_bypass_depth; -static DEFINE_RAW_SPINLOCK(__scx_ops_bypass_lock);  static bool scx_ops_init_task_enabled;  static bool scx_switching_all;  DEFINE_STATIC_KEY_FALSE(__scx_switched_all); @@ -876,6 +884,11 @@ static DEFINE_STATIC_KEY_FALSE(scx_ops_enq_exiting);  static DEFINE_STATIC_KEY_FALSE(scx_ops_cpu_preempt);  static DEFINE_STATIC_KEY_FALSE(scx_builtin_idle_enabled); +#ifdef CONFIG_SMP +static DEFINE_STATIC_KEY_FALSE(scx_selcpu_topo_llc); +static DEFINE_STATIC_KEY_FALSE(scx_selcpu_topo_numa); +#endif +  static struct static_key_false scx_has_op[SCX_OPI_END] =  	{ [0 ... SCX_OPI_END-1] = STATIC_KEY_FALSE_INIT }; @@ -2309,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  	 * 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.  	 */  	if (!task_allowed_on_cpu(p, cpu)) { @@ -2397,11 +2410,115 @@ static inline bool task_can_run_on_remote_rq(struct task_struct *p, struct rq *r  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 */ +/** + * move_task_between_dsqs() - Move a task from one DSQ to another + * @p: target task + * @enq_flags: %SCX_ENQ_* + * @src_dsq: DSQ @p is currently on, must not be a local DSQ + * @dst_dsq: DSQ @p is being moved to, can be any DSQ + * + * Must be called with @p's task_rq and @src_dsq locked. If @dst_dsq is a local + * DSQ and @p is on a different CPU, @p will be migrated and thus its task_rq + * will change. As @p's task_rq is locked, this function doesn't need to use the + * holding_cpu mechanism. + * + * On return, @src_dsq is unlocked and only @p's new task_rq, which is the + * return value, is locked. + */ +static struct rq *move_task_between_dsqs(struct task_struct *p, u64 enq_flags, +					 struct scx_dispatch_q *src_dsq, +					 struct scx_dispatch_q *dst_dsq) +{ +	struct rq *src_rq = task_rq(p), *dst_rq; + +	BUG_ON(src_dsq->id == SCX_DSQ_LOCAL); +	lockdep_assert_held(&src_dsq->lock); +	lockdep_assert_rq_held(src_rq); + +	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 = find_global_dsq(p); +			dst_rq = src_rq; +		} +	} else { +		/* no need to migrate if destination is a non-local DSQ */ +		dst_rq = src_rq; +	} + +	/* +	 * Move @p into $dst_dsq. If $dst_dsq is the local DSQ of a different +	 * CPU, @p will be migrated. +	 */ +	if (dst_dsq->id == SCX_DSQ_LOCAL) { +		/* @p is going from a non-local DSQ to a local DSQ */ +		if (src_rq == dst_rq) { +			task_unlink_from_dsq(p, src_dsq); +			move_local_task_to_local_dsq(p, enq_flags, +						     src_dsq, dst_rq); +			raw_spin_unlock(&src_dsq->lock); +		} else { +			raw_spin_unlock(&src_dsq->lock); +			move_remote_task_to_local_dsq(p, enq_flags, +						      src_rq, dst_rq); +		} +	} else { +		/* +		 * @p is going from a non-local DSQ to a non-local DSQ. As +		 * $src_dsq is already locked, do an abbreviated dequeue. +		 */ +		task_unlink_from_dsq(p, src_dsq); +		p->scx.dsq = NULL; +		raw_spin_unlock(&src_dsq->lock); + +		dispatch_enqueue(dst_dsq, p, enq_flags); +	} + +	return dst_rq; +} + +/* + * A poorly behaving BPF scheduler can live-lock the system by e.g. incessantly + * banging on the same DSQ on a large NUMA system to the point where switching + * to the bypass mode can take a long time. Inject artifical delays while the + * bypass mode is switching to guarantee timely completion. + */ +static void scx_ops_breather(struct rq *rq) +{ +	u64 until; + +	lockdep_assert_rq_held(rq); + +	if (likely(!atomic_read(&scx_ops_breather_depth))) +		return; + +	raw_spin_rq_unlock(rq); + +	until = ktime_get_ns() + NSEC_PER_MSEC; + +	do { +		int cnt = 1024; +		while (atomic_read(&scx_ops_breather_depth) && --cnt) +			cpu_relax(); +	} while (atomic_read(&scx_ops_breather_depth) && +		 time_before64(ktime_get_ns(), until)); + +	raw_spin_rq_lock(rq); +} +  static bool consume_dispatch_q(struct rq *rq, struct scx_dispatch_q *dsq)  {  	struct task_struct *p;  retry:  	/* +	 * This retry loop can repeatedly race against scx_ops_bypass() +	 * dequeueing tasks from @dsq trying to put the system into the bypass +	 * mode. On some multi-socket machines (e.g. 2x Intel 8480c), this can +	 * live-lock the machine into soft lockups. Give a breather. +	 */ +	scx_ops_breather(rq); + +	/*  	 * The caller can't expect to successfully consume a task if the task's  	 * addition to @dsq isn't guaranteed to be visible somehow. Test  	 * @dsq->list without locking and skip if it seems empty. @@ -2541,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   * require rq lock dancing. As we don't wanna do either while inside   * 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   * finish up.   * @@ -2573,7 +2690,7 @@ retry:  		/*  		 * If qseq doesn't match, @p has gone through at least one  		 * 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)  			return; @@ -2642,10 +2759,10 @@ static int balance_one(struct rq *rq, struct task_struct *prev)  		 * If the previous sched_class for the current CPU was not SCX,  		 * notify the BPF scheduler that it again has control of the  		 * core. This callback complements ->cpu_release(), which is -		 * emitted in scx_next_task_picked(). +		 * emitted in switch_class().  		 */  		if (SCX_HAS_OP(cpu_acquire)) -			SCX_CALL_OP(0, cpu_acquire, cpu_of(rq), NULL); +			SCX_CALL_OP(SCX_KF_REST, cpu_acquire, cpu_of(rq), NULL);  		rq->scx.cpu_released = false;  	} @@ -3098,28 +3215,216 @@ found:  		goto retry;  } +/* + * Return true if the LLC domains do not perfectly overlap with the NUMA + * domains, false otherwise. + */ +static bool llc_numa_mismatch(void) +{ +	int cpu; + +	/* +	 * We need to scan all online CPUs to verify whether their scheduling +	 * domains overlap. +	 * +	 * While it is rare to encounter architectures with asymmetric NUMA +	 * topologies, CPU hotplugging or virtualized environments can result +	 * in asymmetric configurations. +	 * +	 * For example: +	 * +	 *  NUMA 0: +	 *    - LLC 0: cpu0..cpu7 +	 *    - LLC 1: cpu8..cpu15 [offline] +	 * +	 *  NUMA 1: +	 *    - LLC 0: cpu16..cpu23 +	 *    - LLC 1: cpu24..cpu31 +	 * +	 * In this case, if we only check the first online CPU (cpu0), we might +	 * incorrectly assume that the LLC and NUMA domains are fully +	 * overlapping, which is incorrect (as NUMA 1 has two distinct LLC +	 * domains). +	 */ +	for_each_online_cpu(cpu) { +		const struct cpumask *numa_cpus; +		struct sched_domain *sd; + +		sd = rcu_dereference(per_cpu(sd_llc, cpu)); +		if (!sd) +			return true; + +		numa_cpus = cpumask_of_node(cpu_to_node(cpu)); +		if (sd->span_weight != cpumask_weight(numa_cpus)) +			return true; +	} + +	return false; +} + +/* + * Initialize topology-aware scheduling. + * + * Detect if the system has multiple LLC or multiple NUMA domains and enable + * cache-aware / NUMA-aware scheduling optimizations in the default CPU idle + * selection policy. + * + * Assumption: the kernel's internal topology representation assumes that each + * CPU belongs to a single LLC domain, and that each LLC domain is entirely + * contained within a single NUMA node. + */ +static void update_selcpu_topology(void) +{ +	bool enable_llc = false, enable_numa = false; +	struct sched_domain *sd; +	const struct cpumask *cpus; +	s32 cpu = cpumask_first(cpu_online_mask); + +	/* +	 * Enable LLC domain optimization only when there are multiple LLC +	 * domains among the online CPUs. If all online CPUs are part of a +	 * single LLC domain, the idle CPU selection logic can choose any +	 * online CPU without bias. +	 * +	 * Note that it is sufficient to check the LLC domain of the first +	 * online CPU to determine whether a single LLC domain includes all +	 * CPUs. +	 */ +	rcu_read_lock(); +	sd = rcu_dereference(per_cpu(sd_llc, cpu)); +	if (sd) { +		if (sd->span_weight < num_online_cpus()) +			enable_llc = true; +	} + +	/* +	 * Enable NUMA optimization only when there are multiple NUMA domains +	 * among the online CPUs and the NUMA domains don't perfectly overlaps +	 * with the LLC domains. +	 * +	 * If all CPUs belong to the same NUMA node and the same LLC domain, +	 * enabling both NUMA and LLC optimizations is unnecessary, as checking +	 * for an idle CPU in the same domain twice is redundant. +	 */ +	cpus = cpumask_of_node(cpu_to_node(cpu)); +	if ((cpumask_weight(cpus) < num_online_cpus()) && llc_numa_mismatch()) +		enable_numa = true; +	rcu_read_unlock(); + +	pr_debug("sched_ext: LLC idle selection %s\n", +		 enable_llc ? "enabled" : "disabled"); +	pr_debug("sched_ext: NUMA idle selection %s\n", +		 enable_numa ? "enabled" : "disabled"); + +	if (enable_llc) +		static_branch_enable_cpuslocked(&scx_selcpu_topo_llc); +	else +		static_branch_disable_cpuslocked(&scx_selcpu_topo_llc); +	if (enable_numa) +		static_branch_enable_cpuslocked(&scx_selcpu_topo_numa); +	else +		static_branch_disable_cpuslocked(&scx_selcpu_topo_numa); +} + +/* + * Built-in CPU idle selection policy: + * + * 1. Prioritize full-idle cores: + *   - always prioritize CPUs from fully idle cores (both logical CPUs are + *     idle) to avoid interference caused by SMT. + * + * 2. Reuse the same CPU: + *   - prefer the last used CPU to take advantage of cached data (L1, L2) and + *     branch prediction optimizations. + * + * 3. Pick a CPU within the same LLC (Last-Level Cache): + *   - if the above conditions aren't met, pick a CPU that shares the same LLC + *     to maintain cache locality. + * + * 4. Pick a CPU within the same NUMA node, if enabled: + *   - choose a CPU from the same NUMA node to reduce memory access latency. + * + * Step 3 and 4 are performed only if the system has, respectively, multiple + * LLC domains / multiple NUMA nodes (see scx_selcpu_topo_llc and + * scx_selcpu_topo_numa). + * + * NOTE: tasks that can only run on 1 CPU are excluded by this logic, because + * we never call ops.select_cpu() for them, see select_task_rq(). + */  static s32 scx_select_cpu_dfl(struct task_struct *p, s32 prev_cpu,  			      u64 wake_flags, bool *found)  { +	const struct cpumask *llc_cpus = NULL; +	const struct cpumask *numa_cpus = NULL;  	s32 cpu;  	*found = false; + +	/* +	 * This is necessary to protect llc_cpus. +	 */ +	rcu_read_lock(); + +	/* +	 * Determine the scheduling domain only if the task is allowed to run +	 * on all CPUs. +	 * +	 * This is done primarily for efficiency, as it avoids the overhead of +	 * updating a cpumask every time we need to select an idle CPU (which +	 * can be costly in large SMP systems), but it also aligns logically: +	 * if a task's scheduling domain is restricted by user-space (through +	 * CPU affinity), the task will simply use the flat scheduling domain +	 * defined by user-space. +	 */ +	if (p->nr_cpus_allowed >= num_possible_cpus()) { +		if (static_branch_maybe(CONFIG_NUMA, &scx_selcpu_topo_numa)) +			numa_cpus = cpumask_of_node(cpu_to_node(prev_cpu)); + +		if (static_branch_maybe(CONFIG_SCHED_MC, &scx_selcpu_topo_llc)) { +			struct sched_domain *sd; + +			sd = rcu_dereference(per_cpu(sd_llc, prev_cpu)); +			if (sd) +				llc_cpus = sched_domain_span(sd); +		} +	} +  	/* -	 * If WAKE_SYNC, the waker's local DSQ is empty, and the system is -	 * under utilized, wake up @p to the local DSQ of the waker. Checking -	 * only for an empty local DSQ is insufficient as it could give the -	 * wakee an unfair advantage when the system is oversaturated. -	 * Checking only for the presence of idle CPUs is also insufficient as -	 * the local DSQ of the waker could have tasks piled up on it even if -	 * there is an idle core elsewhere on the system. -	 */ -	cpu = smp_processor_id(); -	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)) +	 * If WAKE_SYNC, try to migrate the wakee to the waker's CPU. +	 */ +	if (wake_flags & SCX_WAKE_SYNC) { +		cpu = smp_processor_id(); + +		/* +		 * If the waker's CPU is cache affine and prev_cpu is idle, +		 * then avoid a migration. +		 */ +		if (cpus_share_cache(cpu, prev_cpu) && +		    test_and_clear_cpu_idle(prev_cpu)) { +			cpu = prev_cpu;  			goto cpu_found; +		} + +		/* +		 * If the waker's local DSQ is empty, and the system is under +		 * utilized, try to wake up @p to the local DSQ of the waker. +		 * +		 * Checking only for an empty local DSQ is insufficient as it +		 * could give the wakee an unfair advantage when the system is +		 * oversaturated. +		 * +		 * Checking only for the presence of idle CPUs is also +		 * insufficient as the local DSQ of the waker could have tasks +		 * piled up on it even if there is an idle core elsewhere on +		 * the system. +		 */ +		if (!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; +		}  	}  	/* @@ -3127,29 +3432,80 @@ static s32 scx_select_cpu_dfl(struct task_struct *p, s32 prev_cpu,  	 * partially idle @prev_cpu.  	 */  	if (sched_smt_active()) { +		/* +		 * Keep using @prev_cpu if it's part of a fully idle core. +		 */  		if (cpumask_test_cpu(prev_cpu, idle_masks.smt) &&  		    test_and_clear_cpu_idle(prev_cpu)) {  			cpu = prev_cpu;  			goto cpu_found;  		} +		/* +		 * Search for any fully idle core in the same LLC domain. +		 */ +		if (llc_cpus) { +			cpu = scx_pick_idle_cpu(llc_cpus, SCX_PICK_IDLE_CORE); +			if (cpu >= 0) +				goto cpu_found; +		} + +		/* +		 * Search for any fully idle core in the same NUMA node. +		 */ +		if (numa_cpus) { +			cpu = scx_pick_idle_cpu(numa_cpus, SCX_PICK_IDLE_CORE); +			if (cpu >= 0) +				goto cpu_found; +		} + +		/* +		 * Search for any full idle core usable by the task. +		 */  		cpu = scx_pick_idle_cpu(p->cpus_ptr, SCX_PICK_IDLE_CORE);  		if (cpu >= 0)  			goto cpu_found;  	} +	/* +	 * Use @prev_cpu if it's idle. +	 */  	if (test_and_clear_cpu_idle(prev_cpu)) {  		cpu = prev_cpu;  		goto cpu_found;  	} +	/* +	 * Search for any idle CPU in the same LLC domain. +	 */ +	if (llc_cpus) { +		cpu = scx_pick_idle_cpu(llc_cpus, 0); +		if (cpu >= 0) +			goto cpu_found; +	} + +	/* +	 * Search for any idle CPU in the same NUMA node. +	 */ +	if (numa_cpus) { +		cpu = scx_pick_idle_cpu(numa_cpus, 0); +		if (cpu >= 0) +			goto cpu_found; +	} + +	/* +	 * Search for any idle CPU usable by the task. +	 */  	cpu = scx_pick_idle_cpu(p->cpus_ptr, 0);  	if (cpu >= 0)  		goto cpu_found; +	rcu_read_unlock();  	return prev_cpu;  cpu_found: +	rcu_read_unlock(); +  	*found = true;  	return cpu;  } @@ -3272,6 +3628,9 @@ static void handle_hotplug(struct rq *rq, bool online)  	atomic_long_inc(&scx_hotplug_seq); +	if (scx_enabled()) +		update_selcpu_topology(); +  	if (online && SCX_HAS_OP(cpu_online))  		SCX_CALL_OP(SCX_KF_UNLOCKED, cpu_online, cpu);  	else if (!online && SCX_HAS_OP(cpu_offline)) @@ -3567,12 +3926,7 @@ static void scx_ops_exit_task(struct task_struct *p)  void init_scx_entity(struct sched_ext_entity *scx)  { -	/* -	 * init_idle() calls this function again after fork sequence is -	 * complete. Don't touch ->tasks_node as it's already linked. -	 */ -	memset(scx, 0, offsetof(struct sched_ext_entity, tasks_node)); - +	memset(scx, 0, sizeof(*scx));  	INIT_LIST_HEAD(&scx->dsq_list.node);  	RB_CLEAR_NODE(&scx->dsq_priq);  	scx->sticky_cpu = -1; @@ -4286,6 +4640,49 @@ bool task_should_scx(int policy)  }  /** + * scx_softlockup - sched_ext softlockup handler + * + * On some multi-socket setups (e.g. 2x Intel 8480c), the BPF scheduler can + * live-lock the system by making many CPUs target the same DSQ to the point + * where soft-lockup detection triggers. This function is called from + * soft-lockup watchdog when the triggering point is close and tries to unjam + * the system by enabling the breather and aborting the BPF scheduler. + */ +void scx_softlockup(u32 dur_s) +{ +	switch (scx_ops_enable_state()) { +	case SCX_OPS_ENABLING: +	case SCX_OPS_ENABLED: +		break; +	default: +		return; +	} + +	/* allow only one instance, cleared at the end of scx_ops_bypass() */ +	if (test_and_set_bit(0, &scx_in_softlockup)) +		return; + +	printk_deferred(KERN_ERR "sched_ext: Soft lockup - CPU%d stuck for %us, disabling \"%s\"\n", +			smp_processor_id(), dur_s, scx_ops.name); + +	/* +	 * Some CPUs may be trapped in the dispatch paths. Enable breather +	 * immediately; otherwise, we might even be able to get to +	 * scx_ops_bypass(). +	 */ +	atomic_inc(&scx_ops_breather_depth); + +	scx_ops_error("soft lockup - CPU#%d stuck for %us", +		      smp_processor_id(), dur_s); +} + +static void scx_clear_softlockup(void) +{ +	if (test_and_clear_bit(0, &scx_in_softlockup)) +		atomic_dec(&scx_ops_breather_depth); +} + +/**   * scx_ops_bypass - [Un]bypass scx_ops and guarantee forward progress   *   * Bypassing guarantees that all runnable tasks make forward progress without @@ -4317,10 +4714,11 @@ bool task_should_scx(int policy)   */  static void scx_ops_bypass(bool bypass)  { +	static DEFINE_RAW_SPINLOCK(bypass_lock);  	int cpu;  	unsigned long flags; -	raw_spin_lock_irqsave(&__scx_ops_bypass_lock, flags); +	raw_spin_lock_irqsave(&bypass_lock, flags);  	if (bypass) {  		scx_ops_bypass_depth++;  		WARN_ON_ONCE(scx_ops_bypass_depth <= 0); @@ -4333,6 +4731,8 @@ static void scx_ops_bypass(bool bypass)  			goto unlock;  	} +	atomic_inc(&scx_ops_breather_depth); +  	/*  	 * No task property is changing. We just need to make sure all currently  	 * queued tasks are re-queued according to the new scx_rq_bypassing() @@ -4388,8 +4788,11 @@ static void scx_ops_bypass(bool bypass)  		/* resched to restore ticks and idle state */  		resched_cpu(cpu);  	} + +	atomic_dec(&scx_ops_breather_depth);  unlock: -	raw_spin_unlock_irqrestore(&__scx_ops_bypass_lock, flags); +	raw_spin_unlock_irqrestore(&bypass_lock, flags); +	scx_clear_softlockup();  }  static void free_exit_info(struct scx_exit_info *ei) @@ -5100,6 +5503,9 @@ static int scx_ops_enable(struct sched_ext_ops *ops, struct bpf_link *link)  			static_branch_enable_cpuslocked(&scx_has_op[i]);  	check_hotplug_seq(ops); +#ifdef CONFIG_SMP +	update_selcpu_topology(); +#endif  	cpus_read_unlock();  	ret = validate_ops(ops); @@ -5307,67 +5713,7 @@ err_disable:  #include <linux/bpf.h>  #include <linux/btf.h> -extern struct btf *btf_vmlinux;  static const struct btf_type *task_struct_type; -static u32 task_struct_type_id; - -static bool set_arg_maybe_null(const char *op, int arg_n, int off, int size, -			       enum bpf_access_type type, -			       const struct bpf_prog *prog, -			       struct bpf_insn_access_aux *info) -{ -	struct btf *btf = bpf_get_btf_vmlinux(); -	const struct bpf_struct_ops_desc *st_ops_desc; -	const struct btf_member *member; -	const struct btf_type *t; -	u32 btf_id, member_idx; -	const char *mname; - -	/* struct_ops op args are all sequential, 64-bit numbers */ -	if (off != arg_n * sizeof(__u64)) -		return false; - -	/* btf_id should be the type id of struct sched_ext_ops */ -	btf_id = prog->aux->attach_btf_id; -	st_ops_desc = bpf_struct_ops_find(btf, btf_id); -	if (!st_ops_desc) -		return false; - -	/* BTF type of struct sched_ext_ops */ -	t = st_ops_desc->type; - -	member_idx = prog->expected_attach_type; -	if (member_idx >= btf_type_vlen(t)) -		return false; - -	/* -	 * Get the member name of this struct_ops program, which corresponds to -	 * a field in struct sched_ext_ops. For example, the member name of the -	 * dispatch struct_ops program (callback) is "dispatch". -	 */ -	member = &btf_type_member(t)[member_idx]; -	mname = btf_name_by_offset(btf_vmlinux, member->name_off); - -	if (!strcmp(mname, op)) { -		/* -		 * The value is a pointer to a type (struct task_struct) given -		 * by a BTF ID (PTR_TO_BTF_ID). It is trusted (PTR_TRUSTED), -		 * however, can be a NULL (PTR_MAYBE_NULL). The BPF program -		 * should check the pointer to make sure it is not NULL before -		 * using it, or the verifier will reject the program. -		 * -		 * Longer term, this is something that should be addressed by -		 * BTF, and be fully contained within the verifier. -		 */ -		info->reg_type = PTR_MAYBE_NULL | PTR_TO_BTF_ID | PTR_TRUSTED; -		info->btf = btf_vmlinux; -		info->btf_id = task_struct_type_id; - -		return true; -	} - -	return false; -}  static bool bpf_scx_is_valid_access(int off, int size,  				    enum bpf_access_type type, @@ -5376,9 +5722,6 @@ static bool bpf_scx_is_valid_access(int off, int size,  {  	if (type != BPF_READ)  		return false; -	if (set_arg_maybe_null("dispatch", 1, off, size, type, prog, info) || -	    set_arg_maybe_null("yield", 1, off, size, type, prog, info)) -		return true;  	if (off < 0 || off >= sizeof(__u64) * MAX_BPF_FUNC_ARGS)  		return false;  	if (off % size != 0) @@ -5513,13 +5856,7 @@ static void bpf_scx_unreg(void *kdata, struct bpf_link *link)  static int bpf_scx_init(struct btf *btf)  { -	s32 type_id; - -	type_id = btf_find_by_name_kind(btf, "task_struct", BTF_KIND_STRUCT); -	if (type_id < 0) -		return -EINVAL; -	task_struct_type = btf_type_by_id(btf, type_id); -	task_struct_type_id = type_id; +	task_struct_type = btf_type_by_id(btf, btf_tracing_ids[BTF_TRACING_TYPE_TASK]);  	return 0;  } @@ -5541,78 +5878,78 @@ static int bpf_scx_validate(void *kdata)  	return 0;  } -static s32 select_cpu_stub(struct task_struct *p, s32 prev_cpu, u64 wake_flags) { return -EINVAL; } -static void enqueue_stub(struct task_struct *p, u64 enq_flags) {} -static void dequeue_stub(struct task_struct *p, u64 enq_flags) {} -static void dispatch_stub(s32 prev_cpu, struct task_struct *p) {} -static void tick_stub(struct task_struct *p) {} -static void runnable_stub(struct task_struct *p, u64 enq_flags) {} -static void running_stub(struct task_struct *p) {} -static void stopping_stub(struct task_struct *p, bool runnable) {} -static void quiescent_stub(struct task_struct *p, u64 deq_flags) {} -static bool yield_stub(struct task_struct *from, struct task_struct *to) { return false; } -static bool core_sched_before_stub(struct task_struct *a, struct task_struct *b) { return false; } -static void set_weight_stub(struct task_struct *p, u32 weight) {} -static void set_cpumask_stub(struct task_struct *p, const struct cpumask *mask) {} -static void update_idle_stub(s32 cpu, bool idle) {} -static void cpu_acquire_stub(s32 cpu, struct scx_cpu_acquire_args *args) {} -static void cpu_release_stub(s32 cpu, struct scx_cpu_release_args *args) {} -static s32 init_task_stub(struct task_struct *p, struct scx_init_task_args *args) { return -EINVAL; } -static void exit_task_stub(struct task_struct *p, struct scx_exit_task_args *args) {} -static void enable_stub(struct task_struct *p) {} -static void disable_stub(struct task_struct *p) {} +static s32 sched_ext_ops__select_cpu(struct task_struct *p, s32 prev_cpu, u64 wake_flags) { return -EINVAL; } +static void sched_ext_ops__enqueue(struct task_struct *p, u64 enq_flags) {} +static void sched_ext_ops__dequeue(struct task_struct *p, u64 enq_flags) {} +static void sched_ext_ops__dispatch(s32 prev_cpu, struct task_struct *prev__nullable) {} +static void sched_ext_ops__tick(struct task_struct *p) {} +static void sched_ext_ops__runnable(struct task_struct *p, u64 enq_flags) {} +static void sched_ext_ops__running(struct task_struct *p) {} +static void sched_ext_ops__stopping(struct task_struct *p, bool runnable) {} +static void sched_ext_ops__quiescent(struct task_struct *p, u64 deq_flags) {} +static bool sched_ext_ops__yield(struct task_struct *from, struct task_struct *to__nullable) { return false; } +static bool sched_ext_ops__core_sched_before(struct task_struct *a, struct task_struct *b) { return false; } +static void sched_ext_ops__set_weight(struct task_struct *p, u32 weight) {} +static void sched_ext_ops__set_cpumask(struct task_struct *p, const struct cpumask *mask) {} +static void sched_ext_ops__update_idle(s32 cpu, bool idle) {} +static void sched_ext_ops__cpu_acquire(s32 cpu, struct scx_cpu_acquire_args *args) {} +static void sched_ext_ops__cpu_release(s32 cpu, struct scx_cpu_release_args *args) {} +static s32 sched_ext_ops__init_task(struct task_struct *p, struct scx_init_task_args *args) { return -EINVAL; } +static void sched_ext_ops__exit_task(struct task_struct *p, struct scx_exit_task_args *args) {} +static void sched_ext_ops__enable(struct task_struct *p) {} +static void sched_ext_ops__disable(struct task_struct *p) {}  #ifdef CONFIG_EXT_GROUP_SCHED -static s32 cgroup_init_stub(struct cgroup *cgrp, struct scx_cgroup_init_args *args) { return -EINVAL; } -static void cgroup_exit_stub(struct cgroup *cgrp) {} -static s32 cgroup_prep_move_stub(struct task_struct *p, struct cgroup *from, struct cgroup *to) { return -EINVAL; } -static void cgroup_move_stub(struct task_struct *p, struct cgroup *from, struct cgroup *to) {} -static void cgroup_cancel_move_stub(struct task_struct *p, struct cgroup *from, struct cgroup *to) {} -static void cgroup_set_weight_stub(struct cgroup *cgrp, u32 weight) {} +static s32 sched_ext_ops__cgroup_init(struct cgroup *cgrp, struct scx_cgroup_init_args *args) { return -EINVAL; } +static void sched_ext_ops__cgroup_exit(struct cgroup *cgrp) {} +static s32 sched_ext_ops__cgroup_prep_move(struct task_struct *p, struct cgroup *from, struct cgroup *to) { return -EINVAL; } +static void sched_ext_ops__cgroup_move(struct task_struct *p, struct cgroup *from, struct cgroup *to) {} +static void sched_ext_ops__cgroup_cancel_move(struct task_struct *p, struct cgroup *from, struct cgroup *to) {} +static void sched_ext_ops__cgroup_set_weight(struct cgroup *cgrp, u32 weight) {}  #endif -static void cpu_online_stub(s32 cpu) {} -static void cpu_offline_stub(s32 cpu) {} -static s32 init_stub(void) { return -EINVAL; } -static void exit_stub(struct scx_exit_info *info) {} -static void dump_stub(struct scx_dump_ctx *ctx) {} -static void dump_cpu_stub(struct scx_dump_ctx *ctx, s32 cpu, bool idle) {} -static void dump_task_stub(struct scx_dump_ctx *ctx, struct task_struct *p) {} +static void sched_ext_ops__cpu_online(s32 cpu) {} +static void sched_ext_ops__cpu_offline(s32 cpu) {} +static s32 sched_ext_ops__init(void) { return -EINVAL; } +static void sched_ext_ops__exit(struct scx_exit_info *info) {} +static void sched_ext_ops__dump(struct scx_dump_ctx *ctx) {} +static void sched_ext_ops__dump_cpu(struct scx_dump_ctx *ctx, s32 cpu, bool idle) {} +static void sched_ext_ops__dump_task(struct scx_dump_ctx *ctx, struct task_struct *p) {}  static struct sched_ext_ops __bpf_ops_sched_ext_ops = { -	.select_cpu = select_cpu_stub, -	.enqueue = enqueue_stub, -	.dequeue = dequeue_stub, -	.dispatch = dispatch_stub, -	.tick = tick_stub, -	.runnable = runnable_stub, -	.running = running_stub, -	.stopping = stopping_stub, -	.quiescent = quiescent_stub, -	.yield = yield_stub, -	.core_sched_before = core_sched_before_stub, -	.set_weight = set_weight_stub, -	.set_cpumask = set_cpumask_stub, -	.update_idle = update_idle_stub, -	.cpu_acquire = cpu_acquire_stub, -	.cpu_release = cpu_release_stub, -	.init_task = init_task_stub, -	.exit_task = exit_task_stub, -	.enable = enable_stub, -	.disable = disable_stub, +	.select_cpu		= sched_ext_ops__select_cpu, +	.enqueue		= sched_ext_ops__enqueue, +	.dequeue		= sched_ext_ops__dequeue, +	.dispatch		= sched_ext_ops__dispatch, +	.tick			= sched_ext_ops__tick, +	.runnable		= sched_ext_ops__runnable, +	.running		= sched_ext_ops__running, +	.stopping		= sched_ext_ops__stopping, +	.quiescent		= sched_ext_ops__quiescent, +	.yield			= sched_ext_ops__yield, +	.core_sched_before	= sched_ext_ops__core_sched_before, +	.set_weight		= sched_ext_ops__set_weight, +	.set_cpumask		= sched_ext_ops__set_cpumask, +	.update_idle		= sched_ext_ops__update_idle, +	.cpu_acquire		= sched_ext_ops__cpu_acquire, +	.cpu_release		= sched_ext_ops__cpu_release, +	.init_task		= sched_ext_ops__init_task, +	.exit_task		= sched_ext_ops__exit_task, +	.enable			= sched_ext_ops__enable, +	.disable		= sched_ext_ops__disable,  #ifdef CONFIG_EXT_GROUP_SCHED -	.cgroup_init = cgroup_init_stub, -	.cgroup_exit = cgroup_exit_stub, -	.cgroup_prep_move = cgroup_prep_move_stub, -	.cgroup_move = cgroup_move_stub, -	.cgroup_cancel_move = cgroup_cancel_move_stub, -	.cgroup_set_weight = cgroup_set_weight_stub, +	.cgroup_init		= sched_ext_ops__cgroup_init, +	.cgroup_exit		= sched_ext_ops__cgroup_exit, +	.cgroup_prep_move	= sched_ext_ops__cgroup_prep_move, +	.cgroup_move		= sched_ext_ops__cgroup_move, +	.cgroup_cancel_move	= sched_ext_ops__cgroup_cancel_move, +	.cgroup_set_weight	= sched_ext_ops__cgroup_set_weight,  #endif -	.cpu_online = cpu_online_stub, -	.cpu_offline = cpu_offline_stub, -	.init = init_stub, -	.exit = exit_stub, -	.dump = dump_stub, -	.dump_cpu = dump_cpu_stub, -	.dump_task = dump_task_stub, +	.cpu_online		= sched_ext_ops__cpu_online, +	.cpu_offline		= sched_ext_ops__cpu_offline, +	.init			= sched_ext_ops__init, +	.exit			= sched_ext_ops__exit, +	.dump			= sched_ext_ops__dump, +	.dump_cpu		= sched_ext_ops__dump_cpu, +	.dump_task		= sched_ext_ops__dump_task,  };  static struct bpf_struct_ops bpf_sched_ext_ops = { @@ -5759,7 +6096,7 @@ static void kick_cpus_irq_workfn(struct irq_work *irq_work)  		if (cpu != cpu_of(this_rq)) {  			/*  			 * Pairs with smp_store_release() issued by this CPU in -			 * scx_next_task_picked() on the resched path. +			 * switch_class() on the resched path.  			 *  			 * We busy-wait here to guarantee that no other task can  			 * be scheduled on our core before the target CPU has @@ -5944,7 +6281,7 @@ static const struct btf_kfunc_id_set scx_kfunc_set_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))  		return false; @@ -5964,7 +6301,8 @@ static bool scx_dispatch_preamble(struct task_struct *p, u64 enq_flags)  	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 task_struct *ddsp_task; @@ -5991,14 +6329,14 @@ static void scx_dispatch_commit(struct task_struct *p, u64 dsq_id, u64 enq_flags  __bpf_kfunc_start_defs();  /** - * scx_bpf_dispatch - Dispatch a task into the FIFO queue of a DSQ - * @p: task_struct to dispatch - * @dsq_id: DSQ to dispatch to + * scx_bpf_dsq_insert - Insert a task into the FIFO queue of a DSQ + * @p: task_struct to insert + * @dsq_id: DSQ to insert into   * @slice: duration @p can run for in nsecs, 0 to keep the current value   * @enq_flags: SCX_ENQ_*   * - * Dispatch @p into the FIFO queue of the DSQ identified by @dsq_id. It is safe - * to call this function spuriously. Can be called from ops.enqueue(), + * Insert @p into the FIFO queue of the DSQ identified by @dsq_id. It is safe to + * call this function spuriously. Can be called from ops.enqueue(),   * ops.select_cpu(), and ops.dispatch().   *   * When called from ops.select_cpu() or ops.enqueue(), it's for direct dispatch @@ -6007,14 +6345,14 @@ __bpf_kfunc_start_defs();   * 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 - * will be directly dispatched to the corresponding dispatch queue after - * ops.select_cpu() returns. If @p is dispatched to SCX_DSQ_LOCAL, it will be - * dispatched to the local DSQ of the CPU returned by ops.select_cpu(). + * will be directly inserted into the corresponding dispatch queue after + * ops.select_cpu() returns. If @p is inserted into SCX_DSQ_LOCAL, it will be + * 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 - * task is dispatched. + * task is inserted.   *   * 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   * remaining slots. scx_bpf_consume() flushes the batch and resets the counter.   * @@ -6026,10 +6364,10 @@ __bpf_kfunc_start_defs();   * %SCX_SLICE_INF, @p never expires and the BPF scheduler must kick the CPU with   * scx_bpf_kick_cpu() to trigger scheduling.   */ -__bpf_kfunc void scx_bpf_dispatch(struct task_struct *p, u64 dsq_id, u64 slice, -				  u64 enq_flags) +__bpf_kfunc void scx_bpf_dsq_insert(struct task_struct *p, u64 dsq_id, u64 slice, +				    u64 enq_flags)  { -	if (!scx_dispatch_preamble(p, enq_flags)) +	if (!scx_dsq_insert_preamble(p, enq_flags))  		return;  	if (slice) @@ -6037,30 +6375,42 @@ __bpf_kfunc void scx_bpf_dispatch(struct task_struct *p, u64 dsq_id, u64 slice,  	else  		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 - * @p: task_struct to dispatch - * @dsq_id: DSQ to dispatch to + * scx_bpf_dsq_insert_vtime - Insert a task into the vtime priority queue of a DSQ + * @p: task_struct to insert + * @dsq_id: DSQ to insert into   * @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   * @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(). + * Insert @p into the vtime priority queue of the DSQ identified by @dsq_id. + * 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_dispatch_vtime(struct task_struct *p, u64 dsq_id, -					u64 slice, u64 vtime, u64 enq_flags) +__bpf_kfunc void scx_bpf_dsq_insert_vtime(struct task_struct *p, u64 dsq_id, +					  u64 slice, u64 vtime, u64 enq_flags)  { -	if (!scx_dispatch_preamble(p, enq_flags)) +	if (!scx_dsq_insert_preamble(p, enq_flags))  		return;  	if (slice) @@ -6070,12 +6420,22 @@ __bpf_kfunc void scx_bpf_dispatch_vtime(struct task_struct *p, u64 dsq_id,  	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();  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_vtime, KF_RCU)  BTF_KFUNCS_END(scx_kfunc_ids_enqueue_dispatch) @@ -6085,12 +6445,11 @@ static const struct btf_kfunc_id_set scx_kfunc_set_enqueue_dispatch = {  	.set			= &scx_kfunc_ids_enqueue_dispatch,  }; -static bool scx_dispatch_from_dsq(struct bpf_iter_scx_dsq_kern *kit, -				  struct task_struct *p, u64 dsq_id, -				  u64 enq_flags) +static bool scx_dsq_move(struct bpf_iter_scx_dsq_kern *kit, +			 struct task_struct *p, u64 dsq_id, u64 enq_flags)  {  	struct scx_dispatch_q *src_dsq = kit->dsq, *dst_dsq; -	struct rq *this_rq, *src_rq, *dst_rq, *locked_rq; +	struct rq *this_rq, *src_rq, *locked_rq;  	bool dispatched = false;  	bool in_balance;  	unsigned long flags; @@ -6118,6 +6477,13 @@ static bool scx_dispatch_from_dsq(struct bpf_iter_scx_dsq_kern *kit,  		raw_spin_rq_lock(src_rq);  	} +	/* +	 * If the BPF scheduler keeps calling this function repeatedly, it can +	 * cause similar live-lock conditions as consume_dispatch_q(). Insert a +	 * breather if necessary. +	 */ +	scx_ops_breather(src_rq); +  	locked_rq = src_rq;  	raw_spin_lock(&src_dsq->lock); @@ -6136,51 +6502,18 @@ static bool scx_dispatch_from_dsq(struct bpf_iter_scx_dsq_kern *kit,  	/* @p is still on $src_dsq and stable, determine the destination */  	dst_dsq = find_dsq_for_dispatch(this_rq, dsq_id, p); -	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 = find_global_dsq(p); -			dst_rq = src_rq; -		} -	} else { -		/* no need to migrate if destination is a non-local DSQ */ -		dst_rq = src_rq; -	} -  	/* -	 * Move @p into $dst_dsq. If $dst_dsq is the local DSQ of a different -	 * CPU, @p will be migrated. +	 * Apply vtime and slice updates before moving so that the new time is +	 * visible before inserting into $dst_dsq. @p is still on $src_dsq but +	 * this is safe as we're locking it.  	 */ -	if (dst_dsq->id == SCX_DSQ_LOCAL) { -		/* @p is going from a non-local DSQ to a local DSQ */ -		if (src_rq == dst_rq) { -			task_unlink_from_dsq(p, src_dsq); -			move_local_task_to_local_dsq(p, enq_flags, -						     src_dsq, dst_rq); -			raw_spin_unlock(&src_dsq->lock); -		} else { -			raw_spin_unlock(&src_dsq->lock); -			move_remote_task_to_local_dsq(p, enq_flags, -						      src_rq, dst_rq); -			locked_rq = dst_rq; -		} -	} else { -		/* -		 * @p is going from a non-local DSQ to a non-local DSQ. As -		 * $src_dsq is already locked, do an abbreviated dequeue. -		 */ -		task_unlink_from_dsq(p, src_dsq); -		p->scx.dsq = NULL; -		raw_spin_unlock(&src_dsq->lock); - -		if (kit->cursor.flags & __SCX_DSQ_ITER_HAS_VTIME) -			p->scx.dsq_vtime = kit->vtime; -		dispatch_enqueue(dst_dsq, p, enq_flags); -	} - +	if (kit->cursor.flags & __SCX_DSQ_ITER_HAS_VTIME) +		p->scx.dsq_vtime = kit->vtime;  	if (kit->cursor.flags & __SCX_DSQ_ITER_HAS_SLICE)  		p->scx.slice = kit->slice; +	/* execute move */ +	locked_rq = move_task_between_dsqs(p, enq_flags, src_dsq, dst_dsq);  	dispatched = true;  out:  	if (in_balance) { @@ -6232,21 +6565,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 - * @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_dispatch() before - * trying to consume the specified DSQ. It may also grab rq locks and thus can't - * be called under any BPF locks. + * This function flushes the in-flight dispatches from scx_bpf_dsq_insert() + * 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; @@ -6276,17 +6608,24 @@ __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_dsq_move_set_slice - Override slice when moving between DSQs   * @it__iter: DSQ iterator in progress - * @slice: duration the dispatched task can run for in nsecs + * @slice: duration the moved task can run for in nsecs   * - * Override the slice of the next task that will be dispatched from @it__iter - * using scx_bpf_dispatch_from_dsq[_vtime](). If this function is not called, - * the previous slice duration is kept. + * Override the slice of the next task that will be moved from @it__iter using + * scx_bpf_dsq_move[_vtime](). If this function is not called, the previous + * slice duration is kept.   */ -__bpf_kfunc void scx_bpf_dispatch_from_dsq_set_slice( -				struct bpf_iter_scx_dsq *it__iter, u64 slice) +__bpf_kfunc void scx_bpf_dsq_move_set_slice(struct bpf_iter_scx_dsq *it__iter, +					    u64 slice)  {  	struct bpf_iter_scx_dsq_kern *kit = (void *)it__iter; @@ -6294,18 +6633,26 @@ __bpf_kfunc void scx_bpf_dispatch_from_dsq_set_slice(  	kit->cursor.flags |= __SCX_DSQ_ITER_HAS_SLICE;  } +/* for backward compatibility, will be removed in v6.15 */ +__bpf_kfunc void scx_bpf_dispatch_from_dsq_set_slice( +			struct bpf_iter_scx_dsq *it__iter, u64 slice) +{ +	printk_deferred_once(KERN_WARNING "sched_ext: scx_bpf_dispatch_from_dsq_set_slice() renamed to scx_bpf_dsq_move_set_slice()"); +	scx_bpf_dsq_move_set_slice(it__iter, slice); +} +  /** - * scx_bpf_dispatch_from_dsq_set_vtime - Override vtime when dispatching from DSQ + * scx_bpf_dsq_move_set_vtime - Override vtime when moving between DSQs   * @it__iter: DSQ iterator in progress   * @vtime: task's ordering inside the vtime-sorted queue of the target DSQ   * - * Override the vtime of the next task that will be dispatched from @it__iter - * using scx_bpf_dispatch_from_dsq_vtime(). If this function is not called, the - * previous slice vtime is kept. If scx_bpf_dispatch_from_dsq() is used to - * dispatch the next task, the override is ignored and cleared. + * Override the vtime of the next task that will be moved from @it__iter using + * scx_bpf_dsq_move_vtime(). If this function is not called, the previous slice + * vtime is kept. If scx_bpf_dsq_move() is used to dispatch the next task, the + * override is ignored and cleared.   */ -__bpf_kfunc void scx_bpf_dispatch_from_dsq_set_vtime( -				struct bpf_iter_scx_dsq *it__iter, u64 vtime) +__bpf_kfunc void scx_bpf_dsq_move_set_vtime(struct bpf_iter_scx_dsq *it__iter, +					    u64 vtime)  {  	struct bpf_iter_scx_dsq_kern *kit = (void *)it__iter; @@ -6313,8 +6660,16 @@ __bpf_kfunc void scx_bpf_dispatch_from_dsq_set_vtime(  	kit->cursor.flags |= __SCX_DSQ_ITER_HAS_VTIME;  } +/* for backward compatibility, will be removed in v6.15 */ +__bpf_kfunc void scx_bpf_dispatch_from_dsq_set_vtime( +			struct bpf_iter_scx_dsq *it__iter, u64 vtime) +{ +	printk_deferred_once(KERN_WARNING "sched_ext: scx_bpf_dispatch_from_dsq_set_vtime() renamed to scx_bpf_dsq_move_set_vtime()"); +	scx_bpf_dsq_move_set_vtime(it__iter, vtime); +} +  /** - * scx_bpf_dispatch_from_dsq - Move a task from DSQ iteration to a DSQ + * scx_bpf_dsq_move - Move a task from DSQ iteration to a DSQ   * @it__iter: DSQ iterator in progress   * @p: task to transfer   * @dsq_id: DSQ to move @p to @@ -6329,8 +6684,7 @@ __bpf_kfunc void scx_bpf_dispatch_from_dsq_set_vtime(   * @p was obtained from the DSQ iteration. @p just has to be on the DSQ and have   * been queued before the iteration started.   * - * @p's slice is kept by default. Use scx_bpf_dispatch_from_dsq_set_slice() to - * update. + * @p's slice is kept by default. Use scx_bpf_dsq_move_set_slice() to update.   *   * Can be called from ops.dispatch() or any BPF context which doesn't hold a rq   * lock (e.g. BPF timers or SYSCALL programs). @@ -6338,16 +6692,25 @@ __bpf_kfunc void scx_bpf_dispatch_from_dsq_set_vtime(   * Returns %true if @p has been consumed, %false if @p had already been consumed   * or dequeued.   */ +__bpf_kfunc bool scx_bpf_dsq_move(struct bpf_iter_scx_dsq *it__iter, +				  struct task_struct *p, u64 dsq_id, +				  u64 enq_flags) +{ +	return scx_dsq_move((struct bpf_iter_scx_dsq_kern *)it__iter, +			    p, dsq_id, enq_flags); +} + +/* for backward compatibility, will be removed in v6.15 */  __bpf_kfunc bool scx_bpf_dispatch_from_dsq(struct bpf_iter_scx_dsq *it__iter,  					   struct task_struct *p, u64 dsq_id,  					   u64 enq_flags)  { -	return scx_dispatch_from_dsq((struct bpf_iter_scx_dsq_kern *)it__iter, -				     p, dsq_id, enq_flags); +	printk_deferred_once(KERN_WARNING "sched_ext: scx_bpf_dispatch_from_dsq() renamed to scx_bpf_dsq_move()"); +	return scx_bpf_dsq_move(it__iter, p, dsq_id, enq_flags);  }  /** - * scx_bpf_dispatch_vtime_from_dsq - Move a task from DSQ iteration to a PRIQ DSQ + * scx_bpf_dsq_move_vtime - Move a task from DSQ iteration to a PRIQ DSQ   * @it__iter: DSQ iterator in progress   * @p: task to transfer   * @dsq_id: DSQ to move @p to @@ -6357,19 +6720,27 @@ __bpf_kfunc bool scx_bpf_dispatch_from_dsq(struct bpf_iter_scx_dsq *it__iter,   * priority queue of the DSQ specified by @dsq_id. The destination must be a   * user DSQ as only user DSQs support priority queue.   * - * @p's slice and vtime are kept by default. Use - * scx_bpf_dispatch_from_dsq_set_slice() and - * scx_bpf_dispatch_from_dsq_set_vtime() to update. + * @p's slice and vtime are kept by default. Use scx_bpf_dsq_move_set_slice() + * and scx_bpf_dsq_move_set_vtime() to update.   * - * All other aspects are identical to scx_bpf_dispatch_from_dsq(). See - * scx_bpf_dispatch_vtime() for more information on @vtime. + * All other aspects are identical to scx_bpf_dsq_move(). See + * scx_bpf_dsq_insert_vtime() for more information on @vtime.   */ +__bpf_kfunc bool scx_bpf_dsq_move_vtime(struct bpf_iter_scx_dsq *it__iter, +					struct task_struct *p, u64 dsq_id, +					u64 enq_flags) +{ +	return scx_dsq_move((struct bpf_iter_scx_dsq_kern *)it__iter, +			    p, dsq_id, enq_flags | SCX_ENQ_DSQ_PRIQ); +} + +/* for backward compatibility, will be removed in v6.15 */  __bpf_kfunc bool scx_bpf_dispatch_vtime_from_dsq(struct bpf_iter_scx_dsq *it__iter,  						 struct task_struct *p, u64 dsq_id,  						 u64 enq_flags)  { -	return scx_dispatch_from_dsq((struct bpf_iter_scx_dsq_kern *)it__iter, -				     p, dsq_id, enq_flags | SCX_ENQ_DSQ_PRIQ); +	printk_deferred_once(KERN_WARNING "sched_ext: scx_bpf_dispatch_from_dsq_vtime() renamed to scx_bpf_dsq_move_vtime()"); +	return scx_bpf_dsq_move_vtime(it__iter, p, dsq_id, enq_flags);  }  __bpf_kfunc_end_defs(); @@ -6377,7 +6748,12 @@ __bpf_kfunc_end_defs();  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_dsq_move_set_slice) +BTF_ID_FLAGS(func, scx_bpf_dsq_move_set_vtime) +BTF_ID_FLAGS(func, scx_bpf_dsq_move, KF_RCU) +BTF_ID_FLAGS(func, scx_bpf_dsq_move_vtime, KF_RCU)  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, KF_RCU) @@ -6478,6 +6854,12 @@ __bpf_kfunc_end_defs();  BTF_KFUNCS_START(scx_kfunc_ids_unlocked)  BTF_ID_FLAGS(func, scx_bpf_create_dsq, KF_SLEEPABLE) +BTF_ID_FLAGS(func, scx_bpf_dsq_move_set_slice) +BTF_ID_FLAGS(func, scx_bpf_dsq_move_set_vtime) +BTF_ID_FLAGS(func, scx_bpf_dsq_move, KF_RCU) +BTF_ID_FLAGS(func, scx_bpf_dsq_move_vtime, KF_RCU) +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, KF_RCU)  BTF_ID_FLAGS(func, scx_bpf_dispatch_vtime_from_dsq, KF_RCU)  BTF_KFUNCS_END(scx_kfunc_ids_unlocked) @@ -7153,15 +7535,8 @@ __bpf_kfunc struct cgroup *scx_bpf_task_cgroup(struct task_struct *p)  	if (!scx_kf_allowed_on_arg_tasks(__SCX_KF_RQ_LOCKED, p))  		goto out; -	/* -	 * A task_group may either be a cgroup or an autogroup. In the latter -	 * case, @tg->css.cgroup is %NULL. A task_group can't become the other -	 * kind once created. -	 */ -	if (tg && tg->css.cgroup) -		cgrp = tg->css.cgroup; -	else -		cgrp = &cgrp_dfl_root.cgrp; +	cgrp = tg_cgrp(tg); +  out:  	cgroup_get(cgrp);  	return cgrp; | 
