mirror of
				https://github.com/torvalds/linux.git
				synced 2025-11-04 10:40:15 +02:00 
			
		
		
		
	sched_ext: Allow SCX_DSQ_LOCAL_ON for direct dispatches
In ops.dispatch(), SCX_DSQ_LOCAL_ON can be used to dispatch the task to the
local DSQ of any CPU. However, during direct dispatch from ops.select_cpu()
and ops.enqueue(), this isn't allowed. This is because dispatching to the
local DSQ of a remote CPU requires locking both the task's current and new
rq's and such double locking can't be done directly from ops.enqueue().
While waking up a task, as ops.select_cpu() can pick any CPU and both
ops.select_cpu() and ops.enqueue() can use SCX_DSQ_LOCAL as the dispatch
target to dispatch to the DSQ of the picked CPU, the BPF scheduler can still
do whatever it wants to do. However, while a task is being enqueued for a
different reason, e.g. after its slice expiration, only ops.enqueue() is
called and there's no way for the BPF scheduler to directly dispatch to the
local DSQ of a remote CPU. This gap in API forces schedulers into
work-arounds which are not straightforward or optimal such as skipping
direct dispatches in such cases.
Implement deferred enqueueing to allow directly dispatching to the local DSQ
of a remote CPU from ops.select_cpu() and ops.enqueue(). Such tasks are
temporarily queued on rq->scx.ddsp_deferred_locals. When the rq lock can be
safely released, the tasks are taken off the list and queued on the target
local DSQs using dispatch_to_local_dsq().
v2: - Add missing return after queue_balance_callback() in
      schedule_deferred(). (David).
    - dispatch_to_local_dsq() now assumes that @rq is locked but unpinned
      and thus no longer takes @rf. Updated accordingly.
    - UP build warning fix.
Signed-off-by: Tejun Heo <tj@kernel.org>
Tested-by: Andrea Righi <righi.andrea@gmail.com>
Acked-by: David Vernet <void@manifault.com>
Cc: Dan Schatzberg <schatzberg.dan@gmail.com>
Cc: Changwoo Min <changwoo@igalia.com>
			
			
This commit is contained in:
		
							parent
							
								
									f47a818950
								
							
						
					
					
						commit
						5b26f7b920
					
				
					 2 changed files with 153 additions and 18 deletions
				
			
		| 
						 | 
					@ -888,6 +888,7 @@ static struct kobject *scx_root_kobj;
 | 
				
			||||||
#define CREATE_TRACE_POINTS
 | 
					#define CREATE_TRACE_POINTS
 | 
				
			||||||
#include <trace/events/sched_ext.h>
 | 
					#include <trace/events/sched_ext.h>
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					static void process_ddsp_deferred_locals(struct rq *rq);
 | 
				
			||||||
static void scx_bpf_kick_cpu(s32 cpu, u64 flags);
 | 
					static void scx_bpf_kick_cpu(s32 cpu, u64 flags);
 | 
				
			||||||
