mirror of
https://github.com/asterinas/asterinas.git
synced 2025-06-18 20:16:42 +00:00
Refactor ostd::task::processor
based on faster CPU-local cells
This commit is contained in:
committed by
Tate, Hongliang Tian
parent
05564ecd4f
commit
37a9590cfe
@ -26,7 +26,7 @@ pub(crate) use ostd::{
|
|||||||
#[macro_export]
|
#[macro_export]
|
||||||
macro_rules! current {
|
macro_rules! current {
|
||||||
() => {
|
() => {
|
||||||
$crate::process::current()
|
$crate::process::Process::current().unwrap()
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -23,8 +23,7 @@ pub use credentials::{credentials, credentials_mut, Credentials, Gid, Uid};
|
|||||||
pub use exit::do_exit_group;
|
pub use exit::do_exit_group;
|
||||||
pub use kill::{kill, kill_all, kill_group, tgkill};
|
pub use kill::{kill, kill_all, kill_group, tgkill};
|
||||||
pub use process::{
|
pub use process::{
|
||||||
current, ExitCode, JobControl, Pgid, Pid, Process, ProcessBuilder, ProcessGroup, Session, Sid,
|
ExitCode, JobControl, Pgid, Pid, Process, ProcessBuilder, ProcessGroup, Session, Sid, Terminal,
|
||||||
Terminal,
|
|
||||||
};
|
};
|
||||||
pub use process_filter::ProcessFilter;
|
pub use process_filter::ProcessFilter;
|
||||||
pub use process_vm::{MAX_ARGV_NUMBER, MAX_ARG_LEN, MAX_ENVP_NUMBER, MAX_ENV_LEN};
|
pub use process_vm::{MAX_ARGV_NUMBER, MAX_ARG_LEN, MAX_ENVP_NUMBER, MAX_ENV_LEN};
|
||||||
|
@ -103,6 +103,15 @@ pub struct Process {
|
|||||||
}
|
}
|
||||||
|
|
||||||
impl Process {
|
impl Process {
|
||||||
|
/// Returns the current process.
|
||||||
|
///
|
||||||
|
/// It returns `None` if:
|
||||||
|
/// - the function is called in the bootstrap context;
|
||||||
|
/// - or if the current task is not associated with a process.
|
||||||
|
pub fn current() -> Option<Arc<Process>> {
|
||||||
|
Some(Thread::current()?.as_posix_thread()?.process())
|
||||||
|
}
|
||||||
|
|
||||||
#[allow(clippy::too_many_arguments)]
|
#[allow(clippy::too_many_arguments)]
|
||||||
fn new(
|
fn new(
|
||||||
pid: Pid,
|
pid: Pid,
|
||||||
@ -636,15 +645,6 @@ impl Process {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
pub fn current() -> Arc<Process> {
|
|
||||||
let current_thread = current_thread!();
|
|
||||||
if let Some(posix_thread) = current_thread.as_posix_thread() {
|
|
||||||
posix_thread.process()
|
|
||||||
} else {
|
|
||||||
panic!("[Internal error]The current thread does not belong to a process");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
#[cfg(ktest)]
|
#[cfg(ktest)]
|
||||||
mod test {
|
mod test {
|
||||||
|
|
||||||
|
@ -50,13 +50,12 @@ impl Thread {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Returns the current thread, or `None` if the current task is not associated with a thread.
|
/// Returns the current thread.
|
||||||
///
|
///
|
||||||
/// Except for unit tests, all tasks should be associated with threads. This method is useful
|
/// This function returns `None` if the current task is not associated with
|
||||||
/// when writing code that can be called directly by unit tests. If this isn't the case,
|
/// a thread, or if called within the bootstrap context.
|
||||||
/// consider using [`current_thread!`] instead.
|
|
||||||
pub fn current() -> Option<Arc<Self>> {
|
pub fn current() -> Option<Arc<Self>> {
|
||||||
Task::current()
|
Task::current()?
|
||||||
.data()
|
.data()
|
||||||
.downcast_ref::<Weak<Thread>>()?
|
.downcast_ref::<Weak<Thread>>()?
|
||||||
.upgrade()
|
.upgrade()
|
||||||
|
@ -5,7 +5,7 @@ use core::mem;
|
|||||||
use aster_rights::Full;
|
use aster_rights::Full;
|
||||||
use ostd::{
|
use ostd::{
|
||||||
mm::{KernelSpace, VmIo, VmReader, VmWriter},
|
mm::{KernelSpace, VmIo, VmReader, VmWriter},
|
||||||
task::current_task,
|
task::Task,
|
||||||
};
|
};
|
||||||
|
|
||||||
use crate::{prelude::*, vm::vmar::Vmar};
|
use crate::{prelude::*, vm::vmar::Vmar};
|
||||||
@ -34,14 +34,8 @@ pub fn read_bytes_from_user(src: Vaddr, dest: &mut VmWriter<'_>) -> Result<()> {
|
|||||||
check_vaddr(src)?;
|
check_vaddr(src)?;
|
||||||
}
|
}
|
||||||
|
|
||||||
let current_task = current_task().ok_or(Error::with_message(
|
let current_task = Task::current().unwrap();
|
||||||
Errno::EFAULT,
|
let user_space = current_task.user_space().unwrap();
|
||||||
"the current task is missing",
|
|
||||||
))?;
|
|
||||||
let user_space = current_task.user_space().ok_or(Error::with_message(
|
|
||||||
Errno::EFAULT,
|
|
||||||
"the user space is missing",
|
|
||||||
))?;
|
|
||||||
|
|
||||||
let mut user_reader = user_space.vm_space().reader(src, copy_len)?;
|
let mut user_reader = user_space.vm_space().reader(src, copy_len)?;
|
||||||
user_reader.read_fallible(dest).map_err(|err| err.0)?;
|
user_reader.read_fallible(dest).map_err(|err| err.0)?;
|
||||||
@ -54,14 +48,8 @@ pub fn read_val_from_user<T: Pod>(src: Vaddr) -> Result<T> {
|
|||||||
check_vaddr(src)?;
|
check_vaddr(src)?;
|
||||||
}
|
}
|
||||||
|
|
||||||
let current_task = current_task().ok_or(Error::with_message(
|
let current_task = Task::current().unwrap();
|
||||||
Errno::EFAULT,
|
let user_space = current_task.user_space().unwrap();
|
||||||
"the current task is missing",
|
|
||||||
))?;
|
|
||||||
let user_space = current_task.user_space().ok_or(Error::with_message(
|
|
||||||
Errno::EFAULT,
|
|
||||||
"the user space is missing",
|
|
||||||
))?;
|
|
||||||
|
|
||||||
let mut user_reader = user_space
|
let mut user_reader = user_space
|
||||||
.vm_space()
|
.vm_space()
|
||||||
@ -88,14 +76,8 @@ pub fn write_bytes_to_user(dest: Vaddr, src: &mut VmReader<'_, KernelSpace>) ->
|
|||||||
check_vaddr(dest)?;
|
check_vaddr(dest)?;
|
||||||
}
|
}
|
||||||
|
|
||||||
let current_task = current_task().ok_or(Error::with_message(
|
let current_task = Task::current().unwrap();
|
||||||
Errno::EFAULT,
|
let user_space = current_task.user_space().unwrap();
|
||||||
"the current task is missing",
|
|
||||||
))?;
|
|
||||||
let user_space = current_task.user_space().ok_or(Error::with_message(
|
|
||||||
Errno::EFAULT,
|
|
||||||
"the user space is missing",
|
|
||||||
))?;
|
|
||||||
|
|
||||||
let mut user_writer = user_space.vm_space().writer(dest, copy_len)?;
|
let mut user_writer = user_space.vm_space().writer(dest, copy_len)?;
|
||||||
user_writer.write_fallible(src).map_err(|err| err.0)?;
|
user_writer.write_fallible(src).map_err(|err| err.0)?;
|
||||||
@ -108,14 +90,8 @@ pub fn write_val_to_user<T: Pod>(dest: Vaddr, val: &T) -> Result<()> {
|
|||||||
check_vaddr(dest)?;
|
check_vaddr(dest)?;
|
||||||
}
|
}
|
||||||
|
|
||||||
let current_task = current_task().ok_or(Error::with_message(
|
let current_task = Task::current().unwrap();
|
||||||
Errno::EFAULT,
|
let user_space = current_task.user_space().unwrap();
|
||||||
"the current task is missing",
|
|
||||||
))?;
|
|
||||||
let user_space = current_task.user_space().ok_or(Error::with_message(
|
|
||||||
Errno::EFAULT,
|
|
||||||
"the user space is missing",
|
|
||||||
))?;
|
|
||||||
|
|
||||||
let mut user_writer = user_space
|
let mut user_writer = user_space
|
||||||
.vm_space()
|
.vm_space()
|
||||||
|
@ -72,7 +72,7 @@ fn create_user_space(program: &[u8]) -> UserSpace {
|
|||||||
|
|
||||||
fn create_user_task(user_space: Arc<UserSpace>) -> Arc<Task> {
|
fn create_user_task(user_space: Arc<UserSpace>) -> Arc<Task> {
|
||||||
fn user_task() {
|
fn user_task() {
|
||||||
let current = Task::current();
|
let current = Task::current().unwrap();
|
||||||
// Switching between user-kernel space is
|
// Switching between user-kernel space is
|
||||||
// performed via the UserMode abstraction.
|
// performed via the UserMode abstraction.
|
||||||
let mut user_mode = {
|
let mut user_mode = {
|
||||||
|
@ -19,7 +19,7 @@ use crate::{
|
|||||||
page_prop::{CachePolicy, PageProperty},
|
page_prop::{CachePolicy, PageProperty},
|
||||||
PageFlags, PrivilegedPageFlags as PrivFlags, MAX_USERSPACE_VADDR, PAGE_SIZE,
|
PageFlags, PrivilegedPageFlags as PrivFlags, MAX_USERSPACE_VADDR, PAGE_SIZE,
|
||||||
},
|
},
|
||||||
task::current_task,
|
task::Task,
|
||||||
trap::call_irq_callback_functions,
|
trap::call_irq_callback_functions,
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -70,7 +70,7 @@ extern "sysv64" fn trap_handler(f: &mut TrapFrame) {
|
|||||||
|
|
||||||
/// Handles page fault from user space.
|
/// Handles page fault from user space.
|
||||||
fn handle_user_page_fault(f: &mut TrapFrame, page_fault_addr: u64) {
|
fn handle_user_page_fault(f: &mut TrapFrame, page_fault_addr: u64) {
|
||||||
let current_task = current_task().unwrap();
|
let current_task = Task::current().unwrap();
|
||||||
let user_space = current_task
|
let user_space = current_task
|
||||||
.user_space()
|
.user_space()
|
||||||
.expect("the user space is missing when a page fault from the user happens.");
|
.expect("the user space is missing when a page fault from the user happens.");
|
||||||
|
@ -80,6 +80,7 @@ cpu_local_cell! {
|
|||||||
/// It should be called only once and only on the BSP.
|
/// It should be called only once and only on the BSP.
|
||||||
pub(crate) unsafe fn early_init_bsp_local_base() {
|
pub(crate) unsafe fn early_init_bsp_local_base() {
|
||||||
let start_base_va = __cpu_local_start as usize as u64;
|
let start_base_va = __cpu_local_start as usize as u64;
|
||||||
|
|
||||||
// SAFETY: The base to be set is the start of the `.cpu_local` section,
|
// SAFETY: The base to be set is the start of the `.cpu_local` section,
|
||||||
// where accessing the CPU-local objects have defined behaviors.
|
// where accessing the CPU-local objects have defined behaviors.
|
||||||
unsafe {
|
unsafe {
|
||||||
@ -127,7 +128,7 @@ pub unsafe fn init_on_bsp() {
|
|||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
// SAFETY: the first 4 bytes is reserved for storing CPU ID.
|
// SAFETY: bytes `0:4` are reserved for storing CPU ID.
|
||||||
unsafe {
|
unsafe {
|
||||||
(ap_pages_ptr as *mut u32).write(cpu_i);
|
(ap_pages_ptr as *mut u32).write(cpu_i);
|
||||||
}
|
}
|
||||||
@ -138,11 +139,18 @@ pub unsafe fn init_on_bsp() {
|
|||||||
// section and the pointer to that static is the offset in the CPU-
|
// section and the pointer to that static is the offset in the CPU-
|
||||||
// local area. It is a `usize` so it is safe to be overwritten.
|
// local area. It is a `usize` so it is safe to be overwritten.
|
||||||
unsafe {
|
unsafe {
|
||||||
let preempt_count_offset = &PREEMPT_LOCK_COUNT as *const _ as usize;
|
let preempt_count_ptr = &PREEMPT_LOCK_COUNT as *const _ as usize;
|
||||||
|
let preempt_count_offset = preempt_count_ptr - __cpu_local_start as usize;
|
||||||
let ap_preempt_count_ptr = ap_pages_ptr.add(preempt_count_offset) as *mut usize;
|
let ap_preempt_count_ptr = ap_pages_ptr.add(preempt_count_offset) as *mut usize;
|
||||||
ap_preempt_count_ptr.write(0);
|
ap_preempt_count_ptr.write(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// SAFETY: bytes `8:16` are reserved for storing the pointer to the
|
||||||
|
// current task. We initialize it to null.
|
||||||
|
unsafe {
|
||||||
|
(ap_pages_ptr as *mut u64).add(1).write(0);
|
||||||
|
}
|
||||||
|
|
||||||
cpu_local_storages.push(ap_pages);
|
cpu_local_storages.push(ap_pages);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -48,6 +48,7 @@ use crate::{
|
|||||||
/// A `VmSpace` can also attach a page fault handler, which will be invoked to
|
/// A `VmSpace` can also attach a page fault handler, which will be invoked to
|
||||||
/// handle page faults generated from user space.
|
/// handle page faults generated from user space.
|
||||||
#[allow(clippy::type_complexity)]
|
#[allow(clippy::type_complexity)]
|
||||||
|
#[derive(Debug)]
|
||||||
pub struct VmSpace {
|
pub struct VmSpace {
|
||||||
pt: PageTable<UserMode>,
|
pt: PageTable<UserMode>,
|
||||||
page_fault_handler: Once<fn(&VmSpace, &CpuExceptionInfo) -> core::result::Result<(), ()>>,
|
page_fault_handler: Once<fn(&VmSpace, &CpuExceptionInfo) -> core::result::Result<(), ()>>,
|
||||||
|
@ -8,7 +8,6 @@
|
|||||||
pub type Result<T> = core::result::Result<T, crate::error::Error>;
|
pub type Result<T> = core::result::Result<T, crate::error::Error>;
|
||||||
|
|
||||||
pub(crate) use alloc::{boxed::Box, sync::Arc, vec::Vec};
|
pub(crate) use alloc::{boxed::Box, sync::Arc, vec::Vec};
|
||||||
pub(crate) use core::any::Any;
|
|
||||||
|
|
||||||
#[cfg(ktest)]
|
#[cfg(ktest)]
|
||||||
pub use ostd_macros::ktest;
|
pub use ostd_macros::ktest;
|
||||||
|
@ -4,7 +4,7 @@ use alloc::{collections::VecDeque, sync::Arc};
|
|||||||
use core::sync::atomic::{AtomicBool, AtomicU32, Ordering};
|
use core::sync::atomic::{AtomicBool, AtomicU32, Ordering};
|
||||||
|
|
||||||
use super::SpinLock;
|
use super::SpinLock;
|
||||||
use crate::task::{add_task, current_task, schedule, Task, TaskStatus};
|
use crate::task::{add_task, schedule, Task, TaskStatus};
|
||||||
|
|
||||||
// # Explanation on the memory orders
|
// # Explanation on the memory orders
|
||||||
//
|
//
|
||||||
@ -209,7 +209,7 @@ impl Waiter {
|
|||||||
pub fn new_pair() -> (Self, Arc<Waker>) {
|
pub fn new_pair() -> (Self, Arc<Waker>) {
|
||||||
let waker = Arc::new(Waker {
|
let waker = Arc::new(Waker {
|
||||||
has_woken: AtomicBool::new(false),
|
has_woken: AtomicBool::new(false),
|
||||||
task: current_task().unwrap(),
|
task: Task::current().unwrap(),
|
||||||
});
|
});
|
||||||
let waiter = Self {
|
let waiter = Self {
|
||||||
waker: waker.clone(),
|
waker: waker.clone(),
|
||||||
|
@ -10,7 +10,7 @@ mod task;
|
|||||||
|
|
||||||
pub use self::{
|
pub use self::{
|
||||||
priority::Priority,
|
priority::Priority,
|
||||||
processor::{current_task, disable_preempt, preempt, schedule, DisablePreemptGuard},
|
processor::{disable_preempt, preempt, schedule, DisablePreemptGuard},
|
||||||
scheduler::{add_task, set_scheduler, FifoScheduler, Scheduler},
|
scheduler::{add_task, set_scheduler, FifoScheduler, Scheduler},
|
||||||
task::{Task, TaskAdapter, TaskContextApi, TaskOptions, TaskStatus},
|
task::{Task, TaskAdapter, TaskContextApi, TaskOptions, TaskStatus},
|
||||||
};
|
};
|
||||||
|
@ -7,7 +7,7 @@ pub const REAL_TIME_TASK_PRIORITY: u16 = 100;
|
|||||||
/// Similar to Linux, a larger value represents a lower priority,
|
/// 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,
|
/// 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.
|
/// while those ranging from 100 to 139 are considered normal.
|
||||||
#[derive(Copy, Clone)]
|
#[derive(Copy, Clone, Debug)]
|
||||||
pub struct Priority(u16);
|
pub struct Priority(u16);
|
||||||
|
|
||||||
impl Priority {
|
impl Priority {
|
||||||
|
@ -1,59 +1,40 @@
|
|||||||
// SPDX-License-Identifier: MPL-2.0
|
// SPDX-License-Identifier: MPL-2.0
|
||||||
|
|
||||||
use alloc::sync::Arc;
|
use alloc::sync::Arc;
|
||||||
use core::cell::RefCell;
|
|
||||||
|
|
||||||
use super::{
|
use super::{
|
||||||
scheduler::{fetch_task, GLOBAL_SCHEDULER},
|
scheduler::{fetch_task, GLOBAL_SCHEDULER},
|
||||||
task::{context_switch, TaskContext},
|
task::{context_switch, TaskContext},
|
||||||
Task, TaskStatus,
|
Task, TaskStatus,
|
||||||
};
|
};
|
||||||
use crate::{cpu::local::PREEMPT_LOCK_COUNT, cpu_local};
|
use crate::{cpu::local::PREEMPT_LOCK_COUNT, cpu_local_cell};
|
||||||
|
|
||||||
pub struct Processor {
|
cpu_local_cell! {
|
||||||
current: Option<Arc<Task>>,
|
/// The `Arc<Task>` (casted by [`Arc::into_raw`]) that is the current task.
|
||||||
/// A temporary variable used in [`switch_to_task`] to avoid dropping `current` while running
|
static CURRENT_TASK_PTR: *const Task = core::ptr::null();
|
||||||
/// as `current`.
|
/// The previous task on the processor before switching to the current task.
|
||||||
prev_task: Option<Arc<Task>>,
|
/// It is used for delayed resource release since it would be the current
|
||||||
idle_task_ctx: TaskContext,
|
/// task's job to recycle the previous resources.
|
||||||
|
static PREVIOUS_TASK_PTR: *const Task = core::ptr::null();
|
||||||
|
/// An unsafe cell to store the context of the bootstrap code.
|
||||||
|
static BOOTSTRAP_CONTEXT: TaskContext = TaskContext::new();
|
||||||
}
|
}
|
||||||
|
|
||||||
impl Processor {
|
/// Retrieves a reference to the current task running on the processor.
|
||||||
pub const fn new() -> Self {
|
///
|
||||||
Self {
|
/// It returns `None` if the function is called in the bootstrap context.
|
||||||
current: None,
|
pub(super) fn current_task() -> Option<Arc<Task>> {
|
||||||
prev_task: None,
|
let ptr = CURRENT_TASK_PTR.load();
|
||||||
idle_task_ctx: TaskContext::new(),
|
if ptr.is_null() {
|
||||||
}
|
return None;
|
||||||
}
|
}
|
||||||
fn get_idle_task_ctx_ptr(&mut self) -> *mut TaskContext {
|
// SAFETY: The pointer is set by `switch_to_task` and is guaranteed to be
|
||||||
&mut self.idle_task_ctx as *mut _
|
// built with `Arc::into_raw`.
|
||||||
}
|
let restored = unsafe { Arc::from_raw(ptr) };
|
||||||
pub fn take_current(&mut self) -> Option<Arc<Task>> {
|
// To let the `CURRENT_TASK_PTR` still own the task, we clone and forget it
|
||||||
self.current.take()
|
// to increment the reference count.
|
||||||
}
|
let _ = core::mem::ManuallyDrop::new(restored.clone());
|
||||||
pub fn current(&self) -> Option<Arc<Task>> {
|
Some(restored)
|
||||||
self.current.as_ref().map(Arc::clone)
|
|
||||||
}
|
|
||||||
pub fn set_current_task(&mut self, task: Arc<Task>) {
|
|
||||||
self.current = Some(task.clone());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
cpu_local! {
|
|
||||||
static PROCESSOR: RefCell<Processor> = RefCell::new(Processor::new());
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Retrieves the current task running on the processor.
|
|
||||||
pub fn current_task() -> Option<Arc<Task>> {
|
|
||||||
PROCESSOR.borrow_irq_disabled().borrow().current()
|
|
||||||
}
|
|
||||||
|
|
||||||
pub(crate) fn get_idle_task_ctx_ptr() -> *mut TaskContext {
|
|
||||||
PROCESSOR
|
|
||||||
.borrow_irq_disabled()
|
|
||||||
.borrow_mut()
|
|
||||||
.get_idle_task_ctx_ptr()
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Calls this function to switch to other task by using GLOBAL_SCHEDULER
|
/// Calls this function to switch to other task by using GLOBAL_SCHEDULER
|
||||||
@ -85,11 +66,11 @@ pub fn preempt(task: &Arc<Task>) {
|
|||||||
|
|
||||||
/// Calls this function to switch to other task
|
/// Calls this function to switch to other task
|
||||||
///
|
///
|
||||||
/// if current task is none, then it will use the default task context and it will not return to this function again
|
/// If current task is none, then it will use the default task context and it
|
||||||
|
/// will not return to this function again.
|
||||||
///
|
///
|
||||||
/// if current task status is exit, then it will not add to the scheduler
|
/// If the current task's status not [`TaskStatus::Runnable`], it will not be
|
||||||
///
|
/// added to the scheduler.
|
||||||
/// before context switch, current task will switch to the next task
|
|
||||||
fn switch_to_task(next_task: Arc<Task>) {
|
fn switch_to_task(next_task: Arc<Task>) {
|
||||||
let preemt_lock_count = PREEMPT_LOCK_COUNT.load();
|
let preemt_lock_count = PREEMPT_LOCK_COUNT.load();
|
||||||
if preemt_lock_count != 0 {
|
if preemt_lock_count != 0 {
|
||||||
@ -99,23 +80,34 @@ fn switch_to_task(next_task: Arc<Task>) {
|
|||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
let current_task_ctx_ptr = match current_task() {
|
let irq_guard = crate::trap::disable_local();
|
||||||
None => get_idle_task_ctx_ptr(),
|
|
||||||
Some(current_task) => {
|
|
||||||
let ctx_ptr = current_task.ctx().get();
|
|
||||||
|
|
||||||
let mut task_inner = current_task.inner_exclusive_access();
|
let current_task_ptr = CURRENT_TASK_PTR.load();
|
||||||
|
|
||||||
debug_assert_ne!(task_inner.task_status, TaskStatus::Sleeping);
|
let current_task_ctx_ptr = if current_task_ptr.is_null() {
|
||||||
if task_inner.task_status == TaskStatus::Runnable {
|
// SAFETY: Interrupts are disabled, so the pointer is safe to be fetched.
|
||||||
drop(task_inner);
|
unsafe { BOOTSTRAP_CONTEXT.as_ptr_mut() }
|
||||||
GLOBAL_SCHEDULER.lock_irq_disabled().enqueue(current_task);
|
} else {
|
||||||
} else if task_inner.task_status == TaskStatus::Sleepy {
|
// SAFETY: The pointer is not NULL and set as the current task.
|
||||||
task_inner.task_status = TaskStatus::Sleeping;
|
let cur_task_arc = unsafe {
|
||||||
}
|
let restored = Arc::from_raw(current_task_ptr);
|
||||||
|
let _ = core::mem::ManuallyDrop::new(restored.clone());
|
||||||
|
restored
|
||||||
|
};
|
||||||
|
|
||||||
ctx_ptr
|
let ctx_ptr = cur_task_arc.ctx().get();
|
||||||
|
|
||||||
|
let mut task_inner = cur_task_arc.inner_exclusive_access();
|
||||||
|
|
||||||
|
debug_assert_ne!(task_inner.task_status, TaskStatus::Sleeping);
|
||||||
|
if task_inner.task_status == TaskStatus::Runnable {
|
||||||
|
drop(task_inner);
|
||||||
|
GLOBAL_SCHEDULER.lock().enqueue(cur_task_arc);
|
||||||
|
} else if task_inner.task_status == TaskStatus::Sleepy {
|
||||||
|
task_inner.task_status = TaskStatus::Sleeping;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
ctx_ptr
|
||||||
};
|
};
|
||||||
|
|
||||||
let next_task_ctx_ptr = next_task.ctx().get().cast_const();
|
let next_task_ctx_ptr = next_task.ctx().get().cast_const();
|
||||||
@ -125,17 +117,22 @@ fn switch_to_task(next_task: Arc<Task>) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Change the current task to the next task.
|
// Change the current task to the next task.
|
||||||
{
|
//
|
||||||
let processor_guard = PROCESSOR.borrow_irq_disabled();
|
// We cannot directly drop `current` at this point. Since we are running as
|
||||||
let mut processor = processor_guard.borrow_mut();
|
// `current`, we must avoid dropping `current`. Otherwise, the kernel stack
|
||||||
|
// may be unmapped, leading to instant failure.
|
||||||
// We cannot directly overwrite `current` at this point. Since we are running as `current`,
|
let old_prev = PREVIOUS_TASK_PTR.load();
|
||||||
// we must avoid dropping `current`. Otherwise, the kernel stack may be unmapped, leading
|
PREVIOUS_TASK_PTR.store(current_task_ptr);
|
||||||
// to soundness problems.
|
CURRENT_TASK_PTR.store(Arc::into_raw(next_task));
|
||||||
let old_current = processor.current.replace(next_task);
|
// Drop the old-previously running task.
|
||||||
processor.prev_task = old_current;
|
if !old_prev.is_null() {
|
||||||
|
// SAFETY: The pointer is set by `switch_to_task` and is guaranteed to be
|
||||||
|
// built with `Arc::into_raw`.
|
||||||
|
drop(unsafe { Arc::from_raw(old_prev) });
|
||||||
}
|
}
|
||||||
|
|
||||||
|
drop(irq_guard);
|
||||||
|
|
||||||
// SAFETY:
|
// SAFETY:
|
||||||
// 1. `ctx` is only used in `schedule()`. We have exclusive access to both the current task
|
// 1. `ctx` is only used in `schedule()`. We have exclusive access to both the current task
|
||||||
// context and the next task context.
|
// context and the next task context.
|
||||||
|
@ -3,9 +3,9 @@
|
|||||||
// FIXME: the `intrusive_adapter` macro will generate methods without docs.
|
// FIXME: the `intrusive_adapter` macro will generate methods without docs.
|
||||||
// So we temporary allow missing_docs for this module.
|
// So we temporary allow missing_docs for this module.
|
||||||
#![allow(missing_docs)]
|
#![allow(missing_docs)]
|
||||||
#![allow(dead_code)]
|
|
||||||
|
|
||||||
use core::cell::UnsafeCell;
|
use alloc::{boxed::Box, sync::Arc};
|
||||||
|
use core::{any::Any, cell::UnsafeCell};
|
||||||
|
|
||||||
use intrusive_collections::{intrusive_adapter, LinkedListAtomicLink};
|
use intrusive_collections::{intrusive_adapter, LinkedListAtomicLink};
|
||||||
|
|
||||||
@ -18,7 +18,7 @@ pub(crate) use crate::arch::task::{context_switch, TaskContext};
|
|||||||
use crate::{
|
use crate::{
|
||||||
arch::mm::tlb_flush_addr_range,
|
arch::mm::tlb_flush_addr_range,
|
||||||
cpu::CpuSet,
|
cpu::CpuSet,
|
||||||
mm::{kspace::KERNEL_PAGE_TABLE, FrameAllocOptions, PageFlags, Segment, PAGE_SIZE},
|
mm::{kspace::KERNEL_PAGE_TABLE, FrameAllocOptions, Paddr, PageFlags, Segment, PAGE_SIZE},
|
||||||
prelude::*,
|
prelude::*,
|
||||||
sync::{SpinLock, SpinLockGuard},
|
sync::{SpinLock, SpinLockGuard},
|
||||||
user::UserSpace,
|
user::UserSpace,
|
||||||
@ -41,6 +41,7 @@ pub trait TaskContextApi {
|
|||||||
fn stack_pointer(&self) -> usize;
|
fn stack_pointer(&self) -> usize;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
#[derive(Debug)]
|
||||||
pub struct KernelStack {
|
pub struct KernelStack {
|
||||||
segment: Segment,
|
segment: Segment,
|
||||||
has_guard_page: bool,
|
has_guard_page: bool,
|
||||||
@ -121,6 +122,7 @@ pub struct Task {
|
|||||||
link: LinkedListAtomicLink,
|
link: LinkedListAtomicLink,
|
||||||
priority: Priority,
|
priority: Priority,
|
||||||
// TODO: add multiprocessor support
|
// TODO: add multiprocessor support
|
||||||
|
#[allow(dead_code)]
|
||||||
cpu_affinity: CpuSet,
|
cpu_affinity: CpuSet,
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -131,14 +133,17 @@ intrusive_adapter!(pub TaskAdapter = Arc<Task>: Task { link: LinkedListAtomicLin
|
|||||||
// we have exclusive access to the field.
|
// we have exclusive access to the field.
|
||||||
unsafe impl Sync for Task {}
|
unsafe impl Sync for Task {}
|
||||||
|
|
||||||
|
#[derive(Debug)]
|
||||||
pub(crate) struct TaskInner {
|
pub(crate) struct TaskInner {
|
||||||
pub task_status: TaskStatus,
|
pub task_status: TaskStatus,
|
||||||
}
|
}
|
||||||
|
|
||||||
impl Task {
|
impl Task {
|
||||||
/// Gets the current task.
|
/// Gets the current task.
|
||||||
pub fn current() -> Arc<Task> {
|
///
|
||||||
current_task().unwrap()
|
/// It returns `None` if the function is called in the bootstrap context.
|
||||||
|
pub fn current() -> Option<Arc<Task>> {
|
||||||
|
current_task()
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Gets inner
|
/// Gets inner
|
||||||
|
@ -12,6 +12,7 @@ use crate::{cpu::UserContext, mm::VmSpace, prelude::*, task::Task};
|
|||||||
///
|
///
|
||||||
/// Each user space has a VM address space and allows a task to execute in
|
/// Each user space has a VM address space and allows a task to execute in
|
||||||
/// user mode.
|
/// user mode.
|
||||||
|
#[derive(Debug)]
|
||||||
pub struct UserSpace {
|
pub struct UserSpace {
|
||||||
/// vm space
|
/// vm space
|
||||||
vm_space: Arc<VmSpace>,
|
vm_space: Arc<VmSpace>,
|
||||||
@ -94,7 +95,7 @@ pub trait UserContextApi {
|
|||||||
///
|
///
|
||||||
/// let current = Task::current();
|
/// let current = Task::current();
|
||||||
/// let user_space = current.user_space()
|
/// let user_space = current.user_space()
|
||||||
/// .expect("the current task is associated with a user space");
|
/// .expect("the current task is not associated with a user space");
|
||||||
/// let mut user_mode = user_space.user_mode();
|
/// let mut user_mode = user_space.user_mode();
|
||||||
/// loop {
|
/// loop {
|
||||||
/// // Execute in the user space until some interesting events occur.
|
/// // Execute in the user space until some interesting events occur.
|
||||||
@ -108,14 +109,14 @@ pub struct UserMode<'a> {
|
|||||||
context: UserContext,
|
context: UserContext,
|
||||||
}
|
}
|
||||||
|
|
||||||
// An instance of `UserMode` is bound to the current task. So it cannot be
|
// An instance of `UserMode` is bound to the current task. So it cannot be [`Send`].
|
||||||
impl<'a> !Send for UserMode<'a> {}
|
impl<'a> !Send for UserMode<'a> {}
|
||||||
|
|
||||||
impl<'a> UserMode<'a> {
|
impl<'a> UserMode<'a> {
|
||||||
/// Creates a new `UserMode`.
|
/// Creates a new `UserMode`.
|
||||||
pub fn new(user_space: &'a Arc<UserSpace>) -> Self {
|
pub fn new(user_space: &'a Arc<UserSpace>) -> Self {
|
||||||
Self {
|
Self {
|
||||||
current: Task::current(),
|
current: Task::current().unwrap(),
|
||||||
user_space,
|
user_space,
|
||||||
context: user_space.init_ctx,
|
context: user_space.init_ctx,
|
||||||
}
|
}
|
||||||
@ -136,7 +137,7 @@ impl<'a> UserMode<'a> {
|
|||||||
where
|
where
|
||||||
F: FnMut() -> bool,
|
F: FnMut() -> bool,
|
||||||
{
|
{
|
||||||
debug_assert!(Arc::ptr_eq(&self.current, &Task::current()));
|
debug_assert!(Arc::ptr_eq(&self.current, &Task::current().unwrap()));
|
||||||
self.context.execute(has_kernel_event)
|
self.context.execute(has_kernel_event)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Reference in New Issue
Block a user