diff --git a/kernel/src/lib.rs b/kernel/src/lib.rs index e3be60c54..dc764d443 100644 --- a/kernel/src/lib.rs +++ b/kernel/src/lib.rs @@ -35,6 +35,7 @@ use core::sync::atomic::Ordering; use ostd::{ arch::qemu::{exit_qemu, QemuExitCode}, boot, + cpu::PinCurrentCpu, }; use process::Process; @@ -83,7 +84,13 @@ pub fn main() { component::init_all(component::parse_metadata!()).unwrap(); init(); ostd::IN_BOOTSTRAP_CONTEXT.store(false, Ordering::Relaxed); + + // Spawn all AP idle threads. + ostd::boot::smp::register_ap_entry(ap_init); + + // Spawn the first kernel thread on BSP. Thread::spawn_kernel_thread(ThreadOptions::new(init_thread)); + // Spawning functions in the bootstrap context will not return. unreachable!() } @@ -100,6 +107,25 @@ pub fn init() { process::init(); } +fn ap_init() -> ! { + fn ap_idle_thread() { + let preempt_guard = ostd::task::disable_preempt(); + let cpu_id = preempt_guard.current_cpu(); + drop(preempt_guard); + log::info!("Kernel idle thread for CPU #{} started.", cpu_id); + loop { + Thread::yield_now(); + } + } + let preempt_guard = ostd::task::disable_preempt(); + let cpu_id = preempt_guard.current_cpu(); + drop(preempt_guard); + + Thread::spawn_kernel_thread(ThreadOptions::new(ap_idle_thread).cpu_affinity(cpu_id.into())); + // Spawning functions in the bootstrap context will not return. + unreachable!() +} + fn init_thread() { println!( "[kernel] Spawn init thread, tid = {}", diff --git a/kernel/src/sched/priority_scheduler.rs b/kernel/src/sched/priority_scheduler.rs index 415576430..f91a27f46 100644 --- a/kernel/src/sched/priority_scheduler.rs +++ b/kernel/src/sched/priority_scheduler.rs @@ -1,8 +1,9 @@ // SPDX-License-Identifier: MPL-2.0 use ostd::{ - cpu::{num_cpus, PinCurrentCpu}, + cpu::{num_cpus, CpuSet, PinCurrentCpu}, task::{ + disable_preempt, scheduler::{inject_scheduler, EnqueueFlags, LocalRunQueue, Scheduler, UpdateFlags}, AtomicCpuId, Priority, Task, }, @@ -36,10 +37,31 @@ impl PreemptScheduler { Self { rq } } - /// Selects a cpu for task to run on. - fn select_cpu(&self, _runnable: &Arc) -> u32 { - // FIXME: adopt more reasonable policy once we fully enable SMP. - 0 + /// Selects a CPU for task to run on for the first time. + fn select_cpu(&self, runnable: &Arc) -> u32 { + // If the CPU of a runnable task has been set before, keep scheduling + // the task to that one. + // TODO: Consider migrating tasks between CPUs for load balancing. + if let Some(cpu_id) = runnable.cpu().get() { + return cpu_id; + } + + let preempt_guard = disable_preempt(); + let mut selected = preempt_guard.current_cpu(); + let mut minimum_load = usize::MAX; + + for candidate in runnable.cpu_affinity().iter() { + let rq = self.rq[candidate].lock(); + // A wild guess measuring the load of a runqueue. We assume that + // real-time tasks are 4-times as important as normal tasks. + let load = rq.real_time_entities.len() * 4 + rq.normal_entities.len(); + if load < minimum_load { + selected = candidate as u32; + minimum_load = load; + } + } + + selected } } @@ -214,11 +236,15 @@ impl PreemptSchedInfo for Task { const REAL_TIME_TASK_PRIORITY: Self::PRIORITY = Priority::new(100); fn priority(&self) -> Self::PRIORITY { - self.priority() + self.schedule_info().priority } fn cpu(&self) -> &AtomicCpuId { - self.cpu() + &self.schedule_info().cpu + } + + fn cpu_affinity(&self) -> &CpuSet { + &self.schedule_info().cpu_affinity } } @@ -231,6 +257,8 @@ trait PreemptSchedInfo { fn cpu(&self) -> &AtomicCpuId; + fn cpu_affinity(&self) -> &CpuSet; + fn is_real_time(&self) -> bool { self.priority() < Self::REAL_TIME_TASK_PRIORITY } diff --git a/ostd/src/cpu/mod.rs b/ostd/src/cpu/mod.rs index 43d16dfd0..bd98f8d7c 100644 --- a/ostd/src/cpu/mod.rs +++ b/ostd/src/cpu/mod.rs @@ -166,3 +166,11 @@ impl CpuSet { self.bitset.iter_ones() } } + +impl From for CpuSet { + fn from(cpu_id: u32) -> Self { + let mut set = Self::new_empty(); + set.add(cpu_id); + set + } +} diff --git a/ostd/src/task/kernel_stack.rs b/ostd/src/task/kernel_stack.rs new file mode 100644 index 000000000..c5295fc97 --- /dev/null +++ b/ostd/src/task/kernel_stack.rs @@ -0,0 +1,102 @@ +// SPDX-License-Identifier: MPL-2.0 + +use crate::{ + mm::{kspace::KERNEL_PAGE_TABLE, FrameAllocOptions, Paddr, PageFlags, Segment, PAGE_SIZE}, + prelude::*, +}; + +/// The kernel stack size of a task, specified in pages. +/// +/// By default, we choose a rather large stack size. +/// OSTD users can choose a smaller size by specifying +/// the `OSTD_TASK_STACK_SIZE_IN_PAGES` environment variable +/// at build time. +pub static STACK_SIZE_IN_PAGES: u32 = parse_u32_or_default( + option_env!("OSTD_TASK_STACK_SIZE_IN_PAGES"), + DEFAULT_STACK_SIZE_IN_PAGES, +); + +/// The default kernel stack size of a task, specified in pages. +pub const DEFAULT_STACK_SIZE_IN_PAGES: u32 = 128; + +#[derive(Debug)] +pub struct KernelStack { + segment: Segment, + has_guard_page: bool, +} + +impl KernelStack { + pub fn new() -> Result { + Ok(Self { + segment: FrameAllocOptions::new(STACK_SIZE_IN_PAGES as usize).alloc_contiguous()?, + has_guard_page: false, + }) + } + + /// Generates a kernel stack with a guard page. + /// An additional page is allocated and be regarded as a guard page, which should not be accessed. + pub fn new_with_guard_page() -> Result { + let stack_segment = + FrameAllocOptions::new(STACK_SIZE_IN_PAGES as usize + 1).alloc_contiguous()?; + // FIXME: modifying the the linear mapping is bad. + let page_table = KERNEL_PAGE_TABLE.get().unwrap(); + let guard_page_vaddr = { + let guard_page_paddr = stack_segment.start_paddr(); + crate::mm::paddr_to_vaddr(guard_page_paddr) + }; + // SAFETY: the segment allocated is not used by others so we can protect it. + unsafe { + let vaddr_range = guard_page_vaddr..guard_page_vaddr + PAGE_SIZE; + page_table + .protect_flush_tlb(&vaddr_range, |p| p.flags -= PageFlags::RW) + .unwrap(); + } + Ok(Self { + segment: stack_segment, + has_guard_page: true, + }) + } + + pub fn end_paddr(&self) -> Paddr { + self.segment.end_paddr() + } +} + +impl Drop for KernelStack { + fn drop(&mut self) { + if self.has_guard_page { + // FIXME: modifying the the linear mapping is bad. + let page_table = KERNEL_PAGE_TABLE.get().unwrap(); + let guard_page_vaddr = { + let guard_page_paddr = self.segment.start_paddr(); + crate::mm::paddr_to_vaddr(guard_page_paddr) + }; + // SAFETY: the segment allocated is not used by others so we can protect it. + unsafe { + let vaddr_range = guard_page_vaddr..guard_page_vaddr + PAGE_SIZE; + page_table + .protect_flush_tlb(&vaddr_range, |p| p.flags |= PageFlags::RW) + .unwrap(); + } + } + } +} + +const fn parse_u32_or_default(size: Option<&str>, default: u32) -> u32 { + match size { + Some(value) => parse_u32(value), + None => default, + } +} + +const fn parse_u32(input: &str) -> u32 { + let mut output: u32 = 0; + let bytes = input.as_bytes(); + let mut i = 0; + while i < bytes.len() { + let digit = (bytes[i] - b'0') as u32; + output = output * 10 + digit; + i += 1; + } + output +} diff --git a/ostd/src/task/mod.rs b/ostd/src/task/mod.rs index 3f426b65c..253caf09a 100644 --- a/ostd/src/task/mod.rs +++ b/ostd/src/task/mod.rs @@ -3,15 +3,253 @@ //! Tasks are the unit of code execution. pub(crate) mod atomic_mode; +mod kernel_stack; mod preempt; mod processor; pub mod scheduler; -#[allow(clippy::module_inception)] -mod task; +use core::{any::Any, cell::UnsafeCell}; + +use kernel_stack::KernelStack; pub(crate) use preempt::cpu_local::reset_preempt_info; +use processor::current_task; pub use self::{ preempt::{disable_preempt, DisabledPreemptGuard}, - task::{AtomicCpuId, Priority, Task, TaskAdapter, TaskContextApi, TaskOptions}, + scheduler::info::{AtomicCpuId, Priority, TaskScheduleInfo}, }; +pub(crate) use crate::arch::task::{context_switch, TaskContext}; +use crate::{cpu::CpuSet, prelude::*, user::UserSpace}; + +/// A task that executes a function to the end. +/// +/// Each task is associated with per-task data and an optional user space. +/// If having a user space, the task can switch to the user space to +/// execute user code. Multiple tasks can share a single user space. +pub struct Task { + func: Box, + data: Box, + user_space: Option>, + ctx: UnsafeCell, + /// kernel stack, note that the top is SyscallFrame/TrapFrame + kstack: KernelStack, + + schedule_info: TaskScheduleInfo, +} + +// SAFETY: `UnsafeCell` is not `Sync`. However, we only use it in `schedule()` where +// we have exclusive access to the field. +unsafe impl Sync for Task {} + +impl Task { + /// Gets the current task. + /// + /// It returns `None` if the function is called in the bootstrap context. + pub fn current() -> Option> { + current_task() + } + + pub(super) fn ctx(&self) -> &UnsafeCell { + &self.ctx + } + + /// Yields execution so that another task may be scheduled. + /// + /// Note that this method cannot be simply named "yield" as the name is + /// a Rust keyword. + pub fn yield_now() { + scheduler::yield_now() + } + + /// Runs the task. + /// + /// BUG: This method highly depends on the current scheduling policy. + pub fn run(self: &Arc) { + scheduler::run_new_task(self.clone()); + } + + /// Returns the task data. + pub fn data(&self) -> &Box { + &self.data + } + + /// Get the attached scheduling information. + pub fn schedule_info(&self) -> &TaskScheduleInfo { + &self.schedule_info + } + + /// Returns the user space of this task, if it has. + pub fn user_space(&self) -> Option<&Arc> { + if self.user_space.is_some() { + Some(self.user_space.as_ref().unwrap()) + } else { + None + } + } + + /// Exits the current task. + /// + /// The task `self` must be the task that is currently running. + /// + /// **NOTE:** If there is anything left on the stack, it will be forgotten. This behavior may + /// lead to resource leakage. + fn exit(self: Arc) -> ! { + // `current_task()` still holds a strong reference, so nothing is destroyed at this point, + // neither is the kernel stack. + drop(self); + scheduler::exit_current(); + unreachable!() + } +} + +/// Options to create or spawn a new task. +pub struct TaskOptions { + func: Option>, + data: Option>, + user_space: Option>, + priority: Priority, + cpu_affinity: CpuSet, +} + +impl TaskOptions { + /// Creates a set of options for a task. + pub fn new(func: F) -> Self + where + F: Fn() + Send + Sync + 'static, + { + Self { + func: Some(Box::new(func)), + data: None, + user_space: None, + priority: Priority::normal(), + cpu_affinity: CpuSet::new_full(), + } + } + + /// Sets the function that represents the entry point of the task. + pub fn func(mut self, func: F) -> Self + where + F: Fn() + Send + Sync + 'static, + { + self.func = Some(Box::new(func)); + self + } + + /// Sets the data associated with the task. + pub fn data(mut self, data: T) -> Self + where + T: Any + Send + Sync, + { + self.data = Some(Box::new(data)); + self + } + + /// Sets the user space associated with the task. + pub fn user_space(mut self, user_space: Option>) -> Self { + self.user_space = user_space; + self + } + + /// Sets the priority of the task. + pub fn priority(mut self, priority: Priority) -> Self { + self.priority = priority; + self + } + + /// Sets the CPU affinity mask for the task. + /// + /// The `cpu_affinity` parameter represents + /// the desired set of CPUs to run the task on. + pub fn cpu_affinity(mut self, cpu_affinity: CpuSet) -> Self { + self.cpu_affinity = cpu_affinity; + self + } + + /// Builds a new task without running it immediately. + pub fn build(self) -> Result> { + /// all task will entering this function + /// this function is mean to executing the task_fn in Task + extern "C" fn kernel_task_entry() { + let current_task = current_task() + .expect("no current task, it should have current task in kernel task entry"); + current_task.func.call(()); + current_task.exit(); + } + + let mut new_task = Task { + func: self.func.unwrap(), + data: self.data.unwrap(), + user_space: self.user_space, + ctx: UnsafeCell::new(TaskContext::default()), + kstack: KernelStack::new_with_guard_page()?, + schedule_info: TaskScheduleInfo { + cpu: AtomicCpuId::default(), + priority: self.priority, + cpu_affinity: self.cpu_affinity, + }, + }; + + let ctx = new_task.ctx.get_mut(); + ctx.set_instruction_pointer(kernel_task_entry as usize); + // We should reserve space for the return address in the stack, otherwise + // we will write across the page boundary due to the implementation of + // the context switch. + // + // According to the System V AMD64 ABI, the stack pointer should be aligned + // to at least 16 bytes. And a larger alignment is needed if larger arguments + // are passed to the function. The `kernel_task_entry` function does not + // have any arguments, so we only need to align the stack pointer to 16 bytes. + ctx.set_stack_pointer(crate::mm::paddr_to_vaddr(new_task.kstack.end_paddr() - 16)); + + Ok(Arc::new(new_task)) + } + + /// Builds a new task and run it immediately. + pub fn spawn(self) -> Result> { + let task = self.build()?; + task.run(); + Ok(task) + } +} + +/// Trait for manipulating the task context. +pub trait TaskContextApi { + /// Sets instruction pointer + fn set_instruction_pointer(&mut self, ip: usize); + + /// Gets instruction pointer + fn instruction_pointer(&self) -> usize; + + /// Sets stack pointer + fn set_stack_pointer(&mut self, sp: usize); + + /// Gets stack pointer + fn stack_pointer(&self) -> usize; +} + +#[cfg(ktest)] +mod test { + use crate::prelude::*; + + #[ktest] + fn create_task() { + #[allow(clippy::eq_op)] + let task = || { + assert_eq!(1, 1); + }; + let task_option = crate::task::TaskOptions::new(task) + .data(()) + .build() + .unwrap(); + task_option.run(); + } + + #[ktest] + fn spawn_task() { + #[allow(clippy::eq_op)] + let task = || { + assert_eq!(1, 1); + }; + let _ = crate::task::TaskOptions::new(task).data(()).spawn(); + } +} diff --git a/ostd/src/task/processor.rs b/ostd/src/task/processor.rs index 16ae31c02..4fe814bac 100644 --- a/ostd/src/task/processor.rs +++ b/ostd/src/task/processor.rs @@ -2,7 +2,7 @@ use alloc::sync::Arc; -use super::task::{context_switch, Task, TaskContext}; +use super::{context_switch, Task, TaskContext}; use crate::cpu_local_cell; cpu_local_cell! { diff --git a/ostd/src/task/scheduler/fifo_scheduler.rs b/ostd/src/task/scheduler/fifo_scheduler.rs index c338c6d11..3b0976778 100644 --- a/ostd/src/task/scheduler/fifo_scheduler.rs +++ b/ostd/src/task/scheduler/fifo_scheduler.rs @@ -122,7 +122,7 @@ impl Default for FifoScheduler { impl FifoSchedInfo for Task { fn cpu(&self) -> &AtomicCpuId { - self.cpu() + &self.schedule_info().cpu } } diff --git a/ostd/src/task/scheduler/info.rs b/ostd/src/task/scheduler/info.rs new file mode 100644 index 000000000..baaf32588 --- /dev/null +++ b/ostd/src/task/scheduler/info.rs @@ -0,0 +1,132 @@ +// SPDX-License-Identifier: MPL-2.0 + +//! Scheduling related information in a task. + +use core::sync::atomic::{AtomicU32, Ordering}; + +use crate::cpu::CpuSet; + +/// Fields of a task that OSTD will never touch. +/// +/// The type ought to be defined by the OSTD user and injected into the task. +/// They are not part of the dynamic task data because it's slower there for +/// the user-defined scheduler to access. The better ways to let the user +/// define them, such as +/// [existential types](https://github.com/rust-lang/rfcs/pull/2492) do not +/// exist yet. So we decide to define them in OSTD. +pub struct TaskScheduleInfo { + /// Priority of the task. + pub priority: Priority, + /// The CPU that the task would like to be running on. + pub cpu: AtomicCpuId, + /// The CPUs that this task can run on. + pub cpu_affinity: CpuSet, +} + +/// The priority of a real-time task. +pub const REAL_TIME_TASK_PRIORITY: u16 = 100; + +/// The priority of a task. +/// +/// Similar to Linux, a larger value represents a lower priority, +/// with a range of 0 to 139. Priorities ranging from 0 to 99 are considered real-time, +/// while those ranging from 100 to 139 are considered normal. +#[derive(Copy, Clone, Eq, Ord, PartialEq, PartialOrd)] +pub struct Priority(u16); + +impl Priority { + /// Creates a new `Priority` with the specified value. + /// + /// # Panics + /// + /// Panics if the `val` is greater than 139. + pub const fn new(val: u16) -> Self { + assert!(val <= 139); + Self(val) + } + + /// Returns a `Priority` representing the lowest priority (139). + pub const fn lowest() -> Self { + Self::new(139) + } + + /// Returns a `Priority` representing a low priority. + pub const fn low() -> Self { + Self::new(110) + } + + /// Returns a `Priority` representing a normal priority. + pub const fn normal() -> Self { + Self::new(100) + } + + /// Returns a `Priority` representing a high priority. + pub const fn high() -> Self { + Self::new(10) + } + + /// Returns a `Priority` representing the highest priority (0). + pub const fn highest() -> Self { + Self::new(0) + } + + /// Sets the value of the `Priority`. + pub const fn set(&mut self, val: u16) { + self.0 = val; + } + + /// Returns the value of the `Priority`. + pub const fn get(self) -> u16 { + self.0 + } + + /// Checks if the `Priority` is considered a real-time priority. + pub const fn is_real_time(&self) -> bool { + self.0 < REAL_TIME_TASK_PRIORITY + } +} + +/// An atomic CPUID container. +pub struct AtomicCpuId(AtomicU32); + +impl AtomicCpuId { + /// The null value of CPUID. + /// + /// An `AtomicCpuId` with `AtomicCpuId::NONE` as its inner value is empty. + const NONE: u32 = u32::MAX; + + fn new(cpu_id: u32) -> Self { + Self(AtomicU32::new(cpu_id)) + } + + /// Sets the inner value of an `AtomicCpuId` if it's empty. + /// + /// The return value is a result indicating whether the new value was written + /// and containing the previous value. + pub fn set_if_is_none(&self, cpu_id: u32) -> core::result::Result { + self.0 + .compare_exchange(Self::NONE, cpu_id, Ordering::Relaxed, Ordering::Relaxed) + } + + /// Sets the inner value of an `AtomicCpuId` to `AtomicCpuId::NONE`, i.e. makes + /// an `AtomicCpuId` empty. + pub fn set_to_none(&self) { + self.0.store(Self::NONE, Ordering::Relaxed); + } + + /// Gets the inner value of an `AtomicCpuId`. + pub fn get(&self) -> Option { + let val = self.0.load(Ordering::Relaxed); + if val == Self::NONE { + None + } else { + Some(val) + } + } +} + +impl Default for AtomicCpuId { + fn default() -> Self { + Self::new(Self::NONE) + } +} diff --git a/ostd/src/task/scheduler/mod.rs b/ostd/src/task/scheduler/mod.rs index 738c5c6a3..5baa619bc 100644 --- a/ostd/src/task/scheduler/mod.rs +++ b/ostd/src/task/scheduler/mod.rs @@ -6,12 +6,13 @@ //! and provides useful functions for controlling the execution flow. mod fifo_scheduler; +pub mod info; use core::sync::atomic::{AtomicBool, Ordering}; use spin::Once; -use super::{preempt::cpu_local, processor, task::Task}; +use super::{preempt::cpu_local, processor, Task}; use crate::{arch::timer, cpu::PinCurrentCpu, prelude::*, task::disable_preempt}; /// Injects a scheduler implementation into framework. diff --git a/ostd/src/task/task/mod.rs b/ostd/src/task/task/mod.rs deleted file mode 100644 index 8bf1cc93d..000000000 --- a/ostd/src/task/task/mod.rs +++ /dev/null @@ -1,405 +0,0 @@ -// SPDX-License-Identifier: MPL-2.0 - -// FIXME: the `intrusive_adapter` macro will generate methods without docs. -// So we temporary allow missing_docs for this module. -#![allow(missing_docs)] - -mod priority; - -use core::{ - any::Any, - cell::UnsafeCell, - sync::atomic::{AtomicU32, Ordering}, -}; - -use intrusive_collections::{intrusive_adapter, LinkedListAtomicLink}; -pub use priority::Priority; - -use super::{processor::current_task, scheduler}; -pub(crate) use crate::arch::task::{context_switch, TaskContext}; -use crate::{ - cpu::CpuSet, - mm::{kspace::KERNEL_PAGE_TABLE, FrameAllocOptions, Paddr, PageFlags, Segment, PAGE_SIZE}, - prelude::*, - user::UserSpace, -}; - -/// The kernel stack size of a task, specified in pages. -/// -/// By default, we choose a rather large stack size. -/// OSTD users can choose a smaller size by specifying -/// the `OSTD_TASK_STACK_SIZE_IN_PAGES` environment variable -/// at build time. -pub static STACK_SIZE_IN_PAGES: u32 = parse_u32_or_default( - option_env!("OSTD_TASK_STACK_SIZE_IN_PAGES"), - DEFAULT_STACK_SIZE_IN_PAGES, -); - -/// The default kernel stack size of a task, specified in pages. -pub const DEFAULT_STACK_SIZE_IN_PAGES: u32 = 128; - -const fn parse_u32_or_default(size: Option<&str>, default: u32) -> u32 { - match size { - Some(value) => parse_u32(value), - None => default, - } -} - -const fn parse_u32(input: &str) -> u32 { - let mut output: u32 = 0; - let bytes = input.as_bytes(); - let mut i = 0; - while i < bytes.len() { - let digit = (bytes[i] - b'0') as u32; - output = output * 10 + digit; - i += 1; - } - output -} - -/// Trait for manipulating the task context. -pub trait TaskContextApi { - /// Sets instruction pointer - fn set_instruction_pointer(&mut self, ip: usize); - - /// Gets instruction pointer - fn instruction_pointer(&self) -> usize; - - /// Sets stack pointer - fn set_stack_pointer(&mut self, sp: usize); - - /// Gets stack pointer - fn stack_pointer(&self) -> usize; -} - -#[derive(Debug)] -pub struct KernelStack { - segment: Segment, - has_guard_page: bool, -} - -impl KernelStack { - pub fn new() -> Result { - Ok(Self { - segment: FrameAllocOptions::new(STACK_SIZE_IN_PAGES as usize).alloc_contiguous()?, - has_guard_page: false, - }) - } - - /// Generates a kernel stack with a guard page. - /// An additional page is allocated and be regarded as a guard page, which should not be accessed. - pub fn new_with_guard_page() -> Result { - let stack_segment = - FrameAllocOptions::new(STACK_SIZE_IN_PAGES as usize + 1).alloc_contiguous()?; - // FIXME: modifying the the linear mapping is bad. - let page_table = KERNEL_PAGE_TABLE.get().unwrap(); - let guard_page_vaddr = { - let guard_page_paddr = stack_segment.start_paddr(); - crate::mm::paddr_to_vaddr(guard_page_paddr) - }; - // SAFETY: the segment allocated is not used by others so we can protect it. - unsafe { - let vaddr_range = guard_page_vaddr..guard_page_vaddr + PAGE_SIZE; - page_table - .protect_flush_tlb(&vaddr_range, |p| p.flags -= PageFlags::RW) - .unwrap(); - } - Ok(Self { - segment: stack_segment, - has_guard_page: true, - }) - } - - pub fn end_paddr(&self) -> Paddr { - self.segment.end_paddr() - } -} - -impl Drop for KernelStack { - fn drop(&mut self) { - if self.has_guard_page { - // FIXME: modifying the the linear mapping is bad. - let page_table = KERNEL_PAGE_TABLE.get().unwrap(); - let guard_page_vaddr = { - let guard_page_paddr = self.segment.start_paddr(); - crate::mm::paddr_to_vaddr(guard_page_paddr) - }; - // SAFETY: the segment allocated is not used by others so we can protect it. - unsafe { - let vaddr_range = guard_page_vaddr..guard_page_vaddr + PAGE_SIZE; - page_table - .protect_flush_tlb(&vaddr_range, |p| p.flags |= PageFlags::RW) - .unwrap(); - } - } - } -} - -/// An atomic CPUID container. -pub struct AtomicCpuId(AtomicU32); - -impl AtomicCpuId { - /// The null value of CPUID. - /// - /// An `AtomicCpuId` with `AtomicCpuId::NONE` as its inner value is empty. - const NONE: u32 = u32::MAX; - - fn new(cpu_id: u32) -> Self { - Self(AtomicU32::new(cpu_id)) - } - - /// Sets the inner value of an `AtomicCpuId` if it's empty. - /// - /// The return value is a result indicating whether the new value was written - /// and containing the previous value. - pub fn set_if_is_none(&self, cpu_id: u32) -> core::result::Result { - self.0 - .compare_exchange(Self::NONE, cpu_id, Ordering::Relaxed, Ordering::Relaxed) - } - - /// Sets the inner value of an `AtomicCpuId` to `AtomicCpuId::NONE`, i.e. makes - /// an `AtomicCpuId` empty. - pub fn set_to_none(&self) { - self.0.store(Self::NONE, Ordering::Relaxed); - } -} - -impl Default for AtomicCpuId { - fn default() -> Self { - Self::new(Self::NONE) - } -} - -/// A task that executes a function to the end. -/// -/// Each task is associated with per-task data and an optional user space. -/// If having a user space, the task can switch to the user space to -/// execute user code. Multiple tasks can share a single user space. -pub struct Task { - func: Box, - data: Box, - user_space: Option>, - ctx: UnsafeCell, - /// kernel stack, note that the top is SyscallFrame/TrapFrame - kstack: KernelStack, - link: LinkedListAtomicLink, - cpu: AtomicCpuId, - priority: Priority, - // TODO: add multiprocessor support - #[allow(dead_code)] - cpu_affinity: CpuSet, -} - -// TaskAdapter struct is implemented for building relationships between doubly linked list and Task struct -intrusive_adapter!(pub TaskAdapter = Arc: Task { link: LinkedListAtomicLink }); - -// SAFETY: `UnsafeCell` is not `Sync`. However, we only use it in `schedule()` where -// we have exclusive access to the field. -unsafe impl Sync for Task {} - -impl Task { - /// Gets the current task. - /// - /// It returns `None` if the function is called in the bootstrap context. - pub fn current() -> Option> { - current_task() - } - - pub(super) fn ctx(&self) -> &UnsafeCell { - &self.ctx - } - - /// Yields execution so that another task may be scheduled. - /// - /// Note that this method cannot be simply named "yield" as the name is - /// a Rust keyword. - pub fn yield_now() { - scheduler::yield_now() - } - - /// Runs the task. - /// - /// BUG: This method highly depends on the current scheduling policy. - pub fn run(self: &Arc) { - scheduler::run_new_task(self.clone()); - } - - /// Returns the task data. - pub fn data(&self) -> &Box { - &self.data - } - - /// Returns the user space of this task, if it has. - pub fn user_space(&self) -> Option<&Arc> { - if self.user_space.is_some() { - Some(self.user_space.as_ref().unwrap()) - } else { - None - } - } - - // Returns the cpu of this task. - pub fn cpu(&self) -> &AtomicCpuId { - &self.cpu - } - - /// Returns the priority. - pub fn priority(&self) -> Priority { - self.priority - } - - /// Exits the current task. - /// - /// The task `self` must be the task that is currently running. - /// - /// **NOTE:** If there is anything left on the stack, it will be forgotten. This behavior may - /// lead to resource leakage. - fn exit(self: Arc) -> ! { - // `current_task()` still holds a strong reference, so nothing is destroyed at this point, - // neither is the kernel stack. - drop(self); - scheduler::exit_current(); - unreachable!() - } - - /// Checks if the task has a real-time priority. - pub fn is_real_time(&self) -> bool { - self.priority.is_real_time() - } -} - -/// Options to create or spawn a new task. -pub struct TaskOptions { - func: Option>, - data: Option>, - user_space: Option>, - priority: Priority, - cpu_affinity: CpuSet, -} - -impl TaskOptions { - /// Creates a set of options for a task. - pub fn new(func: F) -> Self - where - F: Fn() + Send + Sync + 'static, - { - Self { - func: Some(Box::new(func)), - data: None, - user_space: None, - priority: Priority::normal(), - cpu_affinity: CpuSet::new_full(), - } - } - - /// Sets the function that represents the entry point of the task. - pub fn func(mut self, func: F) -> Self - where - F: Fn() + Send + Sync + 'static, - { - self.func = Some(Box::new(func)); - self - } - - /// Sets the data associated with the task. - pub fn data(mut self, data: T) -> Self - where - T: Any + Send + Sync, - { - self.data = Some(Box::new(data)); - self - } - - /// Sets the user space associated with the task. - pub fn user_space(mut self, user_space: Option>) -> Self { - self.user_space = user_space; - self - } - - /// Sets the priority of the task. - pub fn priority(mut self, priority: Priority) -> Self { - self.priority = priority; - self - } - - /// Sets the CPU affinity mask for the task. - /// - /// The `cpu_affinity` parameter represents - /// the desired set of CPUs to run the task on. - pub fn cpu_affinity(mut self, cpu_affinity: CpuSet) -> Self { - self.cpu_affinity = cpu_affinity; - self - } - - /// Builds a new task without running it immediately. - pub fn build(self) -> Result> { - /// all task will entering this function - /// this function is mean to executing the task_fn in Task - extern "C" fn kernel_task_entry() { - let current_task = current_task() - .expect("no current task, it should have current task in kernel task entry"); - current_task.func.call(()); - current_task.exit(); - } - - let mut new_task = Task { - func: self.func.unwrap(), - data: self.data.unwrap(), - user_space: self.user_space, - ctx: UnsafeCell::new(TaskContext::default()), - kstack: KernelStack::new_with_guard_page()?, - cpu: AtomicCpuId::default(), - link: LinkedListAtomicLink::new(), - priority: self.priority, - cpu_affinity: self.cpu_affinity, - }; - - let ctx = new_task.ctx.get_mut(); - ctx.set_instruction_pointer(kernel_task_entry as usize); - // We should reserve space for the return address in the stack, otherwise - // we will write across the page boundary due to the implementation of - // the context switch. - // - // According to the System V AMD64 ABI, the stack pointer should be aligned - // to at least 16 bytes. And a larger alignment is needed if larger arguments - // are passed to the function. The `kernel_task_entry` function does not - // have any arguments, so we only need to align the stack pointer to 16 bytes. - ctx.set_stack_pointer(crate::mm::paddr_to_vaddr(new_task.kstack.end_paddr() - 16)); - - Ok(Arc::new(new_task)) - } - - /// Builds a new task and run it immediately. - pub fn spawn(self) -> Result> { - let task = self.build()?; - task.run(); - Ok(task) - } -} - -#[cfg(ktest)] -mod test { - use crate::prelude::*; - - #[ktest] - fn create_task() { - #[allow(clippy::eq_op)] - let task = || { - assert_eq!(1, 1); - }; - let task_option = crate::task::TaskOptions::new(task) - .data(()) - .build() - .unwrap(); - task_option.run(); - } - - #[ktest] - fn spawn_task() { - #[allow(clippy::eq_op)] - let task = || { - assert_eq!(1, 1); - }; - let _ = crate::task::TaskOptions::new(task).data(()).spawn(); - } -} diff --git a/ostd/src/task/task/priority.rs b/ostd/src/task/task/priority.rs deleted file mode 100644 index fd5b3ebe5..000000000 --- a/ostd/src/task/task/priority.rs +++ /dev/null @@ -1,63 +0,0 @@ -// SPDX-License-Identifier: MPL-2.0 - -pub const REAL_TIME_TASK_PRIORITY: u16 = 100; - -/// The priority of a task. -/// -/// Similar to Linux, a larger value represents a lower priority, -/// with a range of 0 to 139. Priorities ranging from 0 to 99 are considered real-time, -/// while those ranging from 100 to 139 are considered normal. -#[derive(Copy, Clone, Eq, Ord, PartialEq, PartialOrd)] -pub struct Priority(u16); - -impl Priority { - /// Creates a new `Priority` with the specified value. - /// - /// # Panics - /// - /// Panics if the `val` is greater than 139. - pub const fn new(val: u16) -> Self { - assert!(val <= 139); - Self(val) - } - - /// Returns a `Priority` representing the lowest priority (139). - pub const fn lowest() -> Self { - Self::new(139) - } - - /// Returns a `Priority` representing a low priority. - pub const fn low() -> Self { - Self::new(110) - } - - /// Returns a `Priority` representing a normal priority. - pub const fn normal() -> Self { - Self::new(100) - } - - /// Returns a `Priority` representing a high priority. - pub const fn high() -> Self { - Self::new(10) - } - - /// Returns a `Priority` representing the highest priority (0). - pub const fn highest() -> Self { - Self::new(0) - } - - /// Sets the value of the `Priority`. - pub const fn set(&mut self, val: u16) { - self.0 = val; - } - - /// Returns the value of the `Priority`. - pub const fn get(self) -> u16 { - self.0 - } - - /// Checks if the `Priority` is considered a real-time priority. - pub const fn is_real_time(&self) -> bool { - self.0 < REAL_TIME_TASK_PRIORITY - } -}