static __printf(3, 4) void scx_ops_exit_kind(enum scx_exit_kind kind,
 | 
					static __printf(3, 4) void scx_ops_exit_kind(enum scx_exit_kind kind,
 | 
				
			||||||
					     s64 exit_code,
 | 
										     s64 exit_code,
 | 
				
			||||||
| 
						 | 
					@ -1362,6 +1363,67 @@ static int ops_sanitize_err(const char *ops_name, s32 err)
 | 
				
			||||||
	return -EPROTO;
 | 
						return -EPROTO;
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					static void run_deferred(struct rq *rq)
 | 
				
			||||||
 | 
					{
 | 
				
			||||||
 | 
						process_ddsp_deferred_locals(rq);
 | 
				
			||||||
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					#ifdef CONFIG_SMP
 | 
				
			||||||
 | 
					static void deferred_bal_cb_workfn(struct rq *rq)
 | 
				
			||||||
 | 
					{
 | 
				
			||||||
 | 
						run_deferred(rq);
 | 
				
			||||||
 | 
					}
 | 
				
			||||||
 | 
					#endif
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					static void deferred_irq_workfn(struct irq_work *irq_work)
 | 
				
			||||||
 | 
					{
 | 
				
			||||||
 | 
						struct rq *rq = container_of(irq_work, struct rq, scx.deferred_irq_work);
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
						raw_spin_rq_lock(rq);
 | 
				
			||||||
 | 
						run_deferred(rq);
 | 
				
			||||||
 | 
						raw_spin_rq_unlock(rq);
 | 
				
			||||||
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					/**
 | 
				
			||||||
 | 
					 * schedule_deferred - Schedule execution of deferred actions on an rq
 | 
				
			||||||
 | 
					 * @rq: target rq
 | 
				
			||||||
 | 
					 *
 | 
				
			||||||
 | 
					 * Schedule execution of deferred actions on @rq. Must be called with @rq
 | 
				
			||||||
 | 
					 * locked. Deferred actions are executed with @rq locked but unpinned, and thus
 | 
				
			||||||
 | 
					 * can unlock @rq to e.g. migrate tasks to other rqs.
 | 
				
			||||||
 | 
					 */
 | 
				
			||||||
 | 
					static void schedule_deferred(struct rq *rq)
 | 
				
			||||||
 | 
					{
 | 
				
			||||||
 | 
						lockdep_assert_rq_held(rq);
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					#ifdef CONFIG_SMP
 | 
				
			||||||
 | 
						/*
 | 
				
			||||||
 | 
						 * If in the middle of waking up a task, task_woken_scx() will be called
 | 
				
			||||||
 | 
						 * afterwards which will then run the deferred actions, no need to
 | 
				
			||||||
 | 
						 * schedule anything.
 | 
				
			||||||
 | 
						 */
 | 
				
			||||||
 | 
						if (rq->scx.flags & SCX_RQ_IN_WAKEUP)
 | 
				
			||||||
 | 
							return;
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
						/*
 | 
				
			||||||
 | 
						 * If in balance, the balance callbacks will be called before rq lock is
 | 
				
			||||||
 | 
						 * released. Schedule one.
 | 
				
			||||||
 | 
						 */
 | 
				
			||||||
 | 
						if (rq->scx.flags & SCX_RQ_IN_BALANCE) {
 | 
				
			||||||
 | 
							queue_balance_callback(rq, &rq->scx.deferred_bal_cb,
 | 
				
			||||||
 | 
									       deferred_bal_cb_workfn);
 | 
				
			||||||
 | 
							return;
 | 
				
			||||||
 | 
						}
 | 
				
			||||||
 | 
					#endif
 | 
				
			||||||
 | 
						/*
 | 
				
			||||||
 | 
						 * No scheduler hooks available. Queue an irq work. They are executed on
 | 
				
			||||||
 | 
						 * IRQ re-enable which may take a bit longer than the scheduler hooks.
 | 
				
			||||||
 | 
						 * The above WAKEUP and BALANCE paths should cover most of the cases and
 | 
				
			||||||
 | 
						 * the time to IRQ re-enable shouldn't be long.
 | 
				
			||||||
 | 
						 */
 | 
				
			||||||
 | 
						irq_work_queue(&rq->scx.deferred_irq_work);
 | 
				
			||||||
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
/**
 | 
					/**
 | 
				
			||||||
 * touch_core_sched - Update timestamp used for core-sched task ordering
 | 
					 * touch_core_sched - Update timestamp used for core-sched task ordering
 | 
				
			||||||
 * @rq: rq to read clock from, must be locked
 | 
					 * @rq: rq to read clock from, must be locked
 | 
				
			||||||
| 
						 | 
					@ -1577,7 +1639,13 @@ static void dispatch_dequeue(struct rq *rq, struct task_struct *p)
 | 
				
			||||||
	bool is_local = dsq == &rq->scx.local_dsq;
 | 
						bool is_local = dsq == &rq->scx.local_dsq;
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	if (!dsq) {
 | 
						if (!dsq) {
 | 
				
			||||||
		WARN_ON_ONCE(!list_empty(&p->scx.dsq_list.node));
 | 
							/*
 | 
				
			||||||
 | 
							 * If !dsq && on-list, @p is on @rq's ddsp_deferred_locals.
 | 
				
			||||||
 | 
							 * Unlinking is all that's needed to cancel.
 | 
				
			||||||
 | 
							 */
 | 
				
			||||||
 | 
							if (unlikely(!list_empty(&p->scx.dsq_list.node)))
 | 
				
			||||||
 | 
								list_del_init(&p->scx.dsq_list.node);
 | 
				
			||||||
 | 
					
 | 
				
			||||||
		/*
 | 
							/*
 | 
				
			||||||
		 * When dispatching directly from the BPF scheduler to a local
 | 
							 * When dispatching directly from the BPF scheduler to a local
 | 
				
			||||||
		 * DSQ, the task isn't associated with any DSQ but
 | 
							 * DSQ, the task isn't associated with any DSQ but
 | 
				
			||||||
| 
						 | 
					@ -1586,6 +1654,7 @@ static void dispatch_dequeue(struct rq *rq, struct task_struct *p)
 | 
				
			||||||
		 */
 | 
							 */
 | 
				
			||||||
		if (p->scx.holding_cpu >= 0)
 | 
							if (p->scx.holding_cpu >= 0)
 | 
				
			||||||
			p->scx.holding_cpu = -1;
 | 
								p->scx.holding_cpu = -1;
 | 
				
			||||||
 | 
					
 | 
				
			||||||
		return;
 | 
							return;
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
| 
						 | 
					@ -1673,17 +1742,6 @@ static void mark_direct_dispatch(struct task_struct *ddsp_task,
 | 
				
			||||||
		return;
 | 
							return;
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	/*
 | 
					 | 
				
			||||||
	 * %SCX_DSQ_LOCAL_ON is not supported during direct dispatch because
 | 
					 | 
				
			||||||
	 * dispatching to the local DSQ of a different CPU requires unlocking
 | 
					 | 
				
			||||||
	 * the current rq which isn't allowed in the enqueue path. Use
 | 
					 | 
				
			||||||
	 * ops.select_cpu() to be on the target CPU and then %SCX_DSQ_LOCAL.
 | 
					 | 
				
			||||||
	 */
 | 
					 | 
				
			||||||
	if (unlikely((dsq_id & SCX_DSQ_LOCAL_ON) == SCX_DSQ_LOCAL_ON)) {
 | 
					 | 
				
			||||||
		scx_ops_error("SCX_DSQ_LOCAL_ON can't be used for direct-dispatch");
 | 
					 | 
				
			||||||
		return;
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
	WARN_ON_ONCE(p->scx.ddsp_dsq_id != SCX_DSQ_INVALID);
 | 
						WARN_ON_ONCE(p->scx.ddsp_dsq_id != SCX_DSQ_INVALID);
 | 
				
			||||||
	WARN_ON_ONCE(p->scx.ddsp_enq_flags);
 | 
						WARN_ON_ONCE(p->scx.ddsp_enq_flags);
 | 
				
			||||||
 | 
					
 | 
				
			||||||
| 
						 | 
					@ -1693,13 +1751,58 @@ static void mark_direct_dispatch(struct task_struct *ddsp_task,
 | 
				
			||||||
 | 
					
 | 
				
			||||||
static void direct_dispatch(struct task_struct *p, u64 enq_flags)
 | 
					static void direct_dispatch(struct task_struct *p, u64 enq_flags)
 | 
				
			||||||
{
 | 
					{
 | 
				
			||||||
 | 
						struct rq *rq = task_rq(p);
 | 
				
			||||||
	struct scx_dispatch_q *dsq;
 | 
						struct scx_dispatch_q *dsq;
 | 
				
			||||||
 | 
						u64 dsq_id = p->scx.ddsp_dsq_id;
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	touch_core_sched_dispatch(task_rq(p), p);
 | 
						touch_core_sched_dispatch(rq, p);
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	enq_flags |= (p->scx.ddsp_enq_flags | SCX_ENQ_CLEAR_OPSS);
 | 
						p->scx.ddsp_enq_flags |= enq_flags;
 | 
				
			||||||
	dsq = find_dsq_for_dispatch(task_rq(p), p->scx.ddsp_dsq_id, p);
 | 
					
 | 
				
			||||||
	dispatch_enqueue(dsq, p, enq_flags);
 | 
						/*
 | 
				
			||||||
 | 
						 * We are in the enqueue path with @rq locked and pinned, and thus can't
 | 
				
			||||||
 | 
						 * double lock a remote rq and enqueue to its local DSQ. For
 | 
				
			||||||
 | 
						 * DSQ_LOCAL_ON verdicts targeting the local DSQ of a remote CPU, defer
 | 
				
			||||||
 | 
						 * the enqueue so that it's executed when @rq can be unlocked.
 | 
				
			||||||
 | 
						 */
 | 
				
			||||||
 | 
						if ((dsq_id & SCX_DSQ_LOCAL_ON) == SCX_DSQ_LOCAL_ON) {
 | 
				
			||||||
 | 
							s32 cpu = dsq_id & SCX_DSQ_LOCAL_CPU_MASK;
 | 
				
			||||||
 | 
							unsigned long opss;
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
							if (cpu == cpu_of(rq)) {
 | 
				
			||||||
 | 
								dsq_id = SCX_DSQ_LOCAL;
 | 
				
			||||||
 | 
								goto dispatch;
 | 
				
			||||||
 | 
							}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
							opss = atomic_long_read(&p->scx.ops_state) & SCX_OPSS_STATE_MASK;
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
							switch (opss & SCX_OPSS_STATE_MASK) {
 | 
				
			||||||
 | 
							case SCX_OPSS_NONE:
 | 
				
			||||||
 | 
								break;
 | 
				
			||||||
 | 
							case SCX_OPSS_QUEUEING:
 | 
				
			||||||
 | 
								/*
 | 
				
			||||||
 | 
								 * As @p was never passed to the BPF side, _release is
 | 
				
			||||||
 | 
								 * not strictly necessary. Still do it for consistency.
 | 
				
			||||||
 | 
								 */
 | 
				
			||||||
 | 
								atomic_long_set_release(&p->scx.ops_state, SCX_OPSS_NONE);
 | 
				
			||||||
 | 
								break;
 | 
				
			||||||
 | 
							default:
 | 
				
			||||||
 | 
								WARN_ONCE(true, "sched_ext: %s[%d] has invalid ops state 0x%lx in direct_dispatch()",
 | 
				
			||||||
 | 
									  p->comm, p->pid, opss);
 | 
				
			||||||
 | 
								atomic_long_set_release(&p->scx.ops_state, SCX_OPSS_NONE);
 | 
				
			||||||
 | 
								break;
 | 
				
			||||||
 | 
							}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
							WARN_ON_ONCE(p->scx.dsq || !list_empty(&p->scx.dsq_list.node));
 | 
				
			||||||
 | 
							list_add_tail(&p->scx.dsq_list.node,
 | 
				
			||||||
 | 
								      &rq->scx.ddsp_deferred_locals);
 | 
				
			||||||
 | 
							schedule_deferred(rq);
 | 
				
			||||||
 | 
							return;
 | 
				
			||||||
 | 
						}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					dispatch:
 | 
				
			||||||
 | 
						dsq = find_dsq_for_dispatch(rq, dsq_id, p);
 | 
				
			||||||
 | 
						dispatch_enqueue(dsq, p, p->scx.ddsp_enq_flags | SCX_ENQ_CLEAR_OPSS);
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
static bool scx_rq_online(struct rq *rq)
 | 
					static bool scx_rq_online(struct rq *rq)
 | 
				
			||||||
| 
						 | 
					@ -2601,6 +2704,29 @@ static void set_next_task_scx(struct rq *rq, struct task_struct *p, bool first)
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					static void process_ddsp_deferred_locals(struct rq *rq)
 | 
				
			||||||
 | 
					{
 | 
				
			||||||
 | 
						struct task_struct *p, *tmp;
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
						lockdep_assert_rq_held(rq);
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
						/*
 | 
				
			||||||
 | 
						 * Now that @rq can be unlocked, execute the deferred enqueueing of
 | 
				
			||||||
 | 
						 * tasks directly dispatched to the local DSQs of other CPUs. See
 | 
				
			||||||
 | 
						 * direct_dispatch().
 | 
				
			||||||
 | 
						 */
 | 
				
			||||||
 | 
						list_for_each_entry_safe(p, tmp, &rq->scx.ddsp_deferred_locals,
 | 
				
			||||||
 | 
									 scx.dsq_list.node) {
 | 
				
			||||||
 | 
							s32 ret;
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
							list_del_init(&p->scx.dsq_list.node);
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
							ret = dispatch_to_local_dsq(rq, p->scx.ddsp_dsq_id, p,
 | 
				
			||||||
 | 
										    p->scx.ddsp_enq_flags);
 | 
				
			||||||
 | 
							WARN_ON_ONCE(ret == DTL_NOT_LOCAL);
 | 
				
			||||||
 | 
						}
 | 
				
			||||||
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
static void put_prev_task_scx(struct rq *rq, struct task_struct *p)
 | 
					static void put_prev_task_scx(struct rq *rq, struct task_struct *p)
 | 
				
			||||||
{
 | 
					{
 | 
				
			||||||
#ifndef CONFIG_SMP
 | 
					#ifndef CONFIG_SMP
 | 
				
			||||||
| 
						 | 
					@ -3022,6 +3148,11 @@ static int select_task_rq_scx(struct task_struct *p, int prev_cpu, int wake_flag
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					static void task_woken_scx(struct rq *rq, struct task_struct *p)
 | 
				
			||||||
 | 
					{
 | 
				
			||||||
 | 
						run_deferred(rq);
 | 
				
			||||||
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
static void set_cpus_allowed_scx(struct task_struct *p,
 | 
					static void set_cpus_allowed_scx(struct task_struct *p,
 | 
				
			||||||
				 struct affinity_context *ac)
 | 
									 struct affinity_context *ac)
 | 
				
			||||||
{
 | 
					{
 | 
				
			||||||
| 
						 | 
					@ -3538,8 +3669,6 @@ bool scx_can_stop_tick(struct rq *rq)
 | 
				
			||||||
 *
 | 
					 *
 | 
				
			||||||
 * - task_fork/dead: We need fork/dead notifications for all tasks regardless of
 | 
					 * - task_fork/dead: We need fork/dead notifications for all tasks regardless of
 | 
				
			||||||
 *   their current sched_class. Call them directly from sched core instead.
 | 
					 *   their current sched_class. Call them directly from sched core instead.
 | 
				
			||||||
 *
 | 
					 | 
				
			||||||
 * - task_woken: Unnecessary.
 | 
					 | 
				
			||||||
 */
 | 
					 */
 | 
				
			||||||
DEFINE_SCHED_CLASS(ext) = {
 | 
					DEFINE_SCHED_CLASS(ext) = {
 | 
				
			||||||
	.enqueue_task		= enqueue_task_scx,
 | 
						.enqueue_task		= enqueue_task_scx,
 | 
				
			||||||
| 
						 | 
					@ -3559,6 +3688,7 @@ DEFINE_SCHED_CLASS(ext) = {
 | 
				
			||||||
#ifdef CONFIG_SMP
 | 
					#ifdef CONFIG_SMP
 | 
				
			||||||
	.balance		= balance_scx,
 | 
						.balance		= balance_scx,
 | 
				
			||||||
	.select_task_rq		= select_task_rq_scx,
 | 
						.select_task_rq		= select_task_rq_scx,
 | 
				
			||||||
 | 
						.task_woken		= task_woken_scx,
 | 
				
			||||||
	.set_cpus_allowed	= set_cpus_allowed_scx,
 | 
						.set_cpus_allowed	= set_cpus_allowed_scx,
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	.rq_online		= rq_online_scx,
 | 
						.rq_online		= rq_online_scx,
 | 
				
			||||||
| 
						 | 
					@ -5263,11 +5393,13 @@ void __init init_sched_ext_class(void)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
		init_dsq(&rq->scx.local_dsq, SCX_DSQ_LOCAL);
 | 
							init_dsq(&rq->scx.local_dsq, SCX_DSQ_LOCAL);
 | 
				
			||||||
		INIT_LIST_HEAD(&rq->scx.runnable_list);
 | 
							INIT_LIST_HEAD(&rq->scx.runnable_list);
 | 
				
			||||||
 | 
							INIT_LIST_HEAD(&rq->scx.ddsp_deferred_locals);
 | 
				
			||||||
 | 
					
 | 
				
			||||||
		BUG_ON(!zalloc_cpumask_var(&rq->scx.cpus_to_kick, GFP_KERNEL));
 | 
							BUG_ON(!zalloc_cpumask_var(&rq->scx.cpus_to_kick, GFP_KERNEL));
 | 
				
			||||||
		BUG_ON(!zalloc_cpumask_var(&rq->scx.cpus_to_kick_if_idle, GFP_KERNEL));
 | 
							BUG_ON(!zalloc_cpumask_var(&rq->scx.cpus_to_kick_if_idle, GFP_KERNEL));
 | 
				
			||||||
		BUG_ON(!zalloc_cpumask_var(&rq->scx.cpus_to_preempt, GFP_KERNEL));
 | 
							BUG_ON(!zalloc_cpumask_var(&rq->scx.cpus_to_preempt, GFP_KERNEL));
 | 
				
			||||||
		BUG_ON(!zalloc_cpumask_var(&rq->scx.cpus_to_wait, GFP_KERNEL));
 | 
							BUG_ON(!zalloc_cpumask_var(&rq->scx.cpus_to_wait, GFP_KERNEL));
 | 
				
			||||||
 | 
							init_irq_work(&rq->scx.deferred_irq_work, deferred_irq_workfn);
 | 
				
			||||||
		init_irq_work(&rq->scx.kick_cpus_irq_work, kick_cpus_irq_workfn);
 | 
							init_irq_work(&rq->scx.kick_cpus_irq_work, kick_cpus_irq_workfn);
 | 
				
			||||||
 | 
					
 | 
				
			||||||
		if (cpu_online(cpu))
 | 
							if (cpu_online(cpu))
 | 
				
			||||||
| 
						 | 
					
 | 
				
			||||||
| 
						 | 
					@ -746,6 +746,7 @@ enum scx_rq_flags {
 | 
				
			||||||
struct scx_rq {
 | 
					struct scx_rq {
 | 
				
			||||||
	struct scx_dispatch_q	local_dsq;
 | 
						struct scx_dispatch_q	local_dsq;
 | 
				
			||||||
	struct list_head	runnable_list;		/* runnable tasks on this rq */
 | 
						struct list_head	runnable_list;		/* runnable tasks on this rq */
 | 
				
			||||||
 | 
						struct list_head	ddsp_deferred_locals;	/* deferred ddsps from enq */
 | 
				
			||||||
	unsigned long		ops_qseq;
 | 
						unsigned long		ops_qseq;
 | 
				
			||||||
	u64			extra_enq_flags;	/* see move_task_to_local_dsq() */
 | 
						u64			extra_enq_flags;	/* see move_task_to_local_dsq() */
 | 
				
			||||||
	u32			nr_running;
 | 
						u32			nr_running;
 | 
				
			||||||
| 
						 | 
					@ -757,6 +758,8 @@ struct scx_rq {
 | 
				
			||||||
	cpumask_var_t		cpus_to_preempt;
 | 
						cpumask_var_t		cpus_to_preempt;
 | 
				
			||||||
	cpumask_var_t		cpus_to_wait;
 | 
						cpumask_var_t		cpus_to_wait;
 | 
				
			||||||
	unsigned long		pnt_seq;
 | 
						unsigned long		pnt_seq;
 | 
				
			||||||
 | 
						struct balance_callback	deferred_bal_cb;
 | 
				
			||||||
 | 
						struct irq_work		deferred_irq_work;
 | 
				
			||||||
	struct irq_work		kick_cpus_irq_work;
 | 
						struct irq_work		kick_cpus_irq_work;
 | 
				
			||||||
};
 | 
					};
 | 
				
			||||||
#endif /* CONFIG_SCHED_CLASS_EXT */
 | 
					#endif /* CONFIG_SCHED_CLASS_EXT */
 | 
				
			||||||
| 
						 | 
					
 | 
				
			||||||
		Loading…
	
		Reference in a new issue