Add mutable functionality to the process VMAR

This commit is contained in:
Chen Chengjun
2025-03-17 10:53:56 +08:00
committed by Tate, Hongliang Tian
parent 248b24fb4e
commit 2a20f6b59a
15 changed files with 119 additions and 63 deletions

View File

@ -2,11 +2,12 @@
//! The context that can be accessed from the current task, thread or process.
use core::mem;
use core::{cell::Ref, mem};
use aster_rights::Full;
use ostd::{
mm::{Fallible, Infallible, VmReader, VmSpace, VmWriter},
task::Task,
mm::{Fallible, Infallible, VmReader, VmWriter},
task::{CurrentTask, Task},
};
use crate::{
@ -16,6 +17,7 @@ use crate::{
Process,
},
thread::Thread,
vm::vmar::Vmar,
};
/// The context that can be accessed from the current POSIX thread.
@ -31,14 +33,14 @@ pub struct Context<'a> {
impl Context<'_> {
/// Gets the userspace of the current task.
pub fn user_space(&self) -> CurrentUserSpace {
CurrentUserSpace::new(self.task)
CurrentUserSpace(self.thread_local.root_vmar().borrow())
}
}
/// The user's memory space of the current task.
///
/// It provides methods to read from or write to the user space efficiently.
pub struct CurrentUserSpace<'a>(&'a VmSpace);
pub struct CurrentUserSpace<'a>(Ref<'a, Option<Vmar<Full>>>);
/// Gets the [`CurrentUserSpace`] from the current task.
///
@ -52,40 +54,39 @@ macro_rules! current_userspace {
}
impl<'a> CurrentUserSpace<'a> {
/// Creates a new `CurrentUserSpace` from the specified task.
///
/// This method is _not_ recommended for use, as it does not verify whether the provided
/// `task` is the current task in release builds.
/// Creates a new `CurrentUserSpace` from the current task.
///
/// If you have access to a [`Context`], it is preferable to call [`Context::user_space`].
///
/// Otherwise, you can use the `current_userspace` macro
/// to obtain an instance of `CurrentUserSpace` if it will only be used once.
pub fn new(current_task: &'a CurrentTask) -> Self {
let thread_local = current_task.as_thread_local().unwrap();
let vmar_ref = thread_local.root_vmar().borrow();
Self(vmar_ref)
}
/// Returns the root `Vmar` of the current userspace.
///
/// # Panics
///
/// This method will panic in debug builds if the specified `task` is not the current task.
pub fn new(task: &'a Task) -> Self {
let user_space = task.user_space().unwrap();
debug_assert!(Arc::ptr_eq(
task.user_space().unwrap(),
Task::current().unwrap().user_space().unwrap()
));
Self(user_space.vm_space())
/// This method will panic if the current process has cleared its `Vmar`.
pub fn root_vmar(&self) -> &Vmar<Full> {
self.0.as_ref().unwrap()
}
/// Creates a reader to read data from the user space of the current task.
///
/// Returns `Err` if the `vaddr` and `len` do not represent a user space memory range.
pub fn reader(&self, vaddr: Vaddr, len: usize) -> Result<VmReader<'_, Fallible>> {
Ok(self.0.reader(vaddr, len)?)
Ok(self.root_vmar().vm_space().reader(vaddr, len)?)
}
/// Creates a writer to write data into the user space.
///
/// Returns `Err` if the `vaddr` and `len` do not represent a user space memory range.
pub fn writer(&self, vaddr: Vaddr, len: usize) -> Result<VmWriter<'_, Fallible>> {
Ok(self.0.writer(vaddr, len)?)
Ok(self.root_vmar().vm_space().writer(vaddr, len)?)
}
/// Reads bytes into the destination `VmWriter` from the user space of the

View File

@ -53,7 +53,7 @@ pub(crate) use crate::{
context::{Context, CurrentUserSpace, ReadCString},
current, current_thread,
error::{Errno, Error},
process::signal::Pause,
process::{posix_thread::AsThreadLocal, signal::Pause},
time::{wait::WaitTimeout, Clock},
};
pub(crate) type Result<T> = core::result::Result<T, Error>;

View File

