mirror of
https://github.com/asterinas/asterinas.git
synced 2025-06-22 17:03:23 +00:00
Implement fine grained locks for the page table and adjust APIs
This commit is contained in:
committed by
Tate, Hongliang Tian
parent
ef1ab72ebe
commit
2dbeb92326
@ -1,143 +1,213 @@
|
||||
// SPDX-License-Identifier: MPL-2.0
|
||||
|
||||
//! The page table cursor for mapping and querying over the page table.
|
||||
//!
|
||||
//! ## The page table lock protocol
|
||||
//!
|
||||
//! We provide a fine-grained lock protocol to allow concurrent accesses to
|
||||
//! the page table. The protocol is originally proposed by Ruihan Li
|
||||
//! <lrh2000@pku.edu.cn>.
|
||||
//!
|
||||
//! [`CursorMut::new`] accepts an address range, which indicates the page table
|
||||
//! entries that may be visited by this cursor.
|
||||
//!
|
||||
//! Then, [`CursorMut::new`] finds an intermediate page table (not necessarily
|
||||
//! the last-level or the top-level) which represents an address range that contains
|
||||
//! the whole specified address range. It requires all locks from the root page
|
||||
//! table to the intermediate page table, but then unlocks all locks excluding the
|
||||
//! one for the intermediate page table. CursorMut then maintains the lock
|
||||
//! guards from one for the intermediate page table to the leaf that the cursor is
|
||||
//! currently manipulating.
|
||||
//!
|
||||
//! For example, if we're going to map the address range shown below:
|
||||
//!
|
||||
//! ```plain
|
||||
//! Top-level page table node A
|
||||
//! /
|
||||
//! B
|
||||
//! / \
|
||||
//! Last-level page table nodes C D
|
||||
//! Last-level PTEs ---**...**---
|
||||
//! \__ __/
|
||||
//! V
|
||||
//! Address range that we're going to map
|
||||
//! ```
|
||||
//!
|
||||
//! When calling [`CursorMut::new`], it will:
|
||||
//! 1. `lock(A)`, `lock(B)`, `unlock(A)`;
|
||||
//! 2. `guards = [ locked(B) ]`.
|
||||
//!
|
||||
//! When calling [`CursorMut::map`], it will:
|
||||
//! 1. `lock(C)`, `guards = [ locked(B), locked(C) ]`;
|
||||
//! 2. Map some pages in `C`;
|
||||
//! 3. `unlock(C)`, `lock_guard = [ locked(B) ]`;
|
||||
//! 4. `lock(D)`, `lock_guard = [ locked(B), locked(D) ]`;
|
||||
//! 5. Map some pages in D;
|
||||
//! 6. `unlock(D)`, `lock_guard = [ locked(B) ]`;
|
||||
//!
|
||||
//! If all the mappings in `B` are cancelled when cursor finished it's traversal,
|
||||
//! and `B` need to be recycled, a page walk from the root page table to `B` is
|
||||
//! required. The cursor unlock all locks, then lock all the way down to `B`, then
|
||||
//! check if `B` is empty, and finally recycle all the resources on the way back.
|
||||
|
||||
use alloc::sync::Arc;
|
||||
use core::{any::TypeId, mem::size_of, ops::Range};
|
||||
use core::{any::TypeId, ops::Range};
|
||||
|
||||
use align_ext::AlignExt;
|
||||
|
||||
use super::{
|
||||
Child, KernelMode, MapInfo, MapOp, MapProperty, PageTable, PageTableConstsTrait,
|
||||
PageTableEntryTrait, PageTableError, PageTableFrame, PageTableMode, PtfRef,
|
||||
PageTableEntryTrait, PageTableError, PageTableFrame, PageTableMode,
|
||||
};
|
||||
use crate::{
|
||||
sync::{ArcSpinLockGuard, SpinLock},
|
||||
vm::{paddr_to_vaddr, Paddr, Vaddr, VmFrame},
|
||||
vm::{Paddr, Vaddr, VmFrame},
|
||||
};
|
||||
|
||||
/// The cursor for forward traversal over the page table.
|
||||
/// The cursor for traversal over the page table.
|
||||
///
|
||||
/// Each method may move the cursor forward, doing mapping unmaping, or
|
||||
/// querying this slot.
|
||||
/// Efficient methods are provided to move the cursor forward by a slot,
|
||||
/// doing mapping, unmaping, or querying for the traversed slot. Also you
|
||||
/// can jump forward or backward by re-walking without releasing the lock.
|
||||
///
|
||||
/// A slot is a PTE at any levels, which correspond to a certain virtual
|
||||
/// memory range sized by the "page size" of the current level.
|
||||
///
|
||||
/// Doing mapping is somewhat like a depth-first search on a tree, except
|
||||
/// that we modify the tree while traversing it. We use a stack to simulate
|
||||
/// the recursion.
|
||||
///
|
||||
/// Any read or write accesses to nodes require exclusive access on the
|
||||
/// entire path from the root to the node. But cursor can be created without
|
||||
/// holding the lock, and can release the lock after yeilding the current
|
||||
/// slot while querying over the page table with a range. Simultaneous
|
||||
/// reading or writing to the same range in the page table will not produce
|
||||
/// consistent results, only validity is guaranteed.
|
||||
pub(super) struct PageTableCursor<
|
||||
'a,
|
||||
M: PageTableMode,
|
||||
E: PageTableEntryTrait,
|
||||
C: PageTableConstsTrait,
|
||||
> where
|
||||
[(); C::NR_ENTRIES_PER_FRAME]:,
|
||||
[(); C::NR_LEVELS]:,
|
||||
{
|
||||
stack: [Option<PtfRef<E, C>>; C::NR_LEVELS],
|
||||
lock_guard: [Option<ArcSpinLockGuard<PageTableFrame<E, C>>>; C::NR_LEVELS],
|
||||
level: usize,
|
||||
va: Vaddr,
|
||||
}
|
||||
|
||||
#[derive(Debug, Clone)]
|
||||
pub(super) enum MapOption {
|
||||
Map {
|
||||
frame: VmFrame,
|
||||
prop: MapProperty,
|
||||
},
|
||||
MapUntyped {
|
||||
pa: Paddr,
|
||||
len: usize,
|
||||
prop: MapProperty,
|
||||
},
|
||||
Unmap {
|
||||
len: usize,
|
||||
},
|
||||
}
|
||||
|
||||
impl MapOption {
|
||||
fn paddr(&self) -> Option<Paddr> {
|
||||
match self {
|
||||
MapOption::Map { frame, prop } => Some(frame.start_paddr()),
|
||||
MapOption::MapUntyped { pa, len, prop } => Some(*pa),
|
||||
MapOption::Unmap { len } => None,
|
||||
}
|
||||
}
|
||||
fn prop(&self) -> Option<MapProperty> {
|
||||
match self {
|
||||
MapOption::Map { frame, prop } => Some(*prop),
|
||||
MapOption::MapUntyped { pa, len, prop } => Some(*prop),
|
||||
MapOption::Unmap { len } => None,
|
||||
}
|
||||
}
|
||||
fn len(&self) -> usize {
|
||||
match self {
|
||||
// A VmFrame currently has a fixed size of 1 base page.
|
||||
MapOption::Map { frame, prop } => crate::arch::mm::PageTableConsts::BASE_PAGE_SIZE,
|
||||
MapOption::MapUntyped { pa, len, prop } => *len,
|
||||
MapOption::Unmap { len: l } => *l,
|
||||
}
|
||||
}
|
||||
fn consume(&mut self, len: usize) -> Self {
|
||||
match self {
|
||||
MapOption::Map { frame, prop } => {
|
||||
debug_assert_eq!(len, crate::arch::mm::PageTableConsts::BASE_PAGE_SIZE);
|
||||
let ret = self.clone();
|
||||
*self = MapOption::Unmap { len: 0 };
|
||||
ret
|
||||
}
|
||||
MapOption::MapUntyped { pa, len: l, prop } => {
|
||||
debug_assert!(*l >= len);
|
||||
let ret = MapOption::MapUntyped {
|
||||
pa: *pa,
|
||||
len,
|
||||
prop: *prop,
|
||||
};
|
||||
*self = MapOption::MapUntyped {
|
||||
pa: *pa + len,
|
||||
len: *l - len,
|
||||
prop: *prop,
|
||||
};
|
||||
ret
|
||||
}
|
||||
MapOption::Unmap { len: l } => {
|
||||
debug_assert!(*l >= len);
|
||||
let ret = MapOption::Unmap { len };
|
||||
*l -= len;
|
||||
ret
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<M: PageTableMode, E: PageTableEntryTrait, C: PageTableConstsTrait> PageTableCursor<'_, M, E, C>
|
||||
/// that we modify the tree while traversing it. We use a guard stack to
|
||||
/// simulate the recursion, and adpot a page table locking protocol to
|
||||
/// provide concurrency.
|
||||
pub(crate) struct CursorMut<'a, M: PageTableMode, E: PageTableEntryTrait, C: PageTableConstsTrait>
|
||||
where
|
||||
[(); C::NR_ENTRIES_PER_FRAME]:,
|
||||
[(); C::NR_LEVELS]:,
|
||||
{
|
||||
pub(super) fn new(pt: &PageTable<M, E, C>, va: Vaddr) -> Self {
|
||||
let mut stack = core::array::from_fn(|_| None);
|
||||
stack[0] = Some(pt.root_frame.clone());
|
||||
let lock_guard = core::array::from_fn(|_| None);
|
||||
Self {
|
||||
stack,
|
||||
lock_guard,
|
||||
pt: &'a PageTable<M, E, C>,
|
||||
guards: [Option<ArcSpinLockGuard<PageTableFrame<E, C>>>; C::NR_LEVELS],
|
||||
level: usize, // current level
|
||||
guard_level: usize, // from guard_level to level, the locks are held
|
||||
va: Vaddr, // current virtual address
|
||||
barrier_va: Range<Vaddr>, // virtual address range that is locked
|
||||
}
|
||||
|
||||
impl<'a, M: PageTableMode, E: PageTableEntryTrait, C: PageTableConstsTrait> CursorMut<'a, M, E, C>
|
||||
where
|
||||
[(); C::NR_ENTRIES_PER_FRAME]:,
|
||||
[(); C::NR_LEVELS]:,
|
||||
{
|
||||
/// Create a cursor exclusively owning the locks for the given range.
|
||||
///
|
||||
/// The cursor created will only be able to map, query or jump within the
|
||||
/// given range.
|
||||
pub(crate) fn new(
|
||||
pt: &'a PageTable<M, E, C>,
|
||||
va: &Range<Vaddr>,
|
||||
) -> Result<Self, PageTableError> {
|
||||
if !M::covers(va) {
|
||||
return Err(PageTableError::InvalidVaddrRange(va.start, va.end));
|
||||
}
|
||||
if va.start % C::BASE_PAGE_SIZE != 0 || va.end % C::BASE_PAGE_SIZE != 0 {
|
||||
return Err(PageTableError::UnalignedVaddr);
|
||||
}
|
||||
// Create a guard array that only hold the root node lock.
|
||||
let guards = core::array::from_fn(|i| {
|
||||
if i == 0 {
|
||||
Some(pt.root_frame.lock_arc())
|
||||
} else {
|
||||
None
|
||||
}
|
||||
});
|
||||
let mut cursor = Self {
|
||||
pt,
|
||||
guards,
|
||||
level: C::NR_LEVELS,
|
||||
va,
|
||||
guard_level: C::NR_LEVELS,
|
||||
va: va.start,
|
||||
barrier_va: va.clone(),
|
||||
};
|
||||
// Go down and get proper locks. The cursor should hold a lock of a
|
||||
// page table node containing the virtual address range.
|
||||
//
|
||||
// While going down, previous guards of too-high levels will be released.
|
||||
loop {
|
||||
let level_too_high = {
|
||||
let start_idx = C::in_frame_index(va.start, cursor.level);
|
||||
let end_idx = C::in_frame_index(va.end - 1, cursor.level);
|
||||
start_idx == end_idx
|
||||
};
|
||||
if !level_too_high || !cursor.cur_child().is_pt() {
|
||||
break;
|
||||
}
|
||||
cursor.level_down(None);
|
||||
cursor.guards[C::NR_LEVELS - cursor.level - 1] = None;
|
||||
cursor.guard_level -= 1;
|
||||
}
|
||||
Ok(cursor)
|
||||
}
|
||||
|
||||
/// Jump to the given virtual address.
|
||||
///
|
||||
/// It panics if the address is out of the range where the cursor is required to operate,
|
||||
/// or has bad alignment.
|
||||
pub(crate) fn jump(&mut self, va: Vaddr) {
|
||||
assert!(self.barrier_va.contains(&va));
|
||||
assert!(va % C::BASE_PAGE_SIZE == 0);
|
||||
loop {
|
||||
let cur_node_start = self.va & !(C::page_size(self.level + 1) - 1);
|
||||
let cur_node_end = cur_node_start + C::page_size(self.level + 1);
|
||||
// If the address is within the current node, we can jump directly.
|
||||
if cur_node_start <= va && va < cur_node_end {
|
||||
self.va = va;
|
||||
return;
|
||||
}
|
||||
// There is a corner case that the cursor is depleted, sitting at the start of the
|
||||
// next node but the next node is not locked because the parent is not locked.
|
||||
if self.va >= self.barrier_va.end && self.level == self.guard_level {
|
||||
self.va = va;
|
||||
return;
|
||||
}
|
||||
debug_assert!(self.level < self.guard_level);
|
||||
self.level_up();
|
||||
}
|
||||
}
|
||||
|
||||
/// Map or unmap the range starting from the current address.
|
||||
/// Map the range starting from the current address to a `VmFrame`.
|
||||
///
|
||||
/// The argument `create` allows you to map the continuous range to a physical
|
||||
/// range with the given map property.
|
||||
/// # Panic
|
||||
///
|
||||
/// This function will panic if
|
||||
/// - the virtual address range to be mapped is out of the range;
|
||||
/// - it is already mapped to a huge page while the caller wants to map a smaller one.
|
||||
///
|
||||
/// # Safety
|
||||
///
|
||||
/// The caller should ensure that the virtual range being mapped does
|
||||
/// not affect kernel's memory safety.
|
||||
pub(crate) unsafe fn map(&mut self, frame: VmFrame, prop: MapProperty) {
|
||||
let end = self.va + C::BASE_PAGE_SIZE;
|
||||
assert!(end <= self.barrier_va.end);
|
||||
// Go down if not applicable.
|
||||
while self.level > C::HIGHEST_TRANSLATION_LEVEL
|
||||
|| self.va % C::page_size(self.level) != 0
|
||||
|| self.va + C::page_size(self.level) > end
|
||||
{
|
||||
self.level_down(Some(prop));
|
||||
continue;
|
||||
}
|
||||
// Map the current page.
|
||||
let idx = self.cur_idx();
|
||||
let level = self.level;
|
||||
self.cur_node_mut()
|
||||
.set_child(idx, Child::Frame(frame), Some(prop), level > 1);
|
||||
self.move_forward();
|
||||
}
|
||||
|
||||
/// Map the range starting from the current address to a physical address range.
|
||||
///
|
||||
/// The function will map as more huge pages as possible, and it will split
|
||||
/// the huge pages into smaller pages if necessary. If the input range is large,
|
||||
/// the resulting mappings may look like this (if very huge pages supported):
|
||||
/// the huge pages into smaller pages if necessary. If the input range is
|
||||
/// large, the resulting mappings may look like this (if very huge pages
|
||||
/// supported):
|
||||
///
|
||||
/// ```text
|
||||
/// start end
|
||||
@ -146,159 +216,245 @@ where
|
||||
/// 4KiB 2MiB 1GiB 4KiB 4KiB
|
||||
/// ```
|
||||
///
|
||||
/// In practice it is suggested to use simple wrappers for this API that maps
|
||||
/// frames for safety and conciseness.
|
||||
/// In practice it is not suggested to use this method for safety and conciseness.
|
||||
///
|
||||
/// # Safety
|
||||
///
|
||||
/// This function manipulates the page table directly, and it is unsafe because
|
||||
/// it may cause undefined behavior if the caller does not ensure that the
|
||||
/// mapped address is valid and the page table is not corrupted if it is used
|
||||
/// by the kernel.
|
||||
pub(super) unsafe fn map(&mut self, option: MapOption) {
|
||||
self.acquire_locks();
|
||||
let len = option.len();
|
||||
let end = self.va + len;
|
||||
let mut option = option;
|
||||
/// The caller should ensure that
|
||||
/// - the range being mapped does not affect kernel's memory safety;
|
||||
/// - the physical address to be mapped is valid and safe to use.
|
||||
pub(crate) unsafe fn map_pa(&mut self, pa: &Range<Paddr>, prop: MapProperty) {
|
||||
let end = self.va + pa.len();
|
||||
let mut pa = pa.start;
|
||||
assert!(end <= self.barrier_va.end);
|
||||
while self.va < end {
|
||||
// Skip if we are unmapping and it is already invalid.
|
||||
let cur_pte = unsafe { self.cur_pte_ptr().read() };
|
||||
if matches!(option, MapOption::Unmap { .. }) && !cur_pte.is_valid() {
|
||||
self.next_slot();
|
||||
// We ensure not mapping in reserved kernel shared tables or releasing it.
|
||||
// Although it may be an invariant for all architectures and will be optimized
|
||||
// out by the compiler since `C::NR_LEVELS - 1 > C::HIGHEST_TRANSLATION_LEVEL`.
|
||||
let is_kernel_shared_node =
|
||||
TypeId::of::<M>() == TypeId::of::<KernelMode>() && self.level >= C::NR_LEVELS - 1;
|
||||
if self.level > C::HIGHEST_TRANSLATION_LEVEL
|
||||
|| is_kernel_shared_node
|
||||
|| self.va % C::page_size(self.level) != 0
|
||||
|| self.va + C::page_size(self.level) > end
|
||||
|| pa % C::page_size(self.level) != 0
|
||||
{
|
||||
self.level_down(Some(prop));
|
||||
continue;
|
||||
}
|
||||
// Map the current page.
|
||||
let idx = self.cur_idx();
|
||||
let level = self.level;
|
||||
self.cur_node_mut()
|
||||
.set_child(idx, Child::Untracked(pa), Some(prop), level > 1);
|
||||
pa += C::page_size(level);
|
||||
self.move_forward();
|
||||
}
|
||||
}
|
||||
|
||||
/// Unmap the range starting from the current address with the given length of virtual address.
|
||||
///
|
||||
/// # Safety
|
||||
///
|
||||
/// The caller should ensure that the range being unmapped does not affect kernel's memory safety.
|
||||
///
|
||||
/// # Panic
|
||||
///
|
||||
/// This function will panic if:
|
||||
/// - the range to be unmapped is out of the range where the cursor is required to operate;
|
||||
/// - the range covers only a part of a page.
|
||||
pub(crate) unsafe fn unmap(&mut self, len: usize) {
|
||||
let end = self.va + len;
|
||||
assert!(end <= self.barrier_va.end);
|
||||
assert!(end % C::BASE_PAGE_SIZE == 0);
|
||||
while self.va < end {
|
||||
// Skip if it is already invalid.
|
||||
if self.cur_child().is_none() {
|
||||
if self.va + C::page_size(self.level) > end {
|
||||
break;
|
||||
}
|
||||
self.move_forward();
|
||||
continue;
|
||||
}
|
||||
|
||||
// We check among the conditions that may lead to a level down.
|
||||
let is_pa_not_aligned = option
|
||||
.paddr()
|
||||
.map(|pa| pa % C::page_size(self.level) != 0)
|
||||
.unwrap_or(false);
|
||||
let map_but_too_huge = self.level > C::HIGHEST_TRANSLATION_LEVEL
|
||||
&& !matches!(option, MapOption::Unmap { .. });
|
||||
// We ensure not mapping in reserved kernel shared tables or releasing it.
|
||||
// Although it may be an invariant for all architectures and will be optimized
|
||||
// out by the compiler since `C::NR_LEVELS - 1 > C::HIGHEST_TRANSLATION_LEVEL`.
|
||||
let kshared_lvl_down =
|
||||
// We ensure not unmapping in reserved kernel shared tables or releasing it.
|
||||
let is_kernel_shared_node =
|
||||
TypeId::of::<M>() == TypeId::of::<KernelMode>() && self.level >= C::NR_LEVELS - 1;
|
||||
if map_but_too_huge
|
||||
|| kshared_lvl_down
|
||||
if is_kernel_shared_node
|
||||
|| self.va % C::page_size(self.level) != 0
|
||||
|| self.va + C::page_size(self.level) > end
|
||||
|| is_pa_not_aligned
|
||||
{
|
||||
let ld_prop = option.prop().unwrap_or(MapProperty::new_invalid());
|
||||
self.level_down(Some(ld_prop));
|
||||
self.level_down(Some(MapProperty::new_invalid()));
|
||||
continue;
|
||||
}
|
||||
self.map_page(option.consume(C::page_size(self.level)));
|
||||
self.next_slot();
|
||||
|
||||
// Unmap the current page.
|
||||
let idx = self.cur_idx();
|
||||
self.cur_node_mut().set_child(idx, Child::None, None, false);
|
||||
self.move_forward();
|
||||
}
|
||||
self.release_locks();
|
||||
}
|
||||
|
||||
/// Apply the given operation to all the mappings within the range.
|
||||
pub(super) unsafe fn protect(
|
||||
///
|
||||
/// The funtction will return an error if it is not allowed to protect an invalid range and
|
||||
/// it does so, or if the range to be protected only covers a part of a page.
|
||||
///
|
||||
/// # Safety
|
||||
///
|
||||
/// The caller should ensure that the range being protected does not affect kernel's memory safety.
|
||||
///
|
||||
/// # Panic
|
||||
///
|
||||
/// This function will panic if:
|
||||
/// - the range to be protected is out of the range where the cursor is required to operate.
|
||||
pub(crate) unsafe fn protect(
|
||||
&mut self,
|
||||
len: usize,
|
||||
op: impl MapOp,
|
||||
allow_protect_invalid: bool,
|
||||
) -> Result<(), PageTableError> {
|
||||
self.acquire_locks();
|
||||
let end = self.va + len;
|
||||
assert!(end <= self.barrier_va.end);
|
||||
while self.va < end {
|
||||
let cur_pte = unsafe { self.cur_pte_ptr().read() };
|
||||
if !cur_pte.is_valid() {
|
||||
if self.cur_child().is_none() {
|
||||
if !allow_protect_invalid {
|
||||
return Err(PageTableError::ProtectingInvalid);
|
||||
}
|
||||
self.next_slot();
|
||||
self.move_forward();
|
||||
continue;
|
||||
}
|
||||
// Go down if it's not a last node or if the page size is too big.
|
||||
if !(cur_pte.is_huge() || self.level == 1)
|
||||
|| (self.va % C::page_size(self.level)) != 0
|
||||
|| self.va + C::page_size(self.level) > end
|
||||
{
|
||||
self.level_down(Some(op(cur_pte.info())));
|
||||
// Go down if it's not a last node.
|
||||
if self.cur_child().is_pt() {
|
||||
self.level_down(None);
|
||||
continue;
|
||||
}
|
||||
// Apply the operation.
|
||||
unsafe {
|
||||
self.cur_pte_ptr().write(E::new(
|
||||
cur_pte.paddr(),
|
||||
op(cur_pte.info()),
|
||||
cur_pte.is_huge(),
|
||||
true,
|
||||
))
|
||||
};
|
||||
self.next_slot();
|
||||
let vaddr_not_fit =
|
||||
self.va % C::page_size(self.level) != 0 || self.va + C::page_size(self.level) > end;
|
||||
let cur_pte_info = self.read_cur_pte_info();
|
||||
let protected_prop = op(cur_pte_info);
|
||||
// Go down if the page size is too big and we are protecting part
|
||||
// of untyped huge pages.
|
||||
if self.cur_child().is_untyped() && vaddr_not_fit {
|
||||
self.level_down(Some(protected_prop));
|
||||
continue;
|
||||
} else if vaddr_not_fit {
|
||||
return Err(PageTableError::ProtectingPartial);
|
||||
}
|
||||
let idx = self.cur_idx();
|
||||
let level = self.level;
|
||||
self.cur_node_mut().protect(idx, protected_prop, level);
|
||||
self.move_forward();
|
||||
}
|
||||
self.release_locks();
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn cur_pte_ptr(&self) -> *mut E {
|
||||
let ptf = self.lock_guard[C::NR_LEVELS - self.level].as_ref().unwrap();
|
||||
let frame_addr = paddr_to_vaddr(ptf.inner.start_paddr());
|
||||
let offset = C::in_frame_index(self.va, self.level);
|
||||
(frame_addr + offset * size_of::<E>()) as *mut E
|
||||
/// Get the information of the current slot and move to the next slot.
|
||||
pub(crate) fn query(&mut self) -> Option<PageTableQueryResult> {
|
||||
if self.va >= self.barrier_va.end {
|
||||
return None;
|
||||
}
|
||||
loop {
|
||||
let level = self.level;
|
||||
let va = self.va;
|
||||
let map_info = self.read_cur_pte_info();
|
||||
match self.cur_child().clone() {
|
||||
Child::Frame(frame) => {
|
||||
self.move_forward();
|
||||
return Some(PageTableQueryResult::Mapped {
|
||||
va,
|
||||
frame,
|
||||
info: map_info,
|
||||
});
|
||||
}
|
||||
Child::PageTable(_) => {
|
||||
// Go down if it's not a last node.
|
||||
self.level_down(None);
|
||||
continue;
|
||||
}
|
||||
Child::Untracked(pa) => {
|
||||
self.move_forward();
|
||||
return Some(PageTableQueryResult::MappedUntyped {
|
||||
va,
|
||||
pa,
|
||||
len: C::page_size(level),
|
||||
info: map_info,
|
||||
});
|
||||
}
|
||||
Child::None => {
|
||||
self.move_forward();
|
||||
return Some(PageTableQueryResult::NotMapped {
|
||||
va,
|
||||
len: C::page_size(level),
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Consume itself and leak the root guard for the caller if it locked the root level.
|
||||
///
|
||||
/// It is useful when the caller wants to keep the root guard while the cursor should be dropped.
|
||||
pub(super) fn leak_root_guard(mut self) -> Option<ArcSpinLockGuard<PageTableFrame<E, C>>> {
|
||||
if self.guard_level != C::NR_LEVELS {
|
||||
return None;
|
||||
}
|
||||
while self.level < C::NR_LEVELS {
|
||||
self.level_up();
|
||||
}
|
||||
self.guards[0].take()
|
||||
// Ok to drop self here because we ensure not to access the page table if the current
|
||||
// level is the root level when running the dropping method.
|
||||
}
|
||||
|
||||
/// Traverse forward in the current level to the next PTE.
|
||||
/// If reached the end of a page table frame, it leads itself up to the next frame of the parent frame.
|
||||
fn next_slot(&mut self) {
|
||||
///
|
||||
/// If reached the end of a page table frame, it leads itself up to the next frame of the parent
|
||||
/// frame if possible.
|
||||
fn move_forward(&mut self) {
|
||||
let page_size = C::page_size(self.level);
|
||||
while self.level < C::NR_LEVELS && C::in_frame_index(self.va + page_size, self.level) == 0 {
|
||||
let next_va = self.va.align_down(page_size) + page_size;
|
||||
while self.level < self.guard_level && C::in_frame_index(next_va, self.level) == 0 {
|
||||
self.level_up();
|
||||
}
|
||||
self.va += page_size;
|
||||
self.va = next_va;
|
||||
}
|
||||
|
||||
/// Go up a level. We release the current frame if it has no mappings since the cursor only moves
|
||||
/// forward. And we will do the final cleanup using `level_up` when the cursor is dropped.
|
||||
/// forward. And if needed we will do the final cleanup using this method after re-walk when the
|
||||
/// cursor is dropped.
|
||||
///
|
||||
/// This method requires locks acquired before calling it. The discarded level will be unlocked.
|
||||
fn level_up(&mut self) {
|
||||
let last_map_cnt_is_zero = {
|
||||
let top_ptf = self.lock_guard[C::NR_LEVELS - self.level].as_ref().unwrap();
|
||||
top_ptf.map_count == 0
|
||||
};
|
||||
self.stack[C::NR_LEVELS - self.level] = None;
|
||||
self.lock_guard[C::NR_LEVELS - self.level] = None;
|
||||
let last_node_all_unmapped = self.cur_node().nr_valid_children() == 0;
|
||||
self.guards[C::NR_LEVELS - self.level] = None;
|
||||
self.level += 1;
|
||||
let can_release_child =
|
||||
TypeId::of::<M>() == TypeId::of::<KernelMode>() && self.level < C::NR_LEVELS;
|
||||
if can_release_child && last_map_cnt_is_zero {
|
||||
let top_ptf = self.lock_guard[C::NR_LEVELS - self.level]
|
||||
.as_deref_mut()
|
||||
.unwrap();
|
||||
let frame_addr = paddr_to_vaddr(top_ptf.inner.start_paddr());
|
||||
let idx = C::in_frame_index(self.va, self.level);
|
||||
unsafe { (frame_addr as *mut E).add(idx).write(E::new_invalid()) }
|
||||
top_ptf.child[idx] = None;
|
||||
top_ptf.map_count -= 1;
|
||||
if can_release_child && last_node_all_unmapped {
|
||||
let idx = self.cur_idx();
|
||||
self.cur_node_mut().set_child(idx, Child::None, None, false);
|
||||
}
|
||||
}
|
||||
|
||||
/// A level down operation during traversal. It may split a huge page into
|
||||
/// smaller pages if we have an end address within the next mapped huge page.
|
||||
/// It may also create a new child frame if the current frame does not have one.
|
||||
/// If that may happen the map property of intermediate level `prop` should be
|
||||
/// A level down operation during traversal. It may create a new child frame if the
|
||||
/// current frame does not have one. It may also split an untyped huge page into
|
||||
/// smaller pages if we have an end address within the next mapped untyped huge page.
|
||||
///
|
||||
/// If creation may happen the map property of intermediate level `prop` should be
|
||||
/// passed in correctly. Whether the map property matters in an intermediate
|
||||
/// level is architecture-dependent.
|
||||
///
|
||||
/// This method requires write locks acquired before calling it. The newly added
|
||||
/// level will still hold the lock.
|
||||
unsafe fn level_down(&mut self, prop: Option<MapProperty>) {
|
||||
/// Also, the staticness of the page table is guaranteed if the caller make sure
|
||||
/// that there is a child node for the current node.
|
||||
fn level_down(&mut self, prop: Option<MapProperty>) {
|
||||
debug_assert!(self.level > 1);
|
||||
// Check if the child frame exists.
|
||||
let nxt_lvl_frame = {
|
||||
let idx = C::in_frame_index(self.va, self.level);
|
||||
let child = {
|
||||
let top_ptf = self.lock_guard[C::NR_LEVELS - self.level].as_ref().unwrap();
|
||||
&top_ptf.child[idx]
|
||||
};
|
||||
if let Some(Child::PageTable(nxt_lvl_frame)) = child {
|
||||
let child = self.cur_child();
|
||||
if let Child::PageTable(nxt_lvl_frame) = child {
|
||||
Some(nxt_lvl_frame.clone())
|
||||
} else {
|
||||
None
|
||||
@ -307,143 +463,89 @@ where
|
||||
// Create a new child frame if it does not exist. Sure it could be done only if
|
||||
// it is allowed to modify the page table.
|
||||
let nxt_lvl_frame = nxt_lvl_frame.unwrap_or_else(|| {
|
||||
let mut new_frame = PageTableFrame::<E, C>::new();
|
||||
// If it already maps a huge page, we should split it.
|
||||
let pte = unsafe { self.cur_pte_ptr().read() };
|
||||
if pte.is_valid() && pte.is_huge() {
|
||||
let pa = pte.paddr();
|
||||
let prop = pte.info().prop;
|
||||
for i in 0..C::NR_ENTRIES_PER_FRAME {
|
||||
let nxt_level = self.level - 1;
|
||||
let nxt_pte = {
|
||||
let frame_addr = paddr_to_vaddr(new_frame.inner.start_paddr());
|
||||
&mut *(frame_addr as *mut E).add(i)
|
||||
};
|
||||
*nxt_pte = E::new(pa + i * C::page_size(nxt_level), prop, nxt_level > 1, true);
|
||||
}
|
||||
new_frame.map_count = C::NR_ENTRIES_PER_FRAME;
|
||||
unsafe {
|
||||
self.cur_pte_ptr().write(E::new(
|
||||
new_frame.inner.start_paddr(),
|
||||
prop,
|
||||
false,
|
||||
false,
|
||||
))
|
||||
}
|
||||
// If it already maps an untyped huge page, we should split it.
|
||||
if self.cur_child().is_untyped() {
|
||||
let level = self.level;
|
||||
let idx = self.cur_idx();
|
||||
self.cur_node_mut().split_untracked_huge(level, idx);
|
||||
let Child::PageTable(nxt_lvl_frame) = self.cur_child() else {
|
||||
unreachable!()
|
||||
};
|
||||
nxt_lvl_frame.clone()
|
||||
} else if self.cur_child().is_none() {
|
||||
let new_frame = Arc::new(SpinLock::new(PageTableFrame::<E, C>::new()));
|
||||
let idx = self.cur_idx();
|
||||
self.cur_node_mut().set_child(
|
||||
idx,
|
||||
Child::PageTable(new_frame.clone()),
|
||||
prop,
|
||||
false,
|
||||
);
|
||||
new_frame
|
||||
} else {
|
||||
// The child couldn't be valid here because child is none and it's not huge.
|
||||
debug_assert!(!pte.is_valid());
|
||||
unsafe {
|
||||
self.cur_pte_ptr().write(E::new(
|
||||
new_frame.inner.start_paddr(),
|
||||
prop.unwrap(),
|
||||
false,
|
||||
false,
|
||||
))
|
||||
}
|
||||
panic!("Trying to level down when it is mapped to a typed frame");
|
||||
}
|
||||
let top_ptf = self.lock_guard[C::NR_LEVELS - self.level]
|
||||
.as_deref_mut()
|
||||
.unwrap();
|
||||
top_ptf.map_count += 1;
|
||||
let new_frame_ref = Arc::new(SpinLock::new(new_frame));
|
||||
top_ptf.child[C::in_frame_index(self.va, self.level)] =
|
||||
Some(Child::PageTable(new_frame_ref.clone()));
|
||||
new_frame_ref
|
||||
});
|
||||
self.lock_guard[C::NR_LEVELS - self.level + 1] = Some(nxt_lvl_frame.lock_arc());
|
||||
self.stack[C::NR_LEVELS - self.level + 1] = Some(nxt_lvl_frame);
|
||||
self.guards[C::NR_LEVELS - self.level + 1] = Some(nxt_lvl_frame.lock_arc());
|
||||
self.level -= 1;
|
||||
}
|
||||
|
||||
/// Map or unmap the page pointed to by the cursor (which could be large).
|
||||
/// If the physical address and the map property are not provided, it unmaps
|
||||
/// the current page.
|
||||
///
|
||||
/// This method requires write locks acquired before calling it.
|
||||
unsafe fn map_page(&mut self, option: MapOption) {
|
||||
let pte_ptr = self.cur_pte_ptr();
|
||||
let top_ptf = self.lock_guard[C::NR_LEVELS - self.level]
|
||||
.as_deref_mut()
|
||||
.unwrap();
|
||||
let child = {
|
||||
let idx = C::in_frame_index(self.va, self.level);
|
||||
if top_ptf.child[idx].is_some() {
|
||||
top_ptf.child[idx] = None;
|
||||
top_ptf.map_count -= 1;
|
||||
}
|
||||
&mut top_ptf.child[idx]
|
||||
};
|
||||
match option {
|
||||
MapOption::Map { frame, prop } => {
|
||||
let pa = frame.start_paddr();
|
||||
unsafe {
|
||||
pte_ptr.write(E::new(pa, prop, self.level > 1, true));
|
||||
}
|
||||
*child = Some(Child::Frame(frame));
|
||||
top_ptf.map_count += 1;
|
||||
}
|
||||
MapOption::MapUntyped { pa, len, prop } => {
|
||||
debug_assert_eq!(len, C::page_size(self.level));
|
||||
unsafe {
|
||||
pte_ptr.write(E::new(pa, prop, self.level > 1, true));
|
||||
}
|
||||
top_ptf.map_count += 1;
|
||||
}
|
||||
MapOption::Unmap { len } => {
|
||||
debug_assert_eq!(len, C::page_size(self.level));
|
||||
unsafe { pte_ptr.write(E::new_invalid()) }
|
||||
}
|
||||
}
|
||||
fn cur_node(&self) -> &ArcSpinLockGuard<PageTableFrame<E, C>> {
|
||||
self.guards[C::NR_LEVELS - self.level].as_ref().unwrap()
|
||||
}
|
||||
|
||||
fn acquire_locks(&mut self) {
|
||||
for i in 0..=C::NR_LEVELS - self.level {
|
||||
let Some(ref ptf) = self.stack[i] else {
|
||||
panic!("Invalid values in PT cursor stack while acuqiring locks");
|
||||
};
|
||||
debug_assert!(self.lock_guard[i].is_none());
|
||||
self.lock_guard[i] = Some(ptf.lock_arc());
|
||||
}
|
||||
fn cur_node_mut(&mut self) -> &mut ArcSpinLockGuard<PageTableFrame<E, C>> {
|
||||
self.guards[C::NR_LEVELS - self.level].as_mut().unwrap()
|
||||
}
|
||||
|
||||
fn release_locks(&mut self) {
|
||||
for i in (0..=C::NR_LEVELS - self.level).rev() {
|
||||
let Some(ref ptf) = self.stack[i] else {
|
||||
panic!("Invalid values in PT cursor stack while releasing locks");
|
||||
};
|
||||
debug_assert!(self.lock_guard[i].is_some());
|
||||
self.lock_guard[i] = None;
|
||||
}
|
||||
fn cur_idx(&self) -> usize {
|
||||
C::in_frame_index(self.va, self.level)
|
||||
}
|
||||
|
||||
fn cur_child(&self) -> &Child<E, C> {
|
||||
self.cur_node().child(self.cur_idx())
|
||||
}
|
||||
|
||||
fn read_cur_pte_info(&self) -> MapInfo {
|
||||
self.cur_node().read_pte_info(self.cur_idx())
|
||||
}
|
||||
}
|
||||
|
||||
/// The iterator for querying over the page table without modifying it.
|
||||
pub struct PageTableIter<'a, M: PageTableMode, E: PageTableEntryTrait, C: PageTableConstsTrait>
|
||||
impl<M: PageTableMode, E: PageTableEntryTrait, C: PageTableConstsTrait> Drop
|
||||
for CursorMut<'_, M, E, C>
|
||||
where
|
||||
[(); C::NR_ENTRIES_PER_FRAME]:,
|
||||
[(); C::NR_LEVELS]:,
|
||||
{
|
||||
cursor: PageTableCursor<'a, M, E, C>,
|
||||
end_va: Vaddr,
|
||||
}
|
||||
|
||||
impl<'a, M: PageTableMode, E: PageTableEntryTrait, C: PageTableConstsTrait>
|
||||
PageTableIter<'a, M, E, C>
|
||||
where
|
||||
[(); C::NR_ENTRIES_PER_FRAME]:,
|
||||
[(); C::NR_LEVELS]:,
|
||||
{
|
||||
pub(super) fn new(pt: &'a PageTable<M, E, C>, va: &Range<Vaddr>) -> Self {
|
||||
Self {
|
||||
cursor: PageTableCursor::new(pt, va.start),
|
||||
end_va: va.end,
|
||||
fn drop(&mut self) {
|
||||
// Recycle what we can recycle now.
|
||||
while self.level < self.guard_level {
|
||||
self.level_up();
|
||||
}
|
||||
// No need to do further cleanup if it is the root node or
|
||||
// there are mappings left.
|
||||
if self.level == self.guard_level || self.cur_node().nr_valid_children() != 0 {
|
||||
return;
|
||||
}
|
||||
// Drop the lock on the guard level.
|
||||
self.guards[C::NR_LEVELS - self.guard_level] = None;
|
||||
// Re-walk the page table to retreive the locks.
|
||||
self.guards[0] = Some(self.pt.root_frame.lock_arc());
|
||||
self.level = C::NR_LEVELS;
|
||||
// Another cursor can unmap the guard level node before this cursor
|
||||
// is dropped, we can just do our best here when re-walking.
|
||||
while self.level > self.guard_level && self.cur_child().is_pt() {
|
||||
self.level_down(None);
|
||||
}
|
||||
// Doing final cleanup by [`CursorMut::level_up`] to the root.
|
||||
while self.level < C::NR_LEVELS {
|
||||
self.level_up();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone, Debug)]
|
||||
pub enum PageTableQueryResult {
|
||||
pub(crate) enum PageTableQueryResult {
|
||||
NotMapped {
|
||||
va: Vaddr,
|
||||
len: usize,
|
||||
@ -461,8 +563,32 @@ pub enum PageTableQueryResult {
|
||||
},
|
||||
}
|
||||
|
||||
/// The read-only cursor for traversal over the page table.
|
||||
///
|
||||
/// It implements the `Iterator` trait to provide a convenient way to query over the page table.
|
||||
pub(crate) struct Cursor<'a, M: PageTableMode, E: PageTableEntryTrait, C: PageTableConstsTrait>
|
||||
where
|
||||
[(); C::NR_ENTRIES_PER_FRAME]:,
|
||||
[(); C::NR_LEVELS]:,
|
||||
{
|
||||
inner: CursorMut<'a, M, E, C>,
|
||||
}
|
||||
|
||||
impl<'a, M: PageTableMode, E: PageTableEntryTrait, C: PageTableConstsTrait> Cursor<'a, M, E, C>
|
||||
where
|
||||
[(); C::NR_ENTRIES_PER_FRAME]:,
|
||||
[(); C::NR_LEVELS]:,
|
||||
{
|
||||
pub(super) fn new(
|
||||
pt: &'a PageTable<M, E, C>,
|
||||
va: &Range<Vaddr>,
|
||||
) -> Result<Self, PageTableError> {
|
||||
CursorMut::new(pt, va).map(|inner| Self { inner })
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a, M: PageTableMode, E: PageTableEntryTrait, C: PageTableConstsTrait> Iterator
|
||||
for PageTableIter<'a, M, E, C>
|
||||
for Cursor<'a, M, E, C>
|
||||
where
|
||||
[(); C::NR_ENTRIES_PER_FRAME]:,
|
||||
[(); C::NR_LEVELS]:,
|
||||
@ -470,63 +596,6 @@ where
|
||||
type Item = PageTableQueryResult;
|
||||
|
||||
fn next(&mut self) -> Option<Self::Item> {
|
||||
self.cursor.acquire_locks();
|
||||
if self.cursor.va >= self.end_va {
|
||||
return None;
|
||||
}
|
||||
loop {
|
||||
let level = self.cursor.level;
|
||||
let va = self.cursor.va;
|
||||
let top_ptf = self.cursor.lock_guard[C::NR_LEVELS - level]
|
||||
.as_ref()
|
||||
.unwrap();
|
||||
let cur_pte = unsafe { self.cursor.cur_pte_ptr().read() };
|
||||
// Yeild if it's not a valid node.
|
||||
if !cur_pte.is_valid() {
|
||||
self.cursor.next_slot();
|
||||
self.cursor.release_locks();
|
||||
return Some(PageTableQueryResult::NotMapped {
|
||||
va,
|
||||
len: C::page_size(level),
|
||||
});
|
||||
}
|
||||
// Go down if it's not a last node.
|
||||
if !(cur_pte.is_huge() || level == 1) {
|
||||
debug_assert!(cur_pte.is_valid());
|
||||
// Safety: it's valid and there should be a child frame here.
|
||||
unsafe {
|
||||
self.cursor.level_down(None);
|
||||
}
|
||||
continue;
|
||||
}
|
||||
// Yield the current mapping.
|
||||
let map_info = cur_pte.info();
|
||||
let idx = C::in_frame_index(self.cursor.va, self.cursor.level);
|
||||
match top_ptf.child[idx] {
|
||||
Some(Child::Frame(ref frame)) => {
|
||||
let frame = frame.clone();
|
||||
self.cursor.next_slot();
|
||||
self.cursor.release_locks();
|
||||
return Some(PageTableQueryResult::Mapped {
|
||||
va,
|
||||
frame,
|
||||
info: map_info,
|
||||
});
|
||||
}
|
||||
Some(Child::PageTable(_)) => {
|
||||
panic!("The child couldn't be page table here because it's valid and not huge");
|
||||
}
|
||||
None => {
|
||||
self.cursor.next_slot();
|
||||
self.cursor.release_locks();
|
||||
return Some(PageTableQueryResult::MappedUntyped {
|
||||
va,
|
||||
pa: cur_pte.paddr(),
|
||||
len: C::page_size(level),
|
||||
info: map_info,
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
self.inner.query()
|
||||
}
|
||||
}
|
||||
|
Reference in New Issue
Block a user