DFS lock protocol for the page table

This commit is contained in:
Zhang Junyang 2024-11-30 03:34:50 +00:00 committed by Tate, Hongliang Tian
parent d4afe3a035
commit d873e121ff
11 changed files with 683 additions and 550 deletions

View File

@ -9,6 +9,7 @@ use super::{
use crate::{mm::Paddr, sync::non_null::NonNullPtr};
/// A struct that can work as `&'a Frame<M>`.
#[derive(Debug)]
pub struct FrameRef<'a, M: AnyFrameMeta + ?Sized> {
inner: ManuallyDrop<Frame<M>>,
_marker: PhantomData<&'a Frame<M>>,

View File

@ -133,14 +133,14 @@ impl<M: AnyFrameMeta + ?Sized> Frame<M> {
self.slot().frame_paddr()
}
/// Gets the paging level of this page.
/// Gets the map level of this page.
///
/// This is the level of the page table entry that maps the frame,
/// which determines the size of the frame.
///
/// Currently, the level is always 1, which means the frame is a regular
/// page frame.
pub const fn level(&self) -> PagingLevel {
pub const fn map_level(&self) -> PagingLevel {
1
}

View File

@ -30,7 +30,7 @@ mod frame {
assert_eq!(frame.meta().value, 42);
assert_eq!(frame.reference_count(), 1);
assert_eq!(frame.size(), PAGE_SIZE);
assert_eq!(frame.level(), 1);
assert_eq!(frame.map_level(), 1);
}
#[ktest]

View File

@ -50,7 +50,6 @@ use super::{
meta::{mapping, KernelMeta, MetaPageMeta},
Frame, Segment,
},
nr_subpage_per_huge,
page_prop::{CachePolicy, PageFlags, PageProperty, PrivilegedPageFlags},
page_table::{KernelMode, PageTable},
Paddr, PagingConstsTrait, Vaddr, PAGE_SIZE,
@ -134,13 +133,7 @@ pub fn init_kernel_page_table(meta_pages: Segment<MetaPageMeta>) {
info!("Initializing the kernel page table");
// Start to initialize the kernel page table.
let kpt = PageTable::<KernelMode>::empty();
// Make shared the page tables mapped by the root table in the kernel space.
{
let pte_index_max = nr_subpage_per_huge::<PagingConsts>();
kpt.make_shared_tables(pte_index_max / 2..pte_index_max);
}
let kpt = PageTable::<KernelMode>::new_kernel_page_table();
// Do linear mappings for the kernel.
{

View File

@ -0,0 +1,240 @@
// SPDX-License-Identifier: MPL-2.0
//! Implementation of the locking protocol.
use core::{marker::PhantomData, ops::Range, sync::atomic::Ordering};
use align_ext::AlignExt;
use super::Cursor;
use crate::{
mm::{
nr_subpage_per_huge, paddr_to_vaddr,
page_table::{
load_pte, page_size, pte_index, Child, MapTrackingStatus, PageTable,
PageTableEntryTrait, PageTableGuard, PageTableMode, PageTableNodeRef,
PagingConstsTrait, PagingLevel,
},
Paddr, Vaddr,
},
task::disable_preempt,
};
pub(super) fn lock_range<'a, M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait>(
pt: &'a PageTable<M, E, C>,
va: &Range<Vaddr>,
new_pt_is_tracked: MapTrackingStatus,
) -> Cursor<'a, M, E, C> {
let preempt_guard = disable_preempt();
let mut subtree_root = traverse_and_lock_subtree_root(pt, va, new_pt_is_tracked);
// Once we have locked the sub-tree that is not stray, we won't read any
// stray nodes in the following traversal since we must lock before reading.
let guard_level = subtree_root.level();
let cur_node_va = va.start.align_down(page_size::<C>(guard_level + 1));
dfs_acquire_lock(&mut subtree_root, cur_node_va, va.clone());
let mut path = core::array::from_fn(|_| None);
path[guard_level as usize - 1] = Some(subtree_root);
Cursor::<'a, M, E, C> {
path,
level: guard_level,
guard_level,
va: va.start,
barrier_va: va.clone(),
preempt_guard,
_phantom: PhantomData,
}
}
pub(super) fn unlock_range<M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait>(
cursor: &mut Cursor<'_, M, E, C>,
) {
for i in (0..cursor.guard_level as usize - 1).rev() {
if let Some(guard) = cursor.path[i].take() {
let _ = guard.into_raw_paddr();
}
}
let guard_node = cursor.path[cursor.guard_level as usize - 1].take().unwrap();
let cur_node_va = cursor.barrier_va.start / page_size::<C>(cursor.guard_level + 1)
* page_size::<C>(cursor.guard_level + 1);
// SAFETY: A cursor maintains that its corresponding sub-tree is locked.
unsafe { dfs_release_lock(guard_node, cur_node_va, cursor.barrier_va.clone()) };
}
/// Finds and locks an intermediate page table node that covers the range.
///
/// If that node (or any of its ancestors) does not exist, we need to lock
/// the parent and create it. After the creation the lock of the parent will
/// be released and the new node will be locked.
fn traverse_and_lock_subtree_root<
'a,
M: PageTableMode,
E: PageTableEntryTrait,
C: PagingConstsTrait,
>(
pt: &'a PageTable<M, E, C>,
va: &Range<Vaddr>,
new_pt_is_tracked: MapTrackingStatus,
) -> PageTableGuard<'a, E, C> {
// # Safety
// Must be called with `cur_pt_addr` and `'a`, `E`, `E` of the residing function.
unsafe fn lock_cur_pt<'a, E: PageTableEntryTrait, C: PagingConstsTrait>(
cur_pt_addr: Paddr,
) -> PageTableGuard<'a, E, C> {
// SAFETY: The reference is valid for `'a` because the page table
// is alive within `'a` and `'a` is under the RCU read guard.
let ptn_ref = unsafe { PageTableNodeRef::<'a, E, C>::borrow_paddr(cur_pt_addr) };
// Forfeit a guard protecting a node that lives for `'a` rather
// than the lifetime of `ptn_ref`.
let pt_addr = ptn_ref.lock().into_raw_paddr();
// SAFETY: The lock guard was forgotten at the above line. We manually
// ensure that the protected node lives for `'a`.
unsafe { PageTableGuard::<'a, E, C>::from_raw_paddr(pt_addr) }
}
let mut cur_node_guard: Option<PageTableGuard<E, C>> = None;
let mut cur_pt_addr = pt.root.start_paddr();
for cur_level in (1..=C::NR_LEVELS).rev() {
let start_idx = pte_index::<C>(va.start, cur_level);
let level_too_high = {
let end_idx = pte_index::<C>(va.end - 1, cur_level);
cur_level > 1 && start_idx == end_idx
};
if !level_too_high {
break;
}
let cur_pt_ptr = paddr_to_vaddr(cur_pt_addr) as *mut E;
// SAFETY:
// - The page table node is alive because (1) the root node is alive and
// (2) all child nodes cannot be recycled because we're in the RCU critical section.
// - The index is inside the bound, so the page table entry is valid.
// - All page table entries are aligned and accessed with atomic operations only.
let cur_pte = unsafe { load_pte(cur_pt_ptr.add(start_idx), Ordering::Acquire) };
if cur_pte.is_present() {
if cur_pte.is_last(cur_level) {
break;
}
cur_pt_addr = cur_pte.paddr();
cur_node_guard = None;
continue;
}
// In case the child is absent, we should lock and allocate a new page table node.
// SAFETY: It is called with the required parameters.
let mut guard = cur_node_guard
.take()
.unwrap_or_else(|| unsafe { lock_cur_pt::<'a, E, C>(cur_pt_addr) });
let mut cur_entry = guard.entry(start_idx);
if cur_entry.is_none() {
let allocated_guard = cur_entry.alloc_if_none(new_pt_is_tracked).unwrap();
cur_pt_addr = allocated_guard.start_paddr();
cur_node_guard = Some(allocated_guard);
} else if cur_entry.is_node() {
let Child::PageTableRef(pt) = cur_entry.to_ref() else {
unreachable!();
};
cur_pt_addr = pt.start_paddr();
cur_node_guard = None;
} else {
break;
}
}
// SAFETY: It is called with the required parameters.
cur_node_guard.unwrap_or_else(|| unsafe { lock_cur_pt::<'a, E, C>(cur_pt_addr) })
}
/// Acquires the locks for the given range in the sub-tree rooted at the node.
///
/// `cur_node_va` must be the virtual address of the `cur_node`. The `va_range`
/// must be within the range of the `cur_node`. The range must not be empty.
///
/// The function will forget all the [`PageTableGuard`] objects in the sub-tree
/// with [`PageTableGuard::into_raw_paddr`].
fn dfs_acquire_lock<E: PageTableEntryTrait, C: PagingConstsTrait>(
cur_node: &mut PageTableGuard<'_, E, C>,
cur_node_va: Vaddr,
va_range: Range<Vaddr>,
) {
let cur_level = cur_node.level();
if cur_level <= 1 {
return;
}
let idx_range = dfs_get_idx_range::<C>(cur_level, cur_node_va, &va_range);
for i in idx_range {
let child = cur_node.entry(i);
match child.to_ref() {
Child::PageTableRef(pt) => {
let mut pt_guard = pt.lock();
let child_node_va = cur_node_va + i * page_size::<C>(cur_level);
let child_node_va_end = child_node_va + page_size::<C>(cur_level);
let va_start = va_range.start.max(child_node_va);
let va_end = va_range.end.min(child_node_va_end);
dfs_acquire_lock(&mut pt_guard, child_node_va, va_start..va_end);
let _ = pt_guard.into_raw_paddr();
}
Child::None | Child::Frame(_, _) | Child::Untracked(_, _, _) | Child::PageTable(_) => {}
}
}
}
/// Releases the locks for the given range in the sub-tree rooted at the node.
///
/// # Safety
///
/// The caller must ensure that the nodes in the specified sub-tree are locked.
unsafe fn dfs_release_lock<E: PageTableEntryTrait, C: PagingConstsTrait>(
mut cur_node: PageTableGuard<E, C>,
cur_node_va: Vaddr,
va_range: Range<Vaddr>,
) {
let cur_level = cur_node.level();
if cur_level <= 1 {
return;
}
let idx_range = dfs_get_idx_range::<C>(cur_level, cur_node_va, &va_range);
for i in idx_range.rev() {
let child = cur_node.entry(i);
match child.to_ref() {
Child::PageTableRef(pt) => {
// SAFETY: The node was locked before and we have a
// reference to the parent node that is still alive.
let child_node =
unsafe { PageTableGuard::<E, C>::from_raw_paddr(pt.start_paddr()) };
let child_node_va = cur_node_va + i * page_size::<C>(cur_level);
let child_node_va_end = child_node_va + page_size::<C>(cur_level);
let va_start = va_range.start.max(child_node_va);
let va_end = va_range.end.min(child_node_va_end);
// SAFETY: The caller ensures that this sub-tree is locked.
unsafe { dfs_release_lock(child_node, child_node_va, va_start..va_end) };
}
Child::None | Child::Frame(_, _) | Child::Untracked(_, _, _) | Child::PageTable(_) => {}
}
}
}
fn dfs_get_idx_range<C: PagingConstsTrait>(
cur_node_level: PagingLevel,
cur_node_va: Vaddr,
va_range: &Range<Vaddr>,
) -> Range<usize> {
debug_assert!(va_range.start >= cur_node_va);
debug_assert!(va_range.end <= cur_node_va.saturating_add(page_size::<C>(cur_node_level + 1)));
let start_idx = (va_range.start - cur_node_va) / page_size::<C>(cur_node_level);
let end_idx = (va_range.end - cur_node_va).div_ceil(page_size::<C>(cur_node_level));
debug_assert!(start_idx < end_idx);
debug_assert!(end_idx <= nr_subpage_per_huge::<C>());
start_idx..end_idx
}

View File

@ -2,89 +2,76 @@
//! The page table cursor for mapping and querying over the page table.
//!
//! ## The page table lock protocol
//! # 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>.
//! We provide a fine-grained ranged mutual-exclusive lock protocol to allow
//! concurrent accesses to non-overlapping virtual ranges in the page table.
//!
//! [`CursorMut::new`] will lock a range in the virtual space and all the
//! operations on the range with the cursor will be atomic as a transaction.
//!
//! The implementation of the lock protocol resembles two-phase locking (2PL).
//! [`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) ]`;
//!
//!
//! ## Validity
//!
//! The page table cursor API will guarantee that the page table, as a data
//! structure, whose occupied memory will not suffer from data races. This is
//! ensured by the page table lock protocol. In other words, any operations
//! provided by the APIs (as long as safety requirements are met) will not
//! break the page table data structure (or other memory).
//!
//! However, the page table cursor creation APIs, [`CursorMut::new`] or
//! [`Cursor::new`], do not guarantee exclusive access to the virtual address
//! area you claim. From the lock protocol, you can see that there are chances
//! to create 2 cursors that claim the same virtual address range (one covers
//! another). In this case, the greater cursor may block if it wants to modify
//! the page table entries covered by the smaller cursor. Also, if the greater
//! cursor destructs the smaller cursor's parent page table node, it won't block
//! and the smaller cursor's change will not be visible. The user of the page
//! table cursor should add additional entry point checks to prevent these defined
//! behaviors if they are not wanted.
//! 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 fully contains the whole
//! specified address range. Then it locks all the nodes in the sub-tree rooted
//! at the intermediate page table node, with a pre-order DFS order. The cursor
//! will only be able to access the page table entries in the locked range.
//! Upon destruction, the cursor will release the locks in the reverse order of
//! acquisition.
use core::{
any::TypeId, marker::PhantomData, mem::ManuallyDrop, ops::Range, sync::atomic::Ordering,
};
mod locking;
use core::{any::TypeId, marker::PhantomData, ops::Range};
use align_ext::AlignExt;
use super::{
page_size, pte_index, Child, Entry, KernelMode, MapTrackingStatus, PageTable,
PageTableEntryTrait, PageTableError, PageTableMode, PageTableNode, PagingConstsTrait,
PagingLevel, RawPageTableNode, UserMode,
PageTableEntryTrait, PageTableError, PageTableGuard, PageTableMode, PagingConstsTrait,
PagingLevel, UserMode,
};
use crate::{
mm::{
frame::{meta::AnyFrameMeta, Frame},
kspace::should_map_as_tracked,
paddr_to_vaddr, Paddr, PageProperty, Vaddr,
Paddr, PageProperty, Vaddr,
},
task::{disable_preempt, DisabledPreemptGuard},
task::DisabledPreemptGuard,
};
/// The cursor for traversal over the page table.
///
/// 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.
///
/// A cursor is able to move to the next slot, to read page properties,
/// and even to jump to a virtual address directly.
#[derive(Debug)]
pub struct Cursor<'a, M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait> {
/// The current path of the cursor.
///
/// The level 1 page table lock guard is at index 0, and the level N page
/// table lock guard is at index N - 1.
path: [Option<PageTableGuard<'a, E, C>>; MAX_NR_LEVELS],
/// The level of the page table that the cursor currently points to.
level: PagingLevel,
/// The top-most level that the cursor is allowed to access.
///
/// From `level` to `guard_level`, the nodes are held in `path`.
guard_level: PagingLevel,
/// The virtual address that the cursor currently points to.
va: Vaddr,
/// The virtual address range that is locked.
barrier_va: Range<Vaddr>,
#[expect(dead_code)]
preempt_guard: DisabledPreemptGuard,
_phantom: PhantomData<&'a PageTable<M, E, C>>,
}
/// The maximum value of `PagingConstsTrait::NR_LEVELS`.
const MAX_NR_LEVELS: usize = 4;
#[derive(Clone, Debug)]
pub enum PageTableItem {
NotMapped {
@ -104,50 +91,12 @@ pub enum PageTableItem {
},
}
/// The cursor for traversal over the page table.
///
/// 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.
///
/// A cursor is able to move to the next slot, to read page properties,
/// and even to jump to a virtual address directly. We use a guard stack to
/// simulate the recursion, and adpot a page table locking protocol to
/// provide concurrency.
#[derive(Debug)]
pub struct Cursor<'a, M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait> {
/// The lock guards of the cursor. The level 1 page table lock guard is at
/// index 0, and the level N page table lock guard is at index N - 1.
///
/// When destructing the cursor, the locks will be released in the order
/// from low to high, exactly the reverse order of the acquisition.
/// This behavior is ensured by the default drop implementation of Rust:
/// <https://doc.rust-lang.org/reference/destructors.html>.
guards: [Option<PageTableNode<E, C>>; MAX_NR_LEVELS],
/// The level of the page table that the cursor points to.
level: PagingLevel,
/// From `guard_level` to `level`, the locks are held in `guards`.
guard_level: PagingLevel,
/// The current virtual address that the cursor points to.
va: Vaddr,
/// The virtual address range that is locked.
barrier_va: Range<Vaddr>,
#[expect(dead_code)]
preempt_guard: DisabledPreemptGuard,
_phantom: PhantomData<&'a PageTable<M, E, C>>,
}
/// The maximum value of `PagingConstsTrait::NR_LEVELS`.
const MAX_NR_LEVELS: usize = 4;
impl<'a, M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait> Cursor<'a, M, E, C> {
/// Creates a cursor claiming the read access for the given range.
/// Creates a cursor claiming exclusive access over the given range.
///
/// The cursor created will only be able to query or jump within the given
/// range. Out-of-bound accesses will result in panics or errors as return values,
/// depending on the access method.
///
/// Note that this function does not ensure exclusive access to the claimed
/// virtual address range. The accesses using this cursor may block or fail.
pub fn new(pt: &'a PageTable<M, E, C>, va: &Range<Vaddr>) -> Result<Self, PageTableError> {
if !M::covers(va) || va.is_empty() {
return Err(PageTableError::InvalidVaddrRange(va.start, va.end));
@ -158,60 +107,13 @@ impl<'a, M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait> Cursor<
const { assert!(C::NR_LEVELS as usize <= MAX_NR_LEVELS) };
let mut cursor = Self {
guards: core::array::from_fn(|_| None),
level: C::NR_LEVELS,
guard_level: C::NR_LEVELS,
va: va.start,
barrier_va: va.clone(),
preempt_guard: disable_preempt(),
_phantom: PhantomData,
let new_pt_is_tracked = if should_map_as_tracked::<M>(va.start) {
MapTrackingStatus::Tracked
} else {
MapTrackingStatus::Untracked
};
let mut cur_pt_addr = pt.root.paddr();
// 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 start_idx = pte_index::<C>(va.start, cursor.level);
let level_too_high = {
let end_idx = pte_index::<C>(va.end - 1, cursor.level);
cursor.level > 1 && start_idx == end_idx
};
if !level_too_high {
break;
}
let cur_pt_ptr = paddr_to_vaddr(cur_pt_addr) as *mut E;
// SAFETY:
// - The page table node is alive because (1) the root node is alive and (2) all child nodes cannot
// be recycled if there are cursors.
// - The index is inside the bound, so the page table entry is valid.
// - All page table entries are aligned and accessed with atomic operations only.
let cur_pte = unsafe { super::load_pte(cur_pt_ptr.add(start_idx), Ordering::Acquire) };
if cur_pte.is_present() {
if cur_pte.is_last(cursor.level) {
break;
} else {
cur_pt_addr = cur_pte.paddr();
}
} else {
break;
}
cursor.level -= 1;
}
// SAFETY: The address and level corresponds to a child converted into
// a PTE and we clone it to get a new handle to the node.
let raw = unsafe { RawPageTableNode::<E, C>::from_raw_parts(cur_pt_addr, cursor.level) };
let _inc_ref = ManuallyDrop::new(raw.clone_shallow());
let lock = raw.lock();
cursor.guards[cursor.level as usize - 1] = Some(lock);
cursor.guard_level = cursor.level;
Ok(cursor)
Ok(locking::lock_range(pt, va, new_pt_is_tracked))
}
/// Gets the information of the current slot.
@ -224,11 +126,19 @@ impl<'a, M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait> Cursor<
let level = self.level;
let va = self.va;
match self.cur_entry().to_owned() {
Child::PageTable(pt) => {
self.push_level(pt.lock());
let entry = self.cur_entry();
match entry.to_ref() {
Child::PageTableRef(pt) => {
let paddr = pt.start_paddr();
// SAFETY: `pt` points to a PT that is attached to a node
// in the locked sub-tree, so that it is locked and alive.
self.push_level(unsafe { PageTableGuard::<E, C>::from_raw_paddr(paddr) });
continue;
}
Child::PageTable(_) => {
unreachable!();
}
Child::None => {
return Ok(PageTableItem::NotMapped {
va,
@ -302,41 +212,36 @@ impl<'a, M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait> Cursor<
}
/// Goes up a level.
///
/// This method releases the previously acquired lock at the discarded level.
fn pop_level(&mut self) {
debug_assert!(self.guards[self.level as usize - 1].is_some());
self.guards[self.level as usize - 1] = None;
let Some(taken) = self.path[self.level as usize - 1].take() else {
panic!("Popping a level without a lock");
};
let _taken = taken.into_raw_paddr();
self.level += 1;
// TODO: Drop the page table if it is empty (it may be necessary to
// rewalk from the top if all the locks have been released).
}
/// Goes down a level to a child page table.
///
/// The lock on the child page table is held until the next [`Self::pop_level`]
/// call at the same level.
fn push_level(&mut self, child_pt: PageTableNode<E, C>) {
fn push_level(&mut self, child_guard: PageTableGuard<'a, E, C>) {
self.level -= 1;
debug_assert_eq!(self.level, child_pt.level());
debug_assert_eq!(self.level, child_guard.level());
self.guards[self.level as usize - 1] = Some(child_pt);
let old = self.path[self.level as usize - 1].replace(child_guard);
debug_assert!(old.is_none());
}
fn should_map_as_tracked(&self) -> bool {
(TypeId::of::<M>() == TypeId::of::<KernelMode>()
|| TypeId::of::<M>() == TypeId::of::<UserMode>())
&& should_map_as_tracked(self.va)
}
fn cur_entry(&mut self) -> Entry<'_, E, C> {
let node = self.guards[self.level as usize - 1].as_mut().unwrap();
fn cur_entry<'s>(&'s mut self) -> Entry<'s, 'a, E, C> {
let node = self.path[self.level as usize - 1].as_mut().unwrap();
node.entry(pte_index::<C>(self.va, self.level))
}
}
impl<M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait> Drop for Cursor<'_, M, E, C> {
fn drop(&mut self) {
locking::unlock_range(self);
}
}
impl<M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait> Iterator
for Cursor<'_, M, E, C>
{
@ -353,23 +258,21 @@ impl<M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait> Iterator
/// The cursor of a page table that is capable of map, unmap or protect pages.
///
/// Also, it has all the capabilities of a [`Cursor`]. A virtual address range
/// in a page table can only be accessed by one cursor whether it is mutable or not.
/// It has all the capabilities of a [`Cursor`], which can navigate over the
/// page table corresponding to the address range. A virtual address range
/// in a page table can only be accessed by one cursor, regardless of the
/// mutability of the cursor.
#[derive(Debug)]
pub struct CursorMut<'a, M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait>(
Cursor<'a, M, E, C>,
);
impl<'a, M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait> CursorMut<'a, M, E, C> {
/// Creates a cursor claiming the write access for the given range.
/// Creates a cursor claiming exclusive access over the given range.
///
/// The cursor created will only be able to map, query or jump within the given
/// range. Out-of-bound accesses will result in panics or errors as return values,
/// depending on the access method.
///
/// Note that this function, the same as [`Cursor::new`], does not ensure exclusive
/// access to the claimed virtual address range. The accesses using this cursor may
/// block or fail.
pub(super) fn new(
pt: &'a PageTable<M, E, C>,
va: &Range<Vaddr>,
@ -416,32 +319,36 @@ impl<'a, M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait> CursorM
/// not affect kernel's memory safety.
pub unsafe fn map(
&mut self,
page: Frame<dyn AnyFrameMeta>,
frame: Frame<dyn AnyFrameMeta>,
prop: PageProperty,
) -> Option<Frame<dyn AnyFrameMeta>> {
let end = self.0.va + page.size();
let end = self.0.va + frame.size();
assert!(end <= self.0.barrier_va.end);
// Go down if not applicable.
while self.0.level > C::HIGHEST_TRANSLATION_LEVEL
while self.0.level > frame.map_level()
|| self.0.va % page_size::<C>(self.0.level) != 0
|| self.0.va + page_size::<C>(self.0.level) > end
{
debug_assert!(self.0.should_map_as_tracked());
let cur_level = self.0.level;
let cur_entry = self.0.cur_entry();
match cur_entry.to_owned() {
Child::PageTable(pt) => {
self.0.push_level(pt.lock());
debug_assert!(should_map_as_tracked::<M>(self.0.va));
let mut cur_entry = self.0.cur_entry();
match cur_entry.to_ref() {
Child::PageTableRef(pt) => {
let paddr = pt.start_paddr();
// SAFETY: `pt` points to a PT that is attached to a node
// in the locked sub-tree, so that it is locked and alive.
self.0
.push_level(unsafe { PageTableGuard::<E, C>::from_raw_paddr(paddr) });
}
Child::PageTable(_) => {
unreachable!();
}
Child::None => {
let pt =
PageTableNode::<E, C>::alloc(cur_level - 1, MapTrackingStatus::Tracked);
let _ = cur_entry.replace(Child::PageTable(pt.clone_raw()));
self.0.push_level(pt);
let child_guard = cur_entry.alloc_if_none(MapTrackingStatus::Tracked).unwrap();
self.0.push_level(child_guard);
}
Child::Frame(_, _) => {
panic!("Mapping a smaller page in an already mapped huge page");
panic!("Mapping a smaller frame in an already mapped huge page");
}
Child::Untracked(_, _, _) => {
panic!("Mapping a tracked page in an untracked range");
@ -449,20 +356,25 @@ impl<'a, M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait> CursorM
}
continue;
}
debug_assert_eq!(self.0.level, page.level());
debug_assert_eq!(self.0.level, frame.map_level());
// Map the current page.
let old = self.0.cur_entry().replace(Child::Frame(page, prop));
self.0.move_forward();
let mut cur_entry = self.0.cur_entry();
let old = cur_entry.replace(Child::Frame(frame, prop));
match old {
let old_frame = match old {
Child::Frame(old_page, _) => Some(old_page),
Child::None => None,
Child::PageTable(_) => {
todo!("Dropping page table nodes while mapping requires TLB flush")
}
Child::Untracked(_, _, _) => panic!("Mapping a tracked page in an untracked range"),
}
Child::PageTableRef(_) => unreachable!(),
};
self.0.move_forward();
old_frame
}
/// Maps the range starting from the current address to a physical address range.
@ -509,19 +421,23 @@ impl<'a, M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait> CursorM
|| self.0.va + page_size::<C>(self.0.level) > end
|| pa % page_size::<C>(self.0.level) != 0
{
let cur_level = self.0.level;
let cur_entry = self.0.cur_entry();
match cur_entry.to_owned() {
Child::PageTable(pt) => {
self.0.push_level(pt.lock());
let mut cur_entry = self.0.cur_entry();
match cur_entry.to_ref() {
Child::PageTableRef(pt) => {
let paddr = pt.start_paddr();
// SAFETY: `pt` points to a PT that is attached to a node
// in the locked sub-tree, so that it is locked and alive.
self.0
.push_level(unsafe { PageTableGuard::<E, C>::from_raw_paddr(paddr) });
}
Child::PageTable(_) => {
unreachable!();
}
Child::None => {
let pt = PageTableNode::<E, C>::alloc(
cur_level - 1,
MapTrackingStatus::Untracked,
);
let _ = cur_entry.replace(Child::PageTable(pt.clone_raw()));
self.0.push_level(pt);
let child_guard = cur_entry
.alloc_if_none(MapTrackingStatus::Untracked)
.unwrap();
self.0.push_level(child_guard);
}
Child::Frame(_, _) => {
panic!("Mapping a smaller page in an already mapped huge page");
@ -534,13 +450,12 @@ impl<'a, M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait> CursorM
continue;
}
// Map the current page.
debug_assert!(!self.0.should_map_as_tracked());
let level = self.0.level;
let _ = self
.0
.cur_entry()
.replace(Child::Untracked(pa, level, prop));
// Map the current page.
debug_assert!(!should_map_as_tracked::<M>(self.0.va));
let mut cur_entry = self.0.cur_entry();
let _ = cur_entry.replace(Child::Untracked(pa, level, prop));
// Move forward.
pa += page_size::<C>(level);
@ -581,7 +496,7 @@ impl<'a, M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait> CursorM
while self.0.va < end {
let cur_va = self.0.va;
let cur_level = self.0.level;
let cur_entry = self.0.cur_entry();
let mut cur_entry = self.0.cur_entry();
// Skip if it is already absent.
if cur_entry.is_none() {
@ -593,20 +508,24 @@ impl<'a, M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait> CursorM
continue;
}
// Go down if not applicable or if the entry points to a child page table.
// Go down if not applicable.
if cur_entry.is_node()
|| cur_va % page_size::<C>(cur_level) != 0
|| cur_va + page_size::<C>(cur_level) > end
{
let child = cur_entry.to_owned();
let child = cur_entry.to_ref();
match child {
Child::PageTable(pt) => {
let pt = pt.lock();
Child::PageTableRef(pt) => {
let paddr = pt.start_paddr();
// SAFETY: `pt` points to a PT that is attached to a node
// in the locked sub-tree, so that it is locked and alive.
let pt = unsafe { PageTableGuard::<E, C>::from_raw_paddr(paddr) };
// If there's no mapped PTEs in the next level, we can
// skip to save time.
if pt.nr_children() != 0 {
self.0.push_level(pt);
} else {
let _ = pt.into_raw_paddr();
if self.0.va + page_size::<C>(self.0.level) > end {
self.0.va = end;
break;
@ -614,6 +533,9 @@ impl<'a, M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait> CursorM
self.0.move_forward();
}
}
Child::PageTable(_) => {
unreachable!();
}
Child::None => {
unreachable!("Already checked");
}
@ -645,7 +567,7 @@ impl<'a, M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait> CursorM
prop,
}
}
Child::PageTable(_) | Child::None => unreachable!(),
Child::PageTable(_) | Child::None | Child::PageTableRef(_) => unreachable!(),
};
self.0.move_forward();
@ -702,15 +624,19 @@ impl<'a, M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait> CursorM
// Go down if it's not a last entry.
if cur_entry.is_node() {
let Child::PageTable(pt) = cur_entry.to_owned() else {
let Child::PageTableRef(pt) = cur_entry.to_ref() else {
unreachable!("Already checked");
};
let pt = pt.lock();
let paddr = pt.start_paddr();
// SAFETY: `pt` points to a PT that is attached to a node
// in the locked sub-tree, so that it is locked and alive.
let pt = unsafe { PageTableGuard::<E, C>::from_raw_paddr(paddr) };
// If there's no mapped PTEs in the next level, we can
// skip to save time.
if pt.nr_children() != 0 {
self.0.push_level(pt);
} else {
pt.into_raw_paddr();
self.0.move_forward();
}
continue;
@ -781,21 +707,26 @@ impl<'a, M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait> CursorM
let src_va = src.0.va;
let mut src_entry = src.0.cur_entry();
match src_entry.to_owned() {
Child::PageTable(pt) => {
let pt = pt.lock();
match src_entry.to_ref() {
Child::PageTableRef(pt) => {
let paddr = pt.start_paddr();
// SAFETY: `pt` points to a PT that is attached to a node
// in the locked sub-tree, so that it is locked and alive.
let pt = unsafe { PageTableGuard::<E, C>::from_raw_paddr(paddr) };
// If there's no mapped PTEs in the next level, we can
// skip to save time.
if pt.nr_children() != 0 {
src.0.push_level(pt);
} else {
pt.into_raw_paddr();
src.0.move_forward();
}
continue;
}
Child::PageTable(_) => {
unreachable!();
}
Child::None => {
src.0.move_forward();
continue;
}
Child::Untracked(_, _, _) => {
panic!("Copying untracked mappings");
@ -821,3 +752,9 @@ impl<'a, M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait> CursorM
}
}
}
fn should_map_as_tracked<M: PageTableMode>(va: Vaddr) -> bool {
(TypeId::of::<M>() == TypeId::of::<KernelMode>()
|| TypeId::of::<M>() == TypeId::of::<UserMode>())
&& crate::mm::kspace::should_map_as_tracked(va)
}

View File

@ -85,7 +85,7 @@ pub struct PageTable<
E: PageTableEntryTrait = PageTableEntry,
C: PagingConstsTrait = PagingConsts,
> {
root: RawPageTableNode<E, C>,
root: PageTableNode<E, C>,
_phantom: PhantomData<M>,
}
@ -106,10 +106,11 @@ impl PageTable<UserMode> {
/// 1. No other cursors are accessing the page table.
/// 2. No other CPUs activates the page table.
pub(in crate::mm) unsafe fn clear(&self) {
let mut root_node = self.root.clone_shallow().lock();
let _guard = crate::task::disable_preempt();
let mut root_node = self.root.lock();
const NR_PTES_PER_NODE: usize = nr_subpage_per_huge::<PagingConsts>();
for i in 0..NR_PTES_PER_NODE / 2 {
let root_entry = root_node.entry(i);
let mut root_entry = root_node.entry(i);
if !root_entry.is_none() {
let old = root_entry.replace(Child::None);
// Since no others are accessing the old child, dropping it is fine.
@ -120,64 +121,66 @@ impl PageTable<UserMode> {
}
impl PageTable<KernelMode> {
/// Create a new kernel page table.
pub(crate) fn new_kernel_page_table() -> Self {
let kpt = Self::empty();
// Make shared the page tables mapped by the root table in the kernel space.
{
const NR_PTES_PER_NODE: usize = nr_subpage_per_huge::<PagingConsts>();
let kernel_space_range = NR_PTES_PER_NODE / 2..NR_PTES_PER_NODE;
let _guard = crate::task::disable_preempt();
let mut root_node = kpt.root.lock();
for i in kernel_space_range {
let mut root_entry = root_node.entry(i);
let is_tracked = if super::kspace::should_map_as_tracked(
i * page_size::<PagingConsts>(PagingConsts::NR_LEVELS - 1),
) {
MapTrackingStatus::Tracked
} else {
MapTrackingStatus::Untracked
};
let _ = root_entry.alloc_if_none(is_tracked).unwrap();
}
}
kpt
}
/// Create a new user page table.
///
/// This should be the only way to create the user page table, that is to
/// duplicate the kernel page table with all the kernel mappings shared.
pub fn create_user_page_table(&self) -> PageTable<UserMode> {
let mut root_node = self.root.clone_shallow().lock();
let mut new_node =
let _preempt_guard = crate::task::disable_preempt();
let mut root_node = self.root.lock();
let new_root =
PageTableNode::alloc(PagingConsts::NR_LEVELS, MapTrackingStatus::NotApplicable);
let mut new_node = new_root.lock();
// Make a shallow copy of the root node in the kernel space range.
// The user space range is not copied.
const NR_PTES_PER_NODE: usize = nr_subpage_per_huge::<PagingConsts>();
for i in NR_PTES_PER_NODE / 2..NR_PTES_PER_NODE {
let root_entry = root_node.entry(i);
if !root_entry.is_none() {
let _ = new_node.entry(i).replace(root_entry.to_owned());
}
let child = root_entry.to_ref();
let Child::PageTableRef(pt) = child else {
panic!("The kernel page table doesn't contain shared nodes");
};
let pt_cloned = pt.clone();
let _ = new_node.entry(i).replace(Child::PageTable(pt_cloned));
}
drop(new_node);
PageTable::<UserMode> {
root: new_node.into_raw(),
root: new_root,
_phantom: PhantomData,
}
}
/// Explicitly make a range of virtual addresses shared between the kernel and user
/// page tables. Mapped pages before generating user page tables are shared either.
/// The virtual address range should be aligned to the root level page size. Considering
/// usize overflows, the caller should provide the index range of the root level pages
/// instead of the virtual address range.
pub fn make_shared_tables(&self, root_index: Range<usize>) {
const NR_PTES_PER_NODE: usize = nr_subpage_per_huge::<PagingConsts>();
let start = root_index.start;
debug_assert!(start >= NR_PTES_PER_NODE / 2);
debug_assert!(start < NR_PTES_PER_NODE);
let end = root_index.end;
debug_assert!(end <= NR_PTES_PER_NODE);
let mut root_node = self.root.clone_shallow().lock();
for i in start..end {
let root_entry = root_node.entry(i);
if root_entry.is_none() {
let nxt_level = PagingConsts::NR_LEVELS - 1;
let is_tracked = if super::kspace::should_map_as_tracked(
i * page_size::<PagingConsts>(nxt_level),
) {
MapTrackingStatus::Tracked
} else {
MapTrackingStatus::Untracked
};
let node = PageTableNode::alloc(nxt_level, is_tracked);
let _ = root_entry.replace(Child::PageTable(node.into_raw()));
}
}
}
/// Protect the given virtual address range in the kernel page table.
///
/// This method flushes the TLB entries when doing protection.
@ -200,11 +203,12 @@ impl PageTable<KernelMode> {
}
impl<'a, M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait> PageTable<M, E, C> {
/// Create a new empty page table. Useful for the kernel page table and IOMMU page tables only.
/// Create a new empty page table.
///
/// Useful for the IOMMU page tables only.
pub fn empty() -> Self {
PageTable {
root: PageTableNode::<E, C>::alloc(C::NR_LEVELS, MapTrackingStatus::NotApplicable)
.into_raw(),
root: PageTableNode::<E, C>::alloc(C::NR_LEVELS, MapTrackingStatus::NotApplicable),
_phantom: PhantomData,
}
}
@ -218,7 +222,7 @@ impl<'a, M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait> PageTab
/// It is dangerous to directly provide the physical address of the root page table to the
/// hardware since the page table node may be dropped, resulting in UAF.
pub unsafe fn root_paddr(&self) -> Paddr {
self.root.paddr()
self.root.start_paddr()
}
pub unsafe fn map(
@ -267,7 +271,7 @@ impl<'a, M: PageTableMode, E: PageTableEntryTrait, C: PagingConstsTrait> PageTab
/// This is only useful for IOMMU page tables. Think twice before using it in other cases.
pub unsafe fn shallow_copy(&self) -> Self {
PageTable {
root: self.root.clone_shallow(),
root: self.root.clone(),
_phantom: PhantomData,
}
}

View File

@ -4,34 +4,29 @@
use core::{mem::ManuallyDrop, panic};
use super::{MapTrackingStatus, PageTableEntryTrait, RawPageTableNode};
use crate::{
arch::mm::{PageTableEntry, PagingConsts},
mm::{
use super::{MapTrackingStatus, PageTableEntryTrait, PageTableNode, PageTableNodeRef};
use crate::mm::{
frame::{inc_frame_ref_count, meta::AnyFrameMeta, Frame},
page_prop::PageProperty,
Paddr, PagingConstsTrait, PagingLevel,
},
};
/// A child of a page table node.
///
/// This is a owning handle to a child of a page table node. If the child is
/// either a page table node or a page, it holds a reference count to the
/// corresponding page.
// TODO: Distinguish between the reference and the owning child.
#[derive(Debug)]
pub(in crate::mm) enum Child<
E: PageTableEntryTrait = PageTableEntry,
C: PagingConstsTrait = PagingConsts,
> {
PageTable(RawPageTableNode<E, C>),
pub(in crate::mm) enum Child<'a, E: PageTableEntryTrait, C: PagingConstsTrait> {
/// A owning handle to a raw page table node.
PageTable(PageTableNode<E, C>),
/// A reference of a child page table node.
PageTableRef(PageTableNodeRef<'a, E, C>),
/// A mapped frame.
Frame(Frame<dyn AnyFrameMeta>, PageProperty),
/// Pages not tracked by handles.
/// Mapped frames that are not tracked by handles.
Untracked(Paddr, PagingLevel, PageProperty),
None,
}
impl<E: PageTableEntryTrait, C: PagingConstsTrait> Child<E, C> {
impl<E: PageTableEntryTrait, C: PagingConstsTrait> Child<'_, E, C> {
/// Returns whether the child does not map to anything.
pub(in crate::mm) fn is_none(&self) -> bool {
matches!(self, Child::None)
@ -48,8 +43,9 @@ impl<E: PageTableEntryTrait, C: PagingConstsTrait> Child<E, C> {
) -> bool {
match self {
Child::PageTable(pt) => node_level == pt.level() + 1,
Child::PageTableRef(_) => false,
Child::Frame(p, _) => {
node_level == p.level() && is_tracked == MapTrackingStatus::Tracked
node_level == p.map_level() && is_tracked == MapTrackingStatus::Tracked
}
Child::Untracked(_, level, _) => {
node_level == *level && is_tracked == MapTrackingStatus::Untracked
@ -71,10 +67,13 @@ impl<E: PageTableEntryTrait, C: PagingConstsTrait> Child<E, C> {
match self {
Child::PageTable(pt) => {
let pt = ManuallyDrop::new(pt);
E::new_pt(pt.paddr())
E::new_pt(pt.start_paddr())
}
Child::PageTableRef(_) => {
panic!("`PageTableRef` should not be converted to PTE");
}
Child::Frame(page, prop) => {
let level = page.level();
let level = page.map_level();
E::new_page(page.into_raw(), level, prop)
}
Child::Untracked(pa, level, prop) => E::new_page(pa, level, prop),
@ -108,7 +107,9 @@ impl<E: PageTableEntryTrait, C: PagingConstsTrait> Child<E, C> {
if !pte.is_last(level) {
// SAFETY: The physical address points to a valid page table node
// at the given level.
return Child::PageTable(unsafe { RawPageTableNode::from_raw_parts(paddr, level - 1) });
let pt = unsafe { PageTableNode::from_raw(paddr) };
debug_assert_eq!(pt.level(), level - 1);
return Child::PageTable(pt);
}
match is_tracked {
@ -122,7 +123,12 @@ impl<E: PageTableEntryTrait, C: PagingConstsTrait> Child<E, C> {
}
}
/// Gains an extra owning reference to the child.
/// Gains an extra reference to the child.
///
/// If the child is a frame, it increases the reference count of the frame.
///
/// If the child is a page table node, it returns a [`PageTableNodeRef`],
/// thus not affecting the reference count of the page table node.
///
/// # Safety
///
@ -131,7 +137,7 @@ impl<E: PageTableEntryTrait, C: PagingConstsTrait> Child<E, C> {
///
/// This method must not be used with a PTE that has been restored to a
/// child using the [`Child::from_pte`] method.
pub(super) unsafe fn clone_from_pte(
pub(super) unsafe fn ref_from_pte(
pte: &E,
level: PagingLevel,
is_tracked: MapTrackingStatus,
@ -143,12 +149,9 @@ impl<E: PageTableEntryTrait, C: PagingConstsTrait> Child<E, C> {
let paddr = pte.paddr();
if !pte.is_last(level) {
// SAFETY: The physical address is valid and the PTE already owns
// the reference to the page.
unsafe { inc_frame_ref_count(paddr) };
// SAFETY: The physical address points to a valid page table node
// at the given level.
return Child::PageTable(unsafe { RawPageTableNode::from_raw_parts(paddr, level - 1) });
// SAFETY: If the caller ensures that the PTE is from a `Child`,
// restoring the reference is safe.
return Child::PageTableRef(unsafe { PageTableNodeRef::borrow_paddr(paddr) });
}
match is_tracked {

View File

@ -2,17 +2,17 @@
//! This module provides accessors to the page table entries in a node.
use super::{Child, MapTrackingStatus, PageTableEntryTrait, PageTableNode};
use super::{Child, MapTrackingStatus, PageTableEntryTrait, PageTableGuard, PageTableNode};
use crate::mm::{nr_subpage_per_huge, page_prop::PageProperty, page_size, PagingConstsTrait};
/// A view of an entry in a page table node.
///
/// It can be borrowed from a node using the [`PageTableNode::entry`] method.
/// It can be borrowed from a node using the [`PageTableGuard::entry`] method.
///
/// This is a static reference to an entry in a node that does not account for
/// a dynamic reference count to the child. It can be used to create a owned
/// handle, which is a [`Child`].
pub(in crate::mm) struct Entry<'a, E: PageTableEntryTrait, C: PagingConstsTrait> {
pub(in crate::mm) struct Entry<'guard, 'pt, E: PageTableEntryTrait, C: PagingConstsTrait> {
/// The page table entry.
///
/// We store the page table entry here to optimize the number of reads from
@ -24,10 +24,10 @@ pub(in crate::mm) struct Entry<'a, E: PageTableEntryTrait, C: PagingConstsTrait>
/// The index of the entry in the node.
idx: usize,
/// The node that contains the entry.
node: &'a mut PageTableNode<E, C>,
node: &'guard mut PageTableGuard<'pt, E, C>,
}
impl<'a, E: PageTableEntryTrait, C: PagingConstsTrait> Entry<'a, E, C> {
impl<'guard, 'pt, E: PageTableEntryTrait, C: PagingConstsTrait> Entry<'guard, 'pt, E, C> {
/// Returns if the entry does not map to anything.
pub(in crate::mm) fn is_none(&self) -> bool {
!self.pte.is_present()
@ -38,11 +38,11 @@ impl<'a, E: PageTableEntryTrait, C: PagingConstsTrait> Entry<'a, E, C> {
self.pte.is_present() && !self.pte.is_last(self.node.level())
}
/// Gets a owned handle to the child.
pub(in crate::mm) fn to_owned(&self) -> Child<E, C> {
/// Gets a reference to the child.
pub(in crate::mm) fn to_ref(&self) -> Child<'_, E, C> {
// SAFETY: The entry structure represents an existent entry with the
// right node information.
unsafe { Child::clone_from_pte(&self.pte, self.node.level(), self.node.is_tracked()) }
unsafe { Child::ref_from_pte(&self.pte, self.node.level(), self.node.is_tracked()) }
}
/// Operates on the mapping properties of the entry.
@ -79,7 +79,7 @@ impl<'a, E: PageTableEntryTrait, C: PagingConstsTrait> Entry<'a, E, C> {
///
/// The method panics if the given child is not compatible with the node.
/// The compatibility is specified by the [`Child::is_compatible`].
pub(in crate::mm) fn replace(self, new_child: Child<E, C>) -> Child<E, C> {
pub(in crate::mm) fn replace(&mut self, new_child: Child<E, C>) -> Child<E, C> {
assert!(new_child.is_compatible(self.node.level(), self.node.is_tracked()));
// SAFETY: The entry structure represents an existent entry with the
@ -94,14 +94,51 @@ impl<'a, E: PageTableEntryTrait, C: PagingConstsTrait> Entry<'a, E, C> {
*self.node.nr_children_mut() -= 1;
}
let new_pte = new_child.into_pte();
// SAFETY:
// 1. The index is within the bounds.
// 2. The new PTE is compatible with the page table node, as asserted above.
unsafe { self.node.write_pte(self.idx, new_child.into_pte()) };
unsafe { self.node.write_pte(self.idx, new_pte) };
self.pte = new_pte;
old_child
}
/// Allocates a new child page table node and replaces the entry with it.
///
/// If the old entry is not none, the operation will fail and return `None`.
/// Otherwise, the lock guard of the new child page table node is returned.
pub(in crate::mm::page_table) fn alloc_if_none(
&mut self,
new_pt_is_tracked: MapTrackingStatus,
) -> Option<PageTableGuard<'pt, E, C>> {
if !(self.is_none() && self.node.level() > 1) {
return None;
}
let level = self.node.level();
let new_page = PageTableNode::<E, C>::alloc(level - 1, new_pt_is_tracked);
let guard_addr = new_page.lock().into_raw_paddr();
// SAFETY:
// 1. The index is within the bounds.
// 2. The new PTE is compatible with the page table node.
unsafe {
self.node
.write_pte(self.idx, Child::PageTable(new_page).into_pte())
};
*self.node.nr_children_mut() += 1;
// SAFETY: The resulting guard lifetime (`'a`) is no shorter than the
// lifetime of the current entry (`'a`), because we store the allocated
// page table in the current node.
Some(unsafe { PageTableGuard::from_raw_paddr(guard_addr) })
}
/// Splits the entry to smaller pages if it maps to a untracked huge page.
///
/// If the entry does map to a untracked huge page, it is split into smaller
@ -110,7 +147,9 @@ impl<'a, E: PageTableEntryTrait, C: PagingConstsTrait> Entry<'a, E, C> {
///
/// If the entry does not map to a untracked huge page, the method returns
/// `None`.
pub(in crate::mm) fn split_if_untracked_huge(self) -> Option<PageTableNode<E, C>> {
pub(in crate::mm::page_table) fn split_if_untracked_huge(
&mut self,
) -> Option<PageTableGuard<'pt, E, C>> {
let level = self.node.level();
if !(self.pte.is_last(level)
@ -123,27 +162,44 @@ impl<'a, E: PageTableEntryTrait, C: PagingConstsTrait> Entry<'a, E, C> {
let pa = self.pte.paddr();
let prop = self.pte.prop();
let mut new_page = PageTableNode::<E, C>::alloc(level - 1, MapTrackingStatus::Untracked);
let new_page = PageTableNode::<E, C>::alloc(level - 1, MapTrackingStatus::Untracked);
let mut guard = new_page.lock();
for i in 0..nr_subpage_per_huge::<C>() {
let small_pa = pa + i * page_size::<C>(level - 1);
let _ = new_page
.entry(i)
.replace(Child::Untracked(small_pa, level - 1, prop));
let mut entry = guard.entry(i);
let old = entry.replace(Child::Untracked(small_pa, level - 1, prop));
debug_assert!(old.is_none());
}
let _ = self.replace(Child::PageTable(new_page.clone_raw()));
let guard_addr = guard.into_raw_paddr();
Some(new_page)
// SAFETY:
// 1. The index is within the bounds.
// 2. The new PTE is compatible with the page table node.
unsafe {
self.node
.write_pte(self.idx, Child::PageTable(new_page).into_pte())
};
// SAFETY: The resulting guard lifetime (`'a`) is no shorter than the
// lifetime of the current entry (`'a`), because we store the allocated
// page table in the current node.
Some(unsafe { PageTableGuard::from_raw_paddr(guard_addr) })
}
/// Create a new entry at the node.
/// Create a new entry at the node with guard.
///
/// # Safety
///
/// The caller must ensure that the index is within the bounds of the node.
pub(super) unsafe fn new_at(node: &'a mut PageTableNode<E, C>, idx: usize) -> Self {
pub(super) unsafe fn new_at(guard: &'guard mut PageTableGuard<'pt, E, C>, idx: usize) -> Self {
// SAFETY: The index is within the bound.
let pte = unsafe { node.read_pte(idx) };
Self { pte, idx, node }
let pte = unsafe { guard.read_pte(idx) };
Self {
pte,
idx,
node: guard,
}
}
}

View File

@ -31,54 +31,61 @@ mod entry;
use core::{
cell::SyncUnsafeCell,
marker::PhantomData,
mem::ManuallyDrop,
ops::Deref,
sync::atomic::{AtomicU8, Ordering},
};
pub(in crate::mm) use self::{child::Child, entry::Entry};
use super::{nr_subpage_per_huge, PageTableEntryTrait};
use crate::{
arch::mm::{PageTableEntry, PagingConsts},
mm::{
frame::{inc_frame_ref_count, meta::AnyFrameMeta, Frame},
use crate::mm::{
frame::{meta::AnyFrameMeta, Frame, FrameRef},
paddr_to_vaddr,
page_table::{load_pte, store_pte},
FrameAllocOptions, Infallible, Paddr, PagingConstsTrait, PagingLevel, VmReader,
},
};
/// The raw handle to a page table node.
/// A smart pointer to a page table node.
///
/// This handle is a referencer of a page table node. Thus creating and dropping it will affect
/// the reference count of the page table node. If dropped the raw handle as the last reference,
/// the page table node and subsequent children will be freed.
/// This smart pointer is an owner of a page table node. Thus creating and
/// dropping it will affect the reference count of the page table node. If
/// dropped it as the last reference, the page table node and subsequent
/// children will be freed.
///
/// Only the CPU or a PTE can access a page table node using a raw handle. To access the page
/// table node from the kernel code, use the handle [`PageTableNode`].
#[derive(Debug)]
pub(super) struct RawPageTableNode<E: PageTableEntryTrait, C: PagingConstsTrait> {
raw: Paddr,
level: PagingLevel,
_phantom: PhantomData<(E, C)>,
}
/// [`PageTableNode`] is read-only. To modify the page table node, lock and use
/// [`PageTableGuard`].
pub(super) type PageTableNode<E, C> = Frame<PageTablePageMeta<E, C>>;
impl<E: PageTableEntryTrait, C: PagingConstsTrait> RawPageTableNode<E, C> {
pub(super) fn paddr(&self) -> Paddr {
self.raw
}
/// A reference to a page table node.
pub(super) type PageTableNodeRef<'a, E, C> = FrameRef<'a, PageTablePageMeta<E, C>>;
impl<E: PageTableEntryTrait, C: PagingConstsTrait> PageTableNode<E, C> {
pub(super) fn level(&self) -> PagingLevel {
self.level
self.meta().level
}
/// Converts a raw handle to an accessible handle by pertaining the lock.
pub(super) fn lock(self) -> PageTableNode<E, C> {
let level = self.level;
let page: Frame<PageTablePageMeta<E, C>> = self.into();
pub(super) fn is_tracked(&self) -> MapTrackingStatus {
self.meta().is_tracked
}
// Acquire the lock.
let meta = page.meta();
while meta
/// Allocates a new empty page table node.
///
/// This function returns a locked owning guard.
pub(super) fn alloc(level: PagingLevel, is_tracked: MapTrackingStatus) -> Self {
let meta = PageTablePageMeta::new(level, is_tracked);
let frame = FrameAllocOptions::new()
.zeroed(true)
.alloc_frame_with(meta)
.expect("Failed to allocate a page table node");
// The allocated frame is zeroed. Make sure zero is absent PTE.
debug_assert!(E::new_absent().as_bytes().iter().all(|&b| b == 0));
frame
}
/// Locks the page table node.
pub(super) fn lock(&self) -> PageTableGuard<'_, E, C> {
while self
.meta()
.lock
.compare_exchange(0, 1, Ordering::Acquire, Ordering::Relaxed)
.is_err()
@ -86,19 +93,8 @@ impl<E: PageTableEntryTrait, C: PagingConstsTrait> RawPageTableNode<E, C> {
core::hint::spin_loop();
}
debug_assert_eq!(page.meta().level, level);
PageTableNode::<E, C> { page }
}
/// Creates a copy of the handle.
pub(super) fn clone_shallow(&self) -> Self {
self.inc_ref_count();
Self {
raw: self.raw,
level: self.level,
_phantom: PhantomData,
PageTableGuard::<'_, E, C> {
inner: self.borrow(),
}
}
@ -124,25 +120,18 @@ impl<E: PageTableEntryTrait, C: PagingConstsTrait> RawPageTableNode<E, C> {
mm::CachePolicy,
};
assert_eq!(self.level, C::NR_LEVELS);
assert_eq!(self.level(), C::NR_LEVELS);
let last_activated_paddr = current_page_table_paddr();
if last_activated_paddr == self.raw {
if last_activated_paddr == self.start_paddr() {
return;
}
activate_page_table(self.raw, CachePolicy::Writeback);
// Increment the reference count of the current page table.
self.inc_ref_count();
activate_page_table(self.clone().into_raw(), CachePolicy::Writeback);
// Restore and drop the last activated page table.
drop(Self {
raw: last_activated_paddr,
level: C::NR_LEVELS,
_phantom: PhantomData,
});
// SAFETY: The physical address is valid and points to a forgotten page table node.
drop(unsafe { Self::from_raw(last_activated_paddr) });
}
/// Activates the (root) page table assuming it is the first activation.
@ -152,135 +141,54 @@ impl<E: PageTableEntryTrait, C: PagingConstsTrait> RawPageTableNode<E, C> {
pub(super) unsafe fn first_activate(&self) {
use crate::{arch::mm::activate_page_table, mm::CachePolicy};
self.inc_ref_count();
activate_page_table(self.raw, CachePolicy::Writeback);
}
fn inc_ref_count(&self) {
// SAFETY: We have a reference count to the page and can safely increase the reference
// count by one more.
unsafe {
inc_frame_ref_count(self.paddr());
activate_page_table(self.clone().into_raw(), CachePolicy::Writeback);
}
}
/// Restores the handle from the physical address and level.
///
/// # Safety
///
/// The caller must ensure that the physical address is valid and points to
/// a forgotten page table node. A forgotten page table node can only be
/// restored once. The level must match the level of the page table node.
pub(super) unsafe fn from_raw_parts(paddr: Paddr, level: PagingLevel) -> Self {
Self {
raw: paddr,
level,
_phantom: PhantomData,
}
}
}
impl<E: PageTableEntryTrait, C: PagingConstsTrait> From<RawPageTableNode<E, C>>
for Frame<PageTablePageMeta<E, C>>
{
fn from(raw: RawPageTableNode<E, C>) -> Self {
let raw = ManuallyDrop::new(raw);
// SAFETY: The physical address in the raw handle is valid and we are
// transferring the ownership to a new handle. No increment of the reference
// count is needed.
unsafe { Frame::<PageTablePageMeta<E, C>>::from_raw(raw.paddr()) }
}
}
impl<E: PageTableEntryTrait, C: PagingConstsTrait> Drop for RawPageTableNode<E, C> {
fn drop(&mut self) {
// SAFETY: The physical address in the raw handle is valid. The restored
// handle is dropped to decrement the reference count.
drop(unsafe { Frame::<PageTablePageMeta<E, C>>::from_raw(self.paddr()) });
}
}
/// A mutable handle to a page table node.
///
/// The page table node can own a set of handles to children, ensuring that the children
/// don't outlive the page table node. Cloning a page table node will create a deep copy
/// of the page table. Dropping the page table node will also drop all handles if the page
/// table node has no references. You can set the page table node as a child of another
/// page table node.
/// A guard that holds the lock of a page table node.
#[derive(Debug)]
pub(super) struct PageTableNode<
E: PageTableEntryTrait = PageTableEntry,
C: PagingConstsTrait = PagingConsts,
> {
page: Frame<PageTablePageMeta<E, C>>,
pub(super) struct PageTableGuard<'a, E: PageTableEntryTrait, C: PagingConstsTrait> {
inner: PageTableNodeRef<'a, E, C>,
}
impl<E: PageTableEntryTrait, C: PagingConstsTrait> PageTableNode<E, C> {
impl<'a, E: PageTableEntryTrait, C: PagingConstsTrait> PageTableGuard<'a, E, C> {
/// Borrows an entry in the node at a given index.
///
/// # Panics
///
/// Panics if the index is not within the bound of
/// [`nr_subpage_per_huge<C>`].
pub(super) fn entry(&mut self, idx: usize) -> Entry<'_, E, C> {
pub(super) fn entry<'s>(&'s mut self, idx: usize) -> Entry<'s, 'a, E, C> {
assert!(idx < nr_subpage_per_huge::<C>());
// SAFETY: The index is within the bound.
unsafe { Entry::new_at(self, idx) }
}
/// Gets the level of the page table node.
pub(super) fn level(&self) -> PagingLevel {
self.page.meta().level
}
/// Gets the tracking status of the page table node.
pub(super) fn is_tracked(&self) -> MapTrackingStatus {
self.page.meta().is_tracked
}
/// Allocates a new empty page table node.
/// Converts the guard into a raw physical address.
///
/// This function returns an owning handle. The newly created handle does not
/// set the lock bit for performance as it is exclusive and unlocking is an
/// extra unnecessary expensive operation.
pub(super) fn alloc(level: PagingLevel, is_tracked: MapTrackingStatus) -> Self {
let meta = PageTablePageMeta::new_locked(level, is_tracked);
let page = FrameAllocOptions::new()
.zeroed(true)
.alloc_frame_with(meta)
.expect("Failed to allocate a page table node");
// The allocated frame is zeroed. Make sure zero is absent PTE.
debug_assert!(E::new_absent().as_bytes().iter().all(|&b| b == 0));
Self { page }
/// It will not release the lock. It may be paired with [`Self::from_raw_paddr`]
/// to manually manage pointers.
pub(super) fn into_raw_paddr(self) -> Paddr {
self.start_paddr()
}
/// Converts the handle into a raw handle to be stored in a PTE or CPU.
pub(super) fn into_raw(self) -> RawPageTableNode<E, C> {
let this = ManuallyDrop::new(self);
// Release the lock.
this.page.meta().lock.store(0, Ordering::Release);
// SAFETY: The provided physical address is valid and the level is
// correct. The reference count is not changed.
unsafe { RawPageTableNode::from_raw_parts(this.page.start_paddr(), this.page.meta().level) }
/// Converts a raw physical address to a guard.
///
/// # Safety
///
/// The caller must ensure that the physical address is valid and points to
/// a forgotten page table node that is locked (see [`Self::into_raw_paddr`]).
pub(super) unsafe fn from_raw_paddr(paddr: Paddr) -> Self {
Self {
// SAFETY: The caller ensures safety.
inner: unsafe { PageTableNodeRef::borrow_paddr(paddr) },
}
/// Gets a raw handle while still preserving the original handle.
pub(super) fn clone_raw(&self) -> RawPageTableNode<E, C> {
let page = ManuallyDrop::new(self.page.clone());
// SAFETY: The provided physical address is valid and the level is
// correct. The reference count is increased by one.
unsafe { RawPageTableNode::from_raw_parts(page.start_paddr(), page.meta().level) }
}
/// Gets the number of valid PTEs in the node.
pub(super) fn nr_children(&self) -> u16 {
// SAFETY: The lock is held so we have an exclusive access.
unsafe { *self.page.meta().nr_children.get() }
unsafe { *self.meta().nr_children.get() }
}
/// Reads a non-owning PTE at the given index.
@ -294,7 +202,7 @@ impl<E: PageTableEntryTrait, C: PagingConstsTrait> PageTableNode<E, C> {
/// The caller must ensure that the index is within the bound.
unsafe fn read_pte(&self, idx: usize) -> E {
debug_assert!(idx < nr_subpage_per_huge::<C>());
let ptr = paddr_to_vaddr(self.page.start_paddr()) as *mut E;
let ptr = paddr_to_vaddr(self.start_paddr()) as *mut E;
// SAFETY:
// - The page table node is alive. The index is inside the bound, so the page table entry is valid.
// - All page table entries are aligned and accessed with atomic operations only.
@ -316,7 +224,7 @@ impl<E: PageTableEntryTrait, C: PagingConstsTrait> PageTableNode<E, C> {
/// (see [`Child::is_compatible`]).
unsafe fn write_pte(&mut self, idx: usize, pte: E) {
debug_assert!(idx < nr_subpage_per_huge::<C>());
let ptr = paddr_to_vaddr(self.page.start_paddr()) as *mut E;
let ptr = paddr_to_vaddr(self.start_paddr()) as *mut E;
// SAFETY:
// - The page table node is alive. The index is inside the bound, so the page table entry is valid.
// - All page table entries are aligned and accessed with atomic operations only.
@ -326,24 +234,28 @@ impl<E: PageTableEntryTrait, C: PagingConstsTrait> PageTableNode<E, C> {
/// Gets the mutable reference to the number of valid PTEs in the node.
fn nr_children_mut(&mut self) -> &mut u16 {
// SAFETY: The lock is held so we have an exclusive access.
unsafe { &mut *self.page.meta().nr_children.get() }
unsafe { &mut *self.meta().nr_children.get() }
}
}
impl<E: PageTableEntryTrait, C: PagingConstsTrait> Drop for PageTableNode<E, C> {
impl<'a, E: PageTableEntryTrait, C: PagingConstsTrait> Deref for PageTableGuard<'a, E, C> {
type Target = PageTableNodeRef<'a, E, C>;
fn deref(&self) -> &Self::Target {
&self.inner
}
}
impl<E: PageTableEntryTrait, C: PagingConstsTrait> Drop for PageTableGuard<'_, E, C> {
fn drop(&mut self) {
// Release the lock.
self.page.meta().lock.store(0, Ordering::Release);
self.inner.meta().lock.store(0, Ordering::Release);
}
}
/// The metadata of any kinds of page table pages.
/// Make sure the the generic parameters don't effect the memory layout.
#[derive(Debug)]
pub(in crate::mm) struct PageTablePageMeta<
E: PageTableEntryTrait = PageTableEntry,
C: PagingConstsTrait = PagingConsts,
> {
pub(in crate::mm) struct PageTablePageMeta<E: PageTableEntryTrait, C: PagingConstsTrait> {
/// The number of valid PTEs. It is mutable if the lock is held.
pub nr_children: SyncUnsafeCell<u16>,
/// The level of the page table page. A page table page cannot be
@ -373,11 +285,11 @@ pub(in crate::mm) enum MapTrackingStatus {
}
impl<E: PageTableEntryTrait, C: PagingConstsTrait> PageTablePageMeta<E, C> {
pub fn new_locked(level: PagingLevel, is_tracked: MapTrackingStatus) -> Self {
pub fn new(level: PagingLevel, is_tracked: MapTrackingStatus) -> Self {
Self {
nr_children: SyncUnsafeCell::new(0),
level,
lock: AtomicU8::new(1),
lock: AtomicU8::new(0),
is_tracked,
_phantom: PhantomData,
}

View File

@ -146,38 +146,30 @@ mod create_page_table {
#[ktest]
fn create_user_page_table() {
let kernel_pt = PageTable::<KernelMode>::empty();
let kernel_pt = PageTable::<KernelMode>::new_kernel_page_table();
let user_pt = kernel_pt.create_user_page_table();
let mut kernel_root = kernel_pt.root.clone_shallow().lock();
let mut user_root = user_pt.root.clone_shallow().lock();
let mut kernel_root = kernel_pt.root.lock();
let mut user_root = user_pt.root.lock();
const NR_PTES_PER_NODE: usize = nr_subpage_per_huge::<PagingConsts>();
for i in NR_PTES_PER_NODE / 2..NR_PTES_PER_NODE {
let kernel_entry = kernel_root.entry(i);
let user_entry = user_root.entry(i);
assert_eq!(kernel_entry.is_node(), user_entry.is_node());
}
}
#[ktest]
fn make_shared_tables() {
let kernel_pt = PageTable::<KernelMode>::empty();
let shared_range =
(nr_subpage_per_huge::<PagingConsts>() / 2)..nr_subpage_per_huge::<PagingConsts>();
kernel_pt.make_shared_tables(shared_range.clone());
// Marks the specified root node index range as shared.
let mut root_node = kernel_pt.root.clone_shallow().lock();
for i in shared_range {
assert!(root_node.entry(i).is_node());
let Child::PageTableRef(kernel_node) = kernel_entry.to_ref() else {
panic!("Expected a node reference at {} of kernel root PT", i);
};
let Child::PageTableRef(user_node) = user_entry.to_ref() else {
panic!("Expected a node reference at {} of user root PT", i);
};
assert_eq!(kernel_node.start_paddr(), user_node.start_paddr());
}
}
#[ktest]
fn clear_user_page_table() {
// Creates a kernel page table.
let kernel_pt = PageTable::<KernelMode>::empty();
let kernel_pt = PageTable::<KernelMode>::new_kernel_page_table();
// Creates a user page table.
let user_pt = kernel_pt.create_user_page_table();
@ -597,9 +589,9 @@ mod tracked_mapping {
);
// Creates a child page table with copy-on-write protection.
let child_pt = {
let parent_range = 0..MAX_USERSPACE_VADDR;
let child_pt = setup_page_table::<UserMode>();
{
let parent_range = 0..MAX_USERSPACE_VADDR;
let mut child_cursor = child_pt.cursor_mut(&parent_range).unwrap();
let mut parent_cursor = page_table.cursor_mut(&parent_range).unwrap();
unsafe {
@ -609,7 +601,6 @@ mod tracked_mapping {
&mut remove_write_flag,
);
}
child_pt
};
// Confirms that parent and child VAs map to the same physical address.
@ -644,9 +635,9 @@ mod tracked_mapping {
);
// Creates a sibling page table (from the now-modified parent).
let sibling_pt = {
let parent_range = 0..MAX_USERSPACE_VADDR;
let sibling_pt = setup_page_table::<UserMode>();
{
let parent_range = 0..MAX_USERSPACE_VADDR;
let mut sibling_cursor = sibling_pt.cursor_mut(&parent_range).unwrap();
let mut parent_cursor = page_table.cursor_mut(&parent_range).unwrap();
unsafe {
@ -656,7 +647,6 @@ mod tracked_mapping {
&mut remove_write_flag,
);
}
sibling_pt
};
// Verifies that the sibling is unmapped as it was created after the parent unmapped the range.
@ -771,9 +761,6 @@ mod untracked_mapping {
let expected_pa_after = physical_range.start + (va_after - virtual_range.start);
assert_eq!(kernel_pt.query(va_after).unwrap().0, expected_pa_after);
}
// Prevents automatic drop to avoid memory leak in the test.
let _ = ManuallyDrop::new(kernel_pt);
}
#[ktest]