@ -24,6 +24,8 @@ pub(super) fn exit_process(thread_local: &ThreadLocal, current_process: &Process
move_children_to_reaper_process(current_process);
send_child_death_signal(current_process);
current_process.lock_root_vmar().clear();
}
/// Sends parent-death signals to the children.

View File

@ -120,6 +120,10 @@ impl PosixThreadBuilder {
let fs = fs.unwrap_or_else(|| Arc::new(ThreadFsInfo::default()));
Arc::new_cyclic(|weak_task| {
let root_vmar = process
.upgrade()
.map(|process| process.lock_root_vmar().get().dup().unwrap());
let posix_thread = {
let prof_clock = ProfClock::new();
let virtual_timer_manager = TimerManager::new(prof_clock.user_clock().clone());
@ -149,7 +153,8 @@ impl PosixThreadBuilder {
sched_policy,
));
let thread_local = ThreadLocal::new(set_child_tid, clear_child_tid, file_table);
let thread_local =
ThreadLocal::new(set_child_tid, clear_child_tid, root_vmar, file_table);
thread_table::add_thread(tid, thread.clone());
task::create_new_user_task(user_ctx, thread, thread_local)

View File

@ -79,6 +79,8 @@ fn exit_internal(term_status: TermStatus, is_exiting_group: bool) {
thread_table::remove_thread(posix_thread.tid());
}
*thread_local.root_vmar().borrow_mut() = None;
if is_last_thread {
exit_process(thread_local, &posix_process);
}

View File

@ -2,10 +2,11 @@
use core::cell::{Cell, RefCell};
use aster_rights::Full;
use ostd::{mm::Vaddr, sync::RwArc, task::CurrentTask};
use super::RobustListHead;
use crate::{fs::file_table::FileTable, process::signal::SigStack};
use crate::{fs::file_table::FileTable, process::signal::SigStack, vm::vmar::Vmar};
/// Local data for a POSIX thread.
pub struct ThreadLocal {
@ -14,6 +15,9 @@ pub struct ThreadLocal {
set_child_tid: Cell<Vaddr>,
clear_child_tid: Cell<Vaddr>,
// Virtual memory address regions.
root_vmar: RefCell<Option<Vmar<Full>>>,
// Robust futexes.
// https://man7.org/linux/man-pages/man2/get_robust_list.2.html
robust_list: RefCell<Option<RobustListHead>>,
@ -34,11 +38,13 @@ impl ThreadLocal {
pub(super) fn new(
set_child_tid: Vaddr,
clear_child_tid: Vaddr,
root_vmar: Option<Vmar<Full>>,
file_table: RwArc<FileTable>,
) -> Self {
Self {
set_child_tid: Cell::new(set_child_tid),
clear_child_tid: Cell::new(clear_child_tid),
root_vmar: RefCell::new(root_vmar),
robust_list: RefCell::new(None),
file_table: RefCell::new(file_table),
sig_context: Cell::new(None),
@ -54,6 +60,10 @@ impl ThreadLocal {
&self.clear_child_tid
}
pub fn root_vmar(&self) -> &RefCell<Option<Vmar<Full>>> {
&self.root_vmar
}
pub fn robust_list(&self) -> &RefCell<Option<RobustListHead>> {
&self.robust_list
}

View File

@ -6,7 +6,7 @@ use self::timer_manager::PosixTimerManager;
use super::{
posix_thread::{allocate_posix_tid, AsPosixThread},
process_table,
process_vm::{Heap, InitStackReader, ProcessVm},
process_vm::{Heap, InitStackReader, ProcessVm, ProcessVmarGuard},
rlimit::ResourceLimits,
signal::{
sig_disposition::SigDispositions,
@ -23,7 +23,6 @@ use crate::{
sched::{AtomicNice, Nice},
thread::{AsThread, Thread},
time::clocks::ProfClock,
vm::vmar::Vmar,
};
mod builder;
@ -33,7 +32,6 @@ mod session;
mod terminal;
mod timer_manager;
use aster_rights::Full;
use atomic_integer_wrapper::define_atomic_version_of_integer_like_type;
pub use builder::ProcessBuilder;
pub use job_control::JobControl;
@ -603,8 +601,8 @@ impl Process {
&self.process_vm
}
pub fn root_vmar(&self) -> &Vmar<Full> {
self.process_vm.root_vmar()
pub fn lock_root_vmar(&self) -> ProcessVmarGuard {
self.process_vm.lock_root_vmar()
}
pub fn heap(&self) -> &Heap {

View File

@ -20,9 +20,10 @@ use core::{
use align_ext::AlignExt;
use aster_rights::Full;
use ostd::mm::{vm_space::VmItem, UntypedMem, VmIo, VmSpace, MAX_USERSPACE_VADDR};
use ostd::mm::{vm_space::VmItem, UntypedMem, VmIo, MAX_USERSPACE_VADDR};
use self::aux_vec::{AuxKey, AuxVec};
use super::ProcessVmarGuard;
use crate::{
prelude::*,
util::random::getrandom,
@ -191,11 +192,11 @@ impl InitStack {
/// Constructs a reader to parse the content of an `InitStack`.
/// The `InitStack` should only be read after initialized
pub(super) fn reader<'a>(&self, vm_space: &'a Arc<VmSpace>) -> InitStackReader<'a> {
pub(super) fn reader<'a>(&self, vmar: ProcessVmarGuard<'a>) -> InitStackReader<'a> {
debug_assert!(self.is_initialized());
InitStackReader {
base: self.pos(),
vm_space,
vmar,
map_addr: self.initial_top - self.max_size,
}
}
@ -373,7 +374,7 @@ fn generate_random_for_aux_vec() -> [u8; 16] {
/// A reader to parse the content of an `InitStack`.
pub struct InitStackReader<'a> {
base: Vaddr,
vm_space: &'a Arc<VmSpace>,
vmar: ProcessVmarGuard<'a>,
/// The mapping address of the `InitStack`.
map_addr: usize,
}
@ -384,9 +385,8 @@ impl InitStackReader<'_> {
let stack_base = self.init_stack_bottom();
let page_base_addr = stack_base.align_down(PAGE_SIZE);
let mut cursor = self
.vm_space
.cursor(&(page_base_addr..page_base_addr + PAGE_SIZE))?;
let vm_space = self.vmar.get().vm_space();
let mut cursor = vm_space.cursor(&(page_base_addr..page_base_addr + PAGE_SIZE))?;
let VmItem::Mapped { frame, .. } = cursor.query()? else {
return_errno_with_message!(Errno::EACCES, "Page not accessible");
};
@ -408,9 +408,9 @@ impl InitStackReader<'_> {
let mut argv = Vec::with_capacity(argc);
let page_base_addr = read_offset.align_down(PAGE_SIZE);
let mut cursor = self
.vm_space
.cursor(&(page_base_addr..page_base_addr + PAGE_SIZE))?;
let vm_space = self.vmar.get().vm_space();
let mut cursor = vm_space.cursor(&(page_base_addr..page_base_addr + PAGE_SIZE))?;
let VmItem::Mapped { frame, .. } = cursor.query()? else {
return_errno_with_message!(Errno::EACCES, "Page not accessible");
};
@ -450,9 +450,9 @@ impl InitStackReader<'_> {
let mut envp = Vec::new();
let page_base_addr = read_offset.align_down(PAGE_SIZE);
let mut cursor = self
.vm_space
.cursor(&(page_base_addr..page_base_addr + PAGE_SIZE))?;
let vm_space = self.vmar.get().vm_space();
let mut cursor = vm_space.cursor(&(page_base_addr..page_base_addr + PAGE_SIZE))?;
let VmItem::Mapped { frame, .. } = cursor.query()? else {
return_errno_with_message!(Errno::EACCES, "Page not accessible");
};

View File

@ -14,6 +14,7 @@ mod init_stack;
use aster_rights::Full;
pub use heap::Heap;
use ostd::sync::MutexGuard;
pub use self::{
heap::USER_HEAP_SIZE_LIMIT,
@ -61,17 +62,43 @@ use crate::{prelude::*, vm::vmar::Vmar};
* (low address)
*/
// The process user space virtual memory
/// The process user space virtual memory
pub struct ProcessVm {
root_vmar: Vmar<Full>,
root_vmar: Mutex<Option<Vmar<Full>>>,
init_stack: InitStack,
heap: Heap,
}
/// A guard to the [`Vmar`] used by a process.
///
/// It is bound to a [`ProcessVm`] and can only be obtained from
/// the [`ProcessVm::lock_root_vmar`] method.
pub struct ProcessVmarGuard<'a> {
inner: MutexGuard<'a, Option<Vmar<Full>>>,
}
impl ProcessVmarGuard<'_> {
/// Gets a reference to the process VMAR.
pub fn get(&self) -> &Vmar<Full> {
self.inner.as_ref().unwrap()
}
/// Sets a new VMAR for the binding process.
pub(super) fn set_new_vmar(&mut self, new_vmar: Vmar<Full>) {
*self.inner = Some(new_vmar);
}
/// Clears the VMAR of the binding process.
pub(super) fn clear(&mut self) {
*self.inner = None;
}
}
impl Clone for ProcessVm {
fn clone(&self) -> Self {
let root_vmar = self.lock_root_vmar();
Self {
root_vmar: self.root_vmar.dup().unwrap(),
root_vmar: Mutex::new(Some(root_vmar.get().dup().unwrap())),
init_stack: self.init_stack.clone(),
heap: self.heap.clone(),
}
@ -86,7 +113,7 @@ impl ProcessVm {
let heap = Heap::new();
heap.alloc_and_map_vm(&root_vmar).unwrap();
Self {
root_vmar,
root_vmar: Mutex::new(Some(root_vmar)),
heap,
init_stack,
}
@ -96,7 +123,8 @@ impl ProcessVm {
///
/// The returned `ProcessVm` will have a forked `Vmar`.
pub fn fork_from(other: &ProcessVm) -> Result<Self> {
let root_vmar = Vmar::<Full>::fork_from(&other.root_vmar)?;
let process_vmar = other.lock_root_vmar();
let root_vmar = Mutex::new(Some(Vmar::<Full>::fork_from(process_vmar.get())?));
Ok(Self {
root_vmar,
heap: other.heap.clone(),
@ -104,14 +132,17 @@ impl ProcessVm {
})
}
pub fn root_vmar(&self) -> &Vmar<Full> {
&self.root_vmar
/// Locks the root VMAR and gets a guard to it.
pub fn lock_root_vmar(&self) -> ProcessVmarGuard {
ProcessVmarGuard {
inner: self.root_vmar.lock(),
}
}
/// Returns a reader for reading contents from
/// the `InitStack`.
pub fn init_stack_reader(&self) -> InitStackReader {
self.init_stack.reader(self.root_vmar().vm_space())
self.init_stack.reader(self.lock_root_vmar())
}
/// Returns the top address of the user stack.
@ -125,8 +156,9 @@ impl ProcessVm {
envp: Vec<CString>,
aux_vec: AuxVec,
) -> Result<()> {
let root_vmar: ProcessVmarGuard<'_> = self.lock_root_vmar();
self.init_stack
.map_and_write(self.root_vmar(), argv, envp, aux_vec)
.map_and_write(root_vmar.get(), argv, envp, aux_vec)
}
pub(super) fn heap(&self) -> &Heap {
@ -135,7 +167,8 @@ impl ProcessVm {
/// Clears existing mappings and then maps stack and heap vmo.
pub(super) fn clear_and_map(&self) {
self.root_vmar.clear().unwrap();
self.heap.alloc_and_map_vm(&self.root_vmar).unwrap();
let root_vmar = self.lock_root_vmar();
root_vmar.get().clear().unwrap();
self.heap.alloc_and_map_vm(&root_vmar.get()).unwrap();
}
}

View File

@ -66,7 +66,7 @@ pub fn load_elf_to_vm(
// the process cannot return to user space again,
// so `Vmar::clear` and `do_exit_group` are called here.
// FIXME: sending a fault signal is an alternative approach.
process_vm.root_vmar().clear().unwrap();
process_vm.lock_root_vmar().get().clear().unwrap();
// FIXME: `current` macro will be used in `do_exit_group`.
// if the macro is used when creating the init process,
@ -115,7 +115,8 @@ fn init_and_map_vmos(
parsed_elf: &Elf,
elf_file: &Dentry,
) -> Result<(Vaddr, AuxVec)> {
let root_vmar = process_vm.root_vmar();
let process_vmar = process_vm.lock_root_vmar();
let root_vmar = process_vmar.get();
// After we clear process vm, if any error happens, we must call exit_group instead of return to user space.
let ldso_load_info = if let Some((ldso_file, ldso_elf)) = ldso {
@ -407,7 +408,8 @@ pub fn init_aux_vec(elf: &Elf, elf_map_addr: Vaddr, ldso_base: Option<Vaddr>) ->
/// Maps the VDSO VMO to the corresponding virtual memory address.
fn map_vdso_to_vm(process_vm: &ProcessVm) -> Option<Vaddr> {
let root_vmar = process_vm.root_vmar();
let process_vmar = process_vm.lock_root_vmar();
let root_vmar = process_vmar.get();
let vdso_vmo = vdso_vmo()?;
let options = root_vmar

View File

@ -130,7 +130,7 @@ pub(super) fn read_linux_sched_attr_from_user(
) -> Result<LinuxSchedAttr> {
let type_size = mem::size_of::<LinuxSchedAttr>();
let space = CurrentUserSpace::new(ctx.task);
let space = ctx.user_space();
let mut attr = LinuxSchedAttr::default();
@ -160,7 +160,7 @@ pub(super) fn write_linux_sched_attr_to_user(
user_size: u32,
ctx: &Context,
) -> Result<()> {
let space = CurrentUserSpace::new(ctx.task);
let space = ctx.user_space();
attr.size = (mem::size_of::<LinuxSchedAttr>() as u32).min(user_size);

View File

@ -10,7 +10,7 @@ pub fn sys_sched_getparam(tid: Tid, addr: Vaddr, ctx: &Context) -> Result<Syscal
_ => 0,
};
let space = CurrentUserSpace::new(ctx.task);
let space = ctx.user_space();
space
.write_val(addr, &rt_prio)
.map_err(|_| Error::new(Errno::EINVAL))?;

View File

@ -4,7 +4,7 @@ use super::{sched_getattr::access_sched_attr_with, SyscallReturn};
use crate::{prelude::*, sched::SchedPolicy, thread::Tid};
pub fn sys_sched_setparam(tid: Tid, addr: Vaddr, ctx: &Context) -> Result<SyscallReturn> {
let space = CurrentUserSpace::new(ctx.task);
let space = ctx.user_space();
let prio: i32 = space
.read_val(addr)
.map_err(|_| Error::new(Errno::EINVAL))?;

View File

@ -12,7 +12,7 @@ pub fn sys_sched_setscheduler(
addr: Vaddr,
ctx: &Context,
) -> Result<SyscallReturn> {
let space = CurrentUserSpace::new(&ctx.task);
let space = ctx.user_space();
let prio = space
.read_val(addr)
.map_err(|_| Error::new(Errno::EINVAL))?;

View File

@ -3,7 +3,7 @@
#![expect(unused_variables)]
use aster_rights::Full;
use ostd::{cpu::*, mm::VmSpace};
use ostd::{cpu::*, mm::VmSpace, task::Task};
use crate::{
prelude::*,
@ -31,7 +31,9 @@ pub fn handle_exception(ctx: &Context, context: &UserContext) {
log_trap_info(trap_info);
if let Ok(page_fault_info) = PageFaultInfo::try_from(trap_info) {
if handle_page_fault_from_vmar(ctx.process.root_vmar(), &page_fault_info).is_ok() {
let user_space = ctx.user_space();
let root_vmar = user_space.root_vmar();
if handle_page_fault_from_vmar(root_vmar, &page_fault_info).is_ok() {
return;
}
}
@ -44,8 +46,9 @@ pub(crate) fn handle_page_fault_from_vm_space(
vm_space: &VmSpace,
page_fault_info: &PageFaultInfo,
) -> core::result::Result<(), ()> {
let current = current!();
let root_vmar = current.root_vmar();
let task = Task::current().unwrap();
let current_root_vmar = task.as_thread_local().unwrap().root_vmar().borrow();
let root_vmar = current_root_vmar.as_ref().unwrap();
// If page is not present or due to write access, we should ask the vmar try to commit this page
debug_assert_eq!(