mirror of
https://github.com/asterinas/asterinas.git
synced 2025-06-10 13:56:48 +00:00
Reduce the held time of scheduling policy lock
This commit is contained in:
parent
ea953209ff
commit
364d6af7c8
@ -1,6 +1,6 @@
|
||||
// SPDX-License-Identifier: MPL-2.0
|
||||
|
||||
use alloc::{collections::binary_heap::BinaryHeap, sync::Arc};
|
||||
use alloc::collections::binary_heap::BinaryHeap;
|
||||
use core::{
|
||||
cmp::{self, Reverse},
|
||||
sync::atomic::{AtomicU64, Ordering::*},
|
||||
@ -13,12 +13,9 @@ use ostd::{
|
||||
|
||||
use super::{
|
||||
time::{base_slice_clocks, min_period_clocks},
|
||||
CurrentRuntime, SchedAttr, SchedClassRq,
|
||||
};
|
||||
use crate::{
|
||||
sched::priority::{Nice, NiceRange},
|
||||
thread::Thread,
|
||||
CurrentRuntime, SchedAttr, SchedClassRq, SchedEntity,
|
||||
};
|
||||
use crate::sched::priority::{Nice, NiceRange};
|
||||
|
||||
const WEIGHT_0: u64 = 1024;
|
||||
pub const fn nice_to_weight(nice: Nice) -> u64 {
|
||||
@ -120,7 +117,7 @@ impl FairAttr {
|
||||
///
|
||||
/// This structure is used to provide the capability for keying in the
|
||||
/// run queue implemented by `BTreeSet` in the `FairClassRq`.
|
||||
struct FairQueueItem(Arc<Thread>);
|
||||
struct FairQueueItem(SchedEntity);
|
||||
|
||||
impl core::fmt::Debug for FairQueueItem {
|
||||
fn fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result {
|
||||
@ -130,7 +127,7 @@ impl core::fmt::Debug for FairQueueItem {
|
||||
|
||||
impl FairQueueItem {
|
||||
fn key(&self) -> u64 {
|
||||
self.0.sched_attr().fair.vruntime.load(Relaxed)
|
||||
self.0 .1.sched_attr().fair.vruntime.load(Relaxed)
|
||||
}
|
||||
}
|
||||
|
||||
@ -165,7 +162,7 @@ pub(super) struct FairClassRq {
|
||||
#[allow(unused)]
|
||||
cpu: CpuId,
|
||||
/// The ready-to-run threads.
|
||||
threads: BinaryHeap<Reverse<FairQueueItem>>,
|
||||
entities: BinaryHeap<Reverse<FairQueueItem>>,
|
||||
/// The minimum of vruntime in the run queue. Serves as the initial
|
||||
/// value of newly-enqueued threads.
|
||||
min_vruntime: u64,
|
||||
@ -176,7 +173,7 @@ impl FairClassRq {
|
||||
pub fn new(cpu: CpuId) -> Self {
|
||||
Self {
|
||||
cpu,
|
||||
threads: BinaryHeap::new(),
|
||||
entities: BinaryHeap::new(),
|
||||
min_vruntime: 0,
|
||||
total_weight: 0,
|
||||
}
|
||||
@ -201,13 +198,13 @@ impl FairClassRq {
|
||||
|
||||
// `+ 1` means including the current running thread.
|
||||
let period_single_cpu =
|
||||
(base_slice_clks * (self.threads.len() + 1) as u64).max(min_period_clks);
|
||||
(base_slice_clks * (self.entities.len() + 1) as u64).max(min_period_clks);
|
||||
period_single_cpu * u64::from((1 + num_cpus()).ilog2())
|
||||
}
|
||||
|
||||
/// The virtual time slice for each thread in the run queue, measured in vruntime clocks.
|
||||
fn vtime_slice(&self) -> u64 {
|
||||
self.period() / (self.threads.len() + 1) as u64
|
||||
self.period() / (self.entities.len() + 1) as u64
|
||||
}
|
||||
|
||||
/// The time slice for each thread in the run queue, measured in sched clocks.
|
||||
@ -217,8 +214,8 @@ impl FairClassRq {
|
||||
}
|
||||
|
||||
impl SchedClassRq for FairClassRq {
|
||||
fn enqueue(&mut self, thread: Arc<Thread>, flags: Option<EnqueueFlags>) {
|
||||
let fair_attr = &thread.sched_attr().fair;
|
||||
fn enqueue(&mut self, entity: SchedEntity, flags: Option<EnqueueFlags>) {
|
||||
let fair_attr = &entity.1.sched_attr().fair;
|
||||
let vruntime = match flags {
|
||||
Some(EnqueueFlags::Spawn) => self.min_vruntime + self.vtime_slice(),
|
||||
_ => self.min_vruntime,
|
||||
@ -226,23 +223,23 @@ impl SchedClassRq for FairClassRq {
|
||||
fair_attr.vruntime.fetch_max(vruntime, Relaxed);
|
||||
|
||||
self.total_weight += fair_attr.weight.load(Relaxed);
|
||||
self.threads.push(Reverse(FairQueueItem(thread)));
|
||||
self.entities.push(Reverse(FairQueueItem(entity)));
|
||||
}
|
||||
|
||||
fn len(&mut self) -> usize {
|
||||
self.threads.len()
|
||||
self.entities.len()
|
||||
}
|
||||
|
||||
fn is_empty(&mut self) -> bool {
|
||||
self.threads.is_empty()
|
||||
self.entities.is_empty()
|
||||
}
|
||||
|
||||
fn pick_next(&mut self) -> Option<Arc<Thread>> {
|
||||
let Reverse(FairQueueItem(thread)) = self.threads.pop()?;
|
||||
fn pick_next(&mut self) -> Option<SchedEntity> {
|
||||
let Reverse(FairQueueItem(entity)) = self.entities.pop()?;
|
||||
|
||||
self.total_weight -= thread.sched_attr().fair.weight.load(Relaxed);
|
||||
self.total_weight -= entity.1.sched_attr().fair.weight.load(Relaxed);
|
||||
|
||||
Some(thread)
|
||||
Some(entity)
|
||||
}
|
||||
|
||||
fn update_current(
|
||||
@ -255,7 +252,7 @@ impl SchedClassRq for FairClassRq {
|
||||
UpdateFlags::Yield => true,
|
||||
UpdateFlags::Tick | UpdateFlags::Wait => {
|
||||
let (vruntime, weight) = attr.fair.update_vruntime(rt.delta);
|
||||
self.min_vruntime = match self.threads.peek() {
|
||||
self.min_vruntime = match self.entities.peek() {
|
||||
Some(Reverse(leftmost)) => vruntime.min(leftmost.key()),
|
||||
None => vruntime,
|
||||
};
|
||||
|
@ -4,20 +4,20 @@ use super::*;
|
||||
|
||||
/// The per-cpu run queue for the IDLE scheduling class.
|
||||
///
|
||||
/// This run queue is used for the per-cpu idle thread, if any.
|
||||
/// This run queue is used for the per-cpu idle entity, if any.
|
||||
pub(super) struct IdleClassRq {
|
||||
thread: Option<Arc<Thread>>,
|
||||
entity: Option<SchedEntity>,
|
||||
}
|
||||
|
||||
impl IdleClassRq {
|
||||
pub fn new() -> Self {
|
||||
Self { thread: None }
|
||||
Self { entity: None }
|
||||
}
|
||||
}
|
||||
|
||||
impl core::fmt::Debug for IdleClassRq {
|
||||
fn fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result {
|
||||
if self.thread.is_some() {
|
||||
if self.entity.is_some() {
|
||||
write!(f, "Idle: occupied")?;
|
||||
} else {
|
||||
write!(f, "Idle: empty")?;
|
||||
@ -27,12 +27,12 @@ impl core::fmt::Debug for IdleClassRq {
|
||||
}
|
||||
|
||||
impl SchedClassRq for IdleClassRq {
|
||||
fn enqueue(&mut self, thread: Arc<Thread>, _: Option<EnqueueFlags>) {
|
||||
let ptr = Arc::as_ptr(&thread);
|
||||
if let Some(t) = self.thread.replace(thread)
|
||||
&& ptr != Arc::as_ptr(&t)
|
||||
fn enqueue(&mut self, entity: SchedEntity, _: Option<EnqueueFlags>) {
|
||||
let ptr = Arc::as_ptr(&entity.0);
|
||||
if let Some(t) = self.entity.replace(entity)
|
||||
&& ptr != Arc::as_ptr(&t.0)
|
||||
{
|
||||
panic!("Multiple `idle` threads spawned")
|
||||
panic!("Multiple `idle` entities spawned")
|
||||
}
|
||||
}
|
||||
|
||||
@ -41,15 +41,15 @@ impl SchedClassRq for IdleClassRq {
|
||||
}
|
||||
|
||||
fn is_empty(&mut self) -> bool {
|
||||
self.thread.is_none()
|
||||
self.entity.is_none()
|
||||
}
|
||||
|
||||
fn pick_next(&mut self) -> Option<Arc<Thread>> {
|
||||
self.thread.clone()
|
||||
fn pick_next(&mut self) -> Option<SchedEntity> {
|
||||
self.entity.clone()
|
||||
}
|
||||
|
||||
fn update_current(&mut self, _: &CurrentRuntime, _: &SchedAttr, _flags: UpdateFlags) -> bool {
|
||||
// Idle threads has the greatest priority value. They should always be preempted.
|
||||
// Idle entities has the greatest priority value. They should always be preempted.
|
||||
true
|
||||
}
|
||||
}
|
||||
|
@ -3,7 +3,13 @@
|
||||
#![warn(unused)]
|
||||
|
||||
use alloc::{boxed::Box, sync::Arc};
|
||||
use core::{fmt, sync::atomic::AtomicU64};
|
||||
use core::{
|
||||
fmt,
|
||||
sync::atomic::{
|
||||
AtomicU64,
|
||||
Ordering::{Relaxed, SeqCst},
|
||||
},
|
||||
};
|
||||
|
||||
use ostd::{
|
||||
cpu::{all_cpus, AtomicCpuSet, CpuId, PinCurrentCpu},
|
||||
@ -18,6 +24,7 @@ use ostd::{
|
||||
trap::disable_local,
|
||||
};
|
||||
|
||||
mod policy;
|
||||
mod time;
|
||||
|
||||
mod fair;
|
||||
@ -27,12 +34,16 @@ mod stop;
|
||||
|
||||
use ostd::arch::read_tsc as sched_clock;
|
||||
|
||||
use self::policy::SchedPolicyKind;
|
||||
pub use self::{policy::SchedPolicy, real_time::RealTimePolicy};
|
||||
use super::{
|
||||
priority::{Nice, Priority, RangedU8},
|
||||
priority::{Nice, RangedU8},
|
||||
stats::SchedulerStats,
|
||||
};
|
||||
use crate::thread::{AsThread, Thread};
|
||||
|
||||
type SchedEntity = (Arc<Task>, Arc<Thread>);
|
||||
|
||||
#[allow(unused)]
|
||||
pub fn init() {
|
||||
inject_scheduler(Box::leak(Box::new(ClassScheduler::new())));
|
||||
@ -53,7 +64,7 @@ struct PerCpuClassRqSet {
|
||||
real_time: real_time::RealTimeClassRq,
|
||||
fair: fair::FairClassRq,
|
||||
idle: idle::IdleClassRq,
|
||||
current: Option<(Arc<Task>, CurrentRuntime)>,
|
||||
current: Option<(SchedEntity, CurrentRuntime)>,
|
||||
}
|
||||
|
||||
/// Stores the runtime information of the current task.
|
||||
@ -89,7 +100,7 @@ impl CurrentRuntime {
|
||||
/// should implement this trait to function as expected.
|
||||
trait SchedClassRq: Send + fmt::Debug {
|
||||
/// Enqueues a task into the run queue.
|
||||
fn enqueue(&mut self, thread: Arc<Thread>, flags: Option<EnqueueFlags>);
|
||||
fn enqueue(&mut self, entity: SchedEntity, flags: Option<EnqueueFlags>);
|
||||
|
||||
/// Returns the number of threads in the run queue.
|
||||
fn len(&mut self) -> usize;
|
||||
@ -100,50 +111,20 @@ trait SchedClassRq: Send + fmt::Debug {
|
||||
}
|
||||
|
||||
/// Picks the next task for running.
|
||||
fn pick_next(&mut self) -> Option<Arc<Thread>>;
|
||||
fn pick_next(&mut self) -> Option<SchedEntity>;
|
||||
|
||||
/// Update the information of the current task.
|
||||
fn update_current(&mut self, rt: &CurrentRuntime, attr: &SchedAttr, flags: UpdateFlags)
|
||||
-> bool;
|
||||
}
|
||||
|
||||
pub use real_time::RealTimePolicy;
|
||||
|
||||
/// The User-chosen scheduling policy.
|
||||
///
|
||||
/// The scheduling policies are specified by the user, usually through its priority.
|
||||
#[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord)]
|
||||
pub enum SchedPolicy {
|
||||
Stop,
|
||||
RealTime {
|
||||
rt_prio: real_time::RtPrio,
|
||||
rt_policy: RealTimePolicy,
|
||||
},
|
||||
Fair(Nice),
|
||||
Idle,
|
||||
}
|
||||
|
||||
impl From<Priority> for SchedPolicy {
|
||||
fn from(priority: Priority) -> Self {
|
||||
match priority.range().get() {
|
||||
0 => SchedPolicy::Stop,
|
||||
rt @ 1..=99 => SchedPolicy::RealTime {
|
||||
rt_prio: RangedU8::new(rt),
|
||||
rt_policy: Default::default(),
|
||||
},
|
||||
100..=139 => SchedPolicy::Fair(priority.into()),
|
||||
_ => SchedPolicy::Idle,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// The scheduling attribute for a thread.
|
||||
///
|
||||
/// This is used to store the scheduling policy and runtime parameters for each
|
||||
/// scheduling class.
|
||||
#[derive(Debug)]
|
||||
pub struct SchedAttr {
|
||||
policy: SpinLock<SchedPolicy>,
|
||||
policy: AtomicU64,
|
||||
|
||||
real_time: real_time::RealTimeAttr,
|
||||
fair: fair::FairAttr,
|
||||
@ -153,7 +134,7 @@ impl SchedAttr {
|
||||
/// Constructs a new `SchedAttr` with the given scheduling policy.
|
||||
pub fn new(policy: SchedPolicy) -> Self {
|
||||
Self {
|
||||
policy: SpinLock::new(policy),
|
||||
policy: SchedPolicy::into_raw(policy).into(),
|
||||
real_time: {
|
||||
let (prio, policy) = match policy {
|
||||
SchedPolicy::RealTime { rt_prio, rt_policy } => (rt_prio.get(), rt_policy),
|
||||
@ -170,7 +151,11 @@ impl SchedAttr {
|
||||
|
||||
/// Retrieves the current scheduling policy of the thread.
|
||||
pub fn policy(&self) -> SchedPolicy {
|
||||
*self.policy.lock()
|
||||
SchedPolicy::from_raw(self.policy.load(Relaxed))
|
||||
}
|
||||
|
||||
fn policy_kind(&self) -> SchedPolicyKind {
|
||||
SchedPolicyKind::from_raw(self.policy.load(Relaxed))
|
||||
}
|
||||
|
||||
/// Updates the scheduling policy of the thread.
|
||||
@ -178,40 +163,42 @@ impl SchedAttr {
|
||||
/// Specifically for real-time policies, if the new policy doesn't
|
||||
/// specify a base slice factor for RR, the old one will be kept.
|
||||
pub fn set_policy(&self, mut policy: SchedPolicy) {
|
||||
let mut guard = self.policy.lock();
|
||||
match policy {
|
||||
SchedPolicy::RealTime { rt_prio, rt_policy } => {
|
||||
self.real_time.update(rt_prio.get(), rt_policy);
|
||||
let _ = self.policy.fetch_update(SeqCst, SeqCst, |raw| {
|
||||
let current = SchedPolicy::from_raw(raw);
|
||||
match policy {
|
||||
SchedPolicy::RealTime { rt_prio, rt_policy } => {
|
||||
self.real_time.update(rt_prio.get(), rt_policy);
|
||||
}
|
||||
SchedPolicy::Fair(nice) => self.fair.update(nice),
|
||||
_ => {}
|
||||
}
|
||||
SchedPolicy::Fair(nice) => self.fair.update(nice),
|
||||
_ => {}
|
||||
}
|
||||
|
||||
// Keep the old base slice factor if the new policy doesn't specify one.
|
||||
if let (
|
||||
SchedPolicy::RealTime {
|
||||
rt_policy:
|
||||
RealTimePolicy::RoundRobin {
|
||||
base_slice_factor: slot,
|
||||
},
|
||||
..
|
||||
},
|
||||
SchedPolicy::RealTime {
|
||||
rt_policy: RealTimePolicy::RoundRobin { base_slice_factor },
|
||||
..
|
||||
},
|
||||
) = (*guard, &mut policy)
|
||||
{
|
||||
*base_slice_factor = slot.or(*base_slice_factor);
|
||||
}
|
||||
// Keep the old base slice factor if the new policy doesn't specify one.
|
||||
if let (
|
||||
SchedPolicy::RealTime {
|
||||
rt_policy:
|
||||
RealTimePolicy::RoundRobin {
|
||||
base_slice_factor: slot,
|
||||
},
|
||||
..
|
||||
},
|
||||
SchedPolicy::RealTime {
|
||||
rt_policy: RealTimePolicy::RoundRobin { base_slice_factor },
|
||||
..
|
||||
},
|
||||
) = (current, &mut policy)
|
||||
{
|
||||
*base_slice_factor = slot.or(*base_slice_factor);
|
||||
}
|
||||
|
||||
*guard = policy;
|
||||
Some(SchedPolicy::into_raw(policy))
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
impl Scheduler for ClassScheduler {
|
||||
fn enqueue(&self, task: Arc<Task>, flags: EnqueueFlags) -> Option<CpuId> {
|
||||
let thread = task.as_thread()?;
|
||||
let thread = task.as_thread()?.clone();
|
||||
|
||||
let (still_in_rq, cpu) = {
|
||||
let selected_cpu_id = self.select_cpu(thread.atomic_cpu_affinity());
|
||||
@ -231,7 +218,7 @@ impl Scheduler for ClassScheduler {
|
||||
return None;
|
||||
}
|
||||
|
||||
rq.enqueue_thread(thread, Some(flags));
|
||||
rq.enqueue_entity((task, thread), Some(flags));
|
||||
Some(cpu)
|
||||
}
|
||||
|
||||
@ -277,23 +264,20 @@ impl ClassScheduler {
|
||||
}
|
||||
|
||||
impl PerCpuClassRqSet {
|
||||
fn pick_next_thread(&mut self) -> Option<Arc<Thread>> {
|
||||
fn pick_next_entity(&mut self) -> Option<SchedEntity> {
|
||||
(self.stop.pick_next())
|
||||
.or_else(|| self.real_time.pick_next())
|
||||
.or_else(|| self.fair.pick_next())
|
||||
.or_else(|| self.idle.pick_next())
|
||||
}
|
||||
|
||||
fn enqueue_thread(&mut self, thread: &Arc<Thread>, flags: Option<EnqueueFlags>) {
|
||||
let attr = thread.sched_attr();
|
||||
|
||||
let cloned = thread.clone();
|
||||
match *attr.policy.lock() {
|
||||
SchedPolicy::Stop => self.stop.enqueue(cloned, flags),
|
||||
SchedPolicy::RealTime { .. } => self.real_time.enqueue(cloned, flags),
|
||||
SchedPolicy::Fair(_) => self.fair.enqueue(cloned, flags),
|
||||
SchedPolicy::Idle => self.idle.enqueue(cloned, flags),
|
||||
};
|
||||
fn enqueue_entity(&mut self, entity: SchedEntity, flags: Option<EnqueueFlags>) {
|
||||
match entity.1.sched_attr().policy_kind() {
|
||||
SchedPolicyKind::Stop => self.stop.enqueue(entity, flags),
|
||||
SchedPolicyKind::RealTime => self.real_time.enqueue(entity, flags),
|
||||
SchedPolicyKind::Fair => self.fair.enqueue(entity, flags),
|
||||
SchedPolicyKind::Idle => self.idle.enqueue(entity, flags),
|
||||
}
|
||||
}
|
||||
|
||||
fn nr_queued_and_running(&mut self) -> (u32, u32) {
|
||||
@ -305,38 +289,32 @@ impl PerCpuClassRqSet {
|
||||
|
||||
impl LocalRunQueue for PerCpuClassRqSet {
|
||||
fn current(&self) -> Option<&Arc<Task>> {
|
||||
self.current.as_ref().map(|(task, _)| task)
|
||||
self.current.as_ref().map(|((task, _), _)| task)
|
||||
}
|
||||
|
||||
fn pick_next_current(&mut self) -> Option<&Arc<Task>> {
|
||||
self.pick_next_thread().and_then(|next| {
|
||||
let next_task = next.task();
|
||||
if let Some((old_task, _)) = self
|
||||
.current
|
||||
.replace((next_task.clone(), CurrentRuntime::new()))
|
||||
{
|
||||
if Arc::ptr_eq(&old_task, &next_task) {
|
||||
self.pick_next_entity().and_then(|next| {
|
||||
let next_ptr = Arc::as_ptr(&next.1);
|
||||
if let Some((old, _)) = self.current.replace((next, CurrentRuntime::new())) {
|
||||
if Arc::as_ptr(&old.1) == next_ptr {
|
||||
return None;
|
||||
}
|
||||
let old = old_task.as_thread().unwrap();
|
||||
self.enqueue_thread(old, None);
|
||||
self.enqueue_entity(old, None);
|
||||
}
|
||||
self.current.as_ref().map(|(task, _)| task)
|
||||
self.current.as_ref().map(|((task, _), _)| task)
|
||||
})
|
||||
}
|
||||
|
||||
fn update_current(&mut self, flags: UpdateFlags) -> bool {
|
||||
if let Some((cur_task, rt)) = &mut self.current
|
||||
&& let Some(cur) = cur_task.as_thread()
|
||||
{
|
||||
if let Some(((_, cur), rt)) = &mut self.current {
|
||||
rt.update();
|
||||
let attr = &cur.sched_attr();
|
||||
|
||||
let (current_expired, lookahead) = match &*attr.policy.lock() {
|
||||
SchedPolicy::Stop => (self.stop.update_current(rt, attr, flags), 0),
|
||||
SchedPolicy::RealTime { .. } => (self.real_time.update_current(rt, attr, flags), 1),
|
||||
SchedPolicy::Fair(_) => (self.fair.update_current(rt, attr, flags), 2),
|
||||
SchedPolicy::Idle => (self.idle.update_current(rt, attr, flags), 3),
|
||||
let (current_expired, lookahead) = match attr.policy_kind() {
|
||||
SchedPolicyKind::Stop => (self.stop.update_current(rt, attr, flags), 0),
|
||||
SchedPolicyKind::RealTime => (self.real_time.update_current(rt, attr, flags), 1),
|
||||
SchedPolicyKind::Fair => (self.fair.update_current(rt, attr, flags), 2),
|
||||
SchedPolicyKind::Idle => (self.idle.update_current(rt, attr, flags), 3),
|
||||
};
|
||||
|
||||
current_expired
|
||||
@ -349,7 +327,7 @@ impl LocalRunQueue for PerCpuClassRqSet {
|
||||
}
|
||||
|
||||
fn dequeue_current(&mut self) -> Option<Arc<Task>> {
|
||||
self.current.take().map(|(cur_task, _)| {
|
||||
self.current.take().map(|((cur_task, _), _)| {
|
||||
cur_task.schedule_info().cpu.set_to_none();
|
||||
cur_task
|
||||
})
|
||||
|
144
kernel/src/sched/sched_class/policy.rs
Normal file
144
kernel/src/sched/sched_class/policy.rs
Normal file
@ -0,0 +1,144 @@
|
||||
// SPDX-License-Identifier: MPL-2.0
|
||||
|
||||
use core::num::NonZero;
|
||||
|
||||
use super::real_time::RealTimePolicy;
|
||||
use crate::sched::priority::{Nice, NiceRange, Priority, RangedU8};
|
||||
|
||||
/// The User-chosen scheduling policy.
|
||||
///
|
||||
/// The scheduling policies are specified by the user, usually through its priority.
|
||||
#[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord)]
|
||||
pub enum SchedPolicy {
|
||||
Stop,
|
||||
RealTime {
|
||||
rt_prio: super::real_time::RtPrio,
|
||||
rt_policy: RealTimePolicy,
|
||||
},
|
||||
Fair(Nice),
|
||||
Idle,
|
||||
}
|
||||
|
||||
#[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord)]
|
||||
pub(super) enum SchedPolicyKind {
|
||||
Stop,
|
||||
RealTime,
|
||||
Fair,
|
||||
Idle,
|
||||
}
|
||||
|
||||
impl From<Priority> for SchedPolicy {
|
||||
fn from(priority: Priority) -> Self {
|
||||
match priority.range().get() {
|
||||
0 => SchedPolicy::Stop,
|
||||
rt @ 1..=99 => SchedPolicy::RealTime {
|
||||
rt_prio: RangedU8::new(rt),
|
||||
rt_policy: Default::default(),
|
||||
},
|
||||
100..=139 => SchedPolicy::Fair(priority.into()),
|
||||
_ => SchedPolicy::Idle,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
const TYPE_MASK: u64 = 0x0000_0000_0000_ffff;
|
||||
const TYPE_SHIFT: u32 = 0;
|
||||
|
||||
const TYPE_STOP: u64 = 0;
|
||||
const TYPE_REAL_TIME: u64 = 1;
|
||||
const TYPE_FAIR: u64 = 2;
|
||||
const TYPE_IDLE: u64 = 3;
|
||||
|
||||
const SUBTYPE_MASK: u64 = 0x0000_0000_00ff_0000;
|
||||
const SUBTYPE_SHIFT: u32 = 16;
|
||||
|
||||
const RT_PRIO_MASK: u64 = SUBTYPE_MASK;
|
||||
const RT_PRIO_SHIFT: u32 = SUBTYPE_SHIFT;
|
||||
|
||||
const FAIR_NICE_MASK: u64 = SUBTYPE_MASK;
|
||||
const FAIR_NICE_SHIFT: u32 = SUBTYPE_SHIFT;
|
||||
|
||||
const RT_TYPE_MASK: u64 = 0x0000_0000_ff00_0000;
|
||||
const RT_TYPE_SHIFT: u32 = 24;
|
||||
|
||||
const RT_TYPE_FIFO: u64 = 0;
|
||||
const RT_TYPE_RR: u64 = 1;
|
||||
|
||||
const RT_FACTOR_MASK: u64 = 0xffff_ffff_0000_0000;
|
||||
const RT_FACTOR_SHIFT: u32 = 32;
|
||||
|
||||
fn get(raw: u64, mask: u64, shift: u32) -> u64 {
|
||||
(raw & mask) >> shift
|
||||
}
|
||||
|
||||
fn set(value: u64, mask: u64, shift: u32) -> u64 {
|
||||
(value << shift) & mask
|
||||
}
|
||||
|
||||
impl SchedPolicy {
|
||||
pub(super) fn from_raw(raw: u64) -> Self {
|
||||
match get(raw, TYPE_MASK, TYPE_SHIFT) {
|
||||
TYPE_STOP => SchedPolicy::Stop,
|
||||
TYPE_REAL_TIME => SchedPolicy::RealTime {
|
||||
rt_prio: RangedU8::new(get(raw, RT_PRIO_MASK, RT_PRIO_SHIFT) as u8),
|
||||
rt_policy: match get(raw, RT_TYPE_MASK, RT_TYPE_SHIFT) {
|
||||
RT_TYPE_FIFO => RealTimePolicy::Fifo,
|
||||
RT_TYPE_RR => RealTimePolicy::RoundRobin {
|
||||
base_slice_factor: NonZero::new(
|
||||
get(raw, RT_FACTOR_MASK, RT_FACTOR_SHIFT) as u32
|
||||
),
|
||||
},
|
||||
_ => unreachable!(),
|
||||
},
|
||||
},
|
||||
TYPE_FAIR => {
|
||||
SchedPolicy::Fair(Nice::new(NiceRange::new(
|
||||
get(raw, FAIR_NICE_MASK, FAIR_NICE_SHIFT) as i8,
|
||||
)))
|
||||
}
|
||||
TYPE_IDLE => SchedPolicy::Idle,
|
||||
_ => unreachable!(),
|
||||
}
|
||||
}
|
||||
|
||||
pub(super) fn into_raw(this: Self) -> u64 {
|
||||
match this {
|
||||
SchedPolicy::Stop => set(TYPE_STOP, TYPE_MASK, TYPE_SHIFT),
|
||||
SchedPolicy::RealTime { rt_prio, rt_policy } => {
|
||||
let ty = set(TYPE_REAL_TIME, TYPE_MASK, TYPE_SHIFT);
|
||||
let rt_prio = set(rt_prio.get() as u64, RT_PRIO_MASK, RT_PRIO_SHIFT);
|
||||
let rt_policy = match rt_policy {
|
||||
RealTimePolicy::Fifo => set(RT_TYPE_FIFO, RT_TYPE_MASK, RT_TYPE_SHIFT),
|
||||
RealTimePolicy::RoundRobin { base_slice_factor } => {
|
||||
let rt_type = set(RT_TYPE_RR, RT_TYPE_MASK, RT_TYPE_SHIFT);
|
||||
let rt_factor = set(
|
||||
base_slice_factor.map_or(0, NonZero::get) as u64,
|
||||
RT_FACTOR_MASK,
|
||||
RT_FACTOR_SHIFT,
|
||||
);
|
||||
rt_type | rt_factor
|
||||
}
|
||||
};
|
||||
ty | rt_prio | rt_policy
|
||||
}
|
||||
SchedPolicy::Fair(nice) => {
|
||||
let ty = set(TYPE_FAIR, TYPE_MASK, TYPE_SHIFT);
|
||||
let nice = set(nice.range().get() as u64, FAIR_NICE_MASK, FAIR_NICE_SHIFT);
|
||||
ty | nice
|
||||
}
|
||||
SchedPolicy::Idle => set(TYPE_IDLE, TYPE_MASK, TYPE_SHIFT),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl SchedPolicyKind {
|
||||
pub fn from_raw(raw: u64) -> Self {
|
||||
match get(raw, TYPE_MASK, TYPE_SHIFT) {
|
||||
TYPE_STOP => SchedPolicyKind::Stop,
|
||||
TYPE_REAL_TIME => SchedPolicyKind::RealTime,
|
||||
TYPE_FAIR => SchedPolicyKind::Fair,
|
||||
TYPE_IDLE => SchedPolicyKind::Idle,
|
||||
_ => unreachable!(),
|
||||
}
|
||||
}
|
||||
}
|
@ -17,7 +17,7 @@ pub type RtPrio = RangedU8<1, 99>;
|
||||
pub enum RealTimePolicy {
|
||||
Fifo,
|
||||
RoundRobin {
|
||||
base_slice_factor: Option<NonZero<u64>>,
|
||||
base_slice_factor: Option<NonZero<u32>>,
|
||||
},
|
||||
}
|
||||
|
||||
@ -34,7 +34,8 @@ impl RealTimePolicy {
|
||||
match self {
|
||||
RealTimePolicy::RoundRobin { base_slice_factor } => {
|
||||
base_slice_clocks()
|
||||
* base_slice_factor.map_or(DEFAULT_BASE_SLICE_FACTOR, NonZero::get)
|
||||
* base_slice_factor
|
||||
.map_or(DEFAULT_BASE_SLICE_FACTOR, |factor| u64::from(factor.get()))
|
||||
}
|
||||
RealTimePolicy::Fifo => 0,
|
||||
}
|
||||
@ -76,7 +77,7 @@ impl RealTimeAttr {
|
||||
|
||||
struct PrioArray {
|
||||
map: BitArr![for 100],
|
||||
queue: [VecDeque<Arc<Thread>>; 100],
|
||||
queue: [VecDeque<SchedEntity>; 100],
|
||||
}
|
||||
|
||||
impl core::fmt::Debug for PrioArray {
|
||||
@ -87,7 +88,7 @@ impl core::fmt::Debug for PrioArray {
|
||||
})
|
||||
.field_with("queue", |f| {
|
||||
f.debug_list()
|
||||
.entries((self.queue.iter().flatten()).map(|thread| thread.sched_attr()))
|
||||
.entries((self.queue.iter().flatten()).map(|(_, thread)| thread.sched_attr()))
|
||||
.finish()
|
||||
})
|
||||
.finish()
|
||||
@ -95,7 +96,7 @@ impl core::fmt::Debug for PrioArray {
|
||||
}
|
||||
|
||||
impl PrioArray {
|
||||
fn enqueue(&mut self, thread: Arc<Thread>, prio: u8) {
|
||||
fn enqueue(&mut self, thread: SchedEntity, prio: u8) {
|
||||
let queue = &mut self.queue[usize::from(prio)];
|
||||
let is_empty = queue.is_empty();
|
||||
queue.push_back(thread);
|
||||
@ -104,7 +105,7 @@ impl PrioArray {
|
||||
}
|
||||
}
|
||||
|
||||
fn pop(&mut self) -> Option<Arc<Thread>> {
|
||||
fn pop(&mut self) -> Option<SchedEntity> {
|
||||
let mut iter = self.map.iter_ones();
|
||||
let prio = iter.next()? as u8;
|
||||
|
||||
@ -165,9 +166,9 @@ impl RealTimeClassRq {
|
||||
}
|
||||
|
||||
impl SchedClassRq for RealTimeClassRq {
|
||||
fn enqueue(&mut self, thread: Arc<Thread>, _: Option<EnqueueFlags>) {
|
||||
let prio = thread.sched_attr().real_time.prio.load(Relaxed);
|
||||
self.inactive_array().enqueue(thread, prio);
|
||||
fn enqueue(&mut self, entity: SchedEntity, _: Option<EnqueueFlags>) {
|
||||
let prio = entity.1.sched_attr().real_time.prio.load(Relaxed);
|
||||
self.inactive_array().enqueue(entity, prio);
|
||||
self.nr_running += 1;
|
||||
}
|
||||
|
||||
@ -179,7 +180,7 @@ impl SchedClassRq for RealTimeClassRq {
|
||||
self.nr_running == 0
|
||||
}
|
||||
|
||||
fn pick_next(&mut self) -> Option<Arc<Thread>> {
|
||||
fn pick_next(&mut self) -> Option<SchedEntity> {
|
||||
if self.nr_running == 0 {
|
||||
return None;
|
||||
}
|
||||
|
@ -1,5 +1,7 @@
|
||||
// SPDX-License-Identifier: MPL-2.0
|
||||
|
||||
use core::sync::atomic::AtomicBool;
|
||||
|
||||
use super::*;
|
||||
|
||||
/// The per-cpu run queue for the STOP scheduling class.
|
||||
@ -32,6 +34,7 @@ impl SchedClassRq for StopClassRq {
|
||||
if self.thread.replace(thread).is_some() {
|
||||
panic!("Multiple `stop` threads spawned")
|
||||
}
|
||||
self.has_value.store(true, Relaxed);
|
||||
}
|
||||
|
||||
fn len(&mut self) -> usize {
|
||||
|
Loading…
x
Reference in New Issue
Block a user