Merge changes from topic "microdroid-kernel-with-modules-proto"
* changes:
Add prebuilt kernel that supports loading modules
Add an arg to run-microdroid/run-app cmds to provide custom kernel
diff --git a/pvmfw/src/config.rs b/pvmfw/src/config.rs
index 8d0c047..4086af7 100644
--- a/pvmfw/src/config.rs
+++ b/pvmfw/src/config.rs
@@ -14,11 +14,11 @@
//! Support for the pvmfw configuration data format.
-use crate::helpers;
use core::fmt;
use core::mem;
use core::ops::Range;
use core::result;
+use vmbase::util::unchecked_align_up;
use zerocopy::{FromBytes, LayoutVerified};
/// Configuration data header.
@@ -101,8 +101,7 @@
impl Header {
const MAGIC: u32 = u32::from_ne_bytes(*b"pvmf");
const VERSION_1_0: u32 = Self::version(1, 0);
- const PADDED_SIZE: usize =
- helpers::unchecked_align_up(mem::size_of::<Self>(), mem::size_of::<u64>());
+ const PADDED_SIZE: usize = unchecked_align_up(mem::size_of::<Self>(), mem::size_of::<u64>());
pub const fn version(major: u16, minor: u16) -> u32 {
((major as u32) << 16) | (minor as u32)
diff --git a/pvmfw/src/dice.rs b/pvmfw/src/dice.rs
index 0aa1273..3116456 100644
--- a/pvmfw/src/dice.rs
+++ b/pvmfw/src/dice.rs
@@ -15,10 +15,10 @@
//! Support for DICE derivation and BCC generation.
use crate::cstr;
-use crate::helpers::flushed_zeroize;
use core::ffi::c_void;
use core::mem::size_of;
use core::slice;
+use vmbase::memory::flushed_zeroize;
use diced_open_dice::{
bcc_format_config_descriptor, bcc_handover_main_flow, hash, Config, DiceMode, Hash,
diff --git a/pvmfw/src/entry.rs b/pvmfw/src/entry.rs
index 762b88b..0731fb8 100644
--- a/pvmfw/src/entry.rs
+++ b/pvmfw/src/entry.rs
@@ -18,8 +18,6 @@
use crate::crypto;
use crate::fdt;
use crate::heap;
-use crate::helpers;
-use crate::helpers::RangeExt as _;
use crate::memory::{self, MemoryTracker, MEMORY};
use crate::rand;
use core::arch::asm;
@@ -33,7 +31,14 @@
use log::info;
use log::warn;
use log::LevelFilter;
-use vmbase::{console, layout, logger, main, power::reboot};
+use vmbase::util::RangeExt as _;
+use vmbase::{
+ console,
+ layout::{self, crosvm},
+ logger, main,
+ memory::{min_dcache_line_size, SIZE_2MB, SIZE_4KB},
+ power::reboot,
+};
use zeroize::Zeroize;
#[derive(Debug, Clone)]
@@ -84,7 +89,7 @@
impl<'a> MemorySlices<'a> {
fn new(fdt: usize, kernel: usize, kernel_size: usize) -> Result<Self, RebootReason> {
// SAFETY - SIZE_2MB is non-zero.
- const FDT_SIZE: NonZeroUsize = unsafe { NonZeroUsize::new_unchecked(helpers::SIZE_2MB) };
+ const FDT_SIZE: NonZeroUsize = unsafe { NonZeroUsize::new_unchecked(SIZE_2MB) };
// TODO - Only map the FDT as read-only, until we modify it right before jump_to_payload()
// e.g. by generating a DTBO for a template DT in main() and, on return, re-map DT as RW,
// overwrite with the template DT and apply the DTBO.
@@ -220,7 +225,12 @@
let (bcc_slice, debug_policy) = appended.get_entries();
// Up to this point, we were using the built-in static (from .rodata) page tables.
- MEMORY.lock().replace(MemoryTracker::new(page_table));
+ MEMORY.lock().replace(MemoryTracker::new(
+ page_table,
+ crosvm::MEM_START..memory::MAX_ADDR,
+ crosvm::MMIO_START..crosvm::MMIO_END,
+ memory::appended_payload_range(),
+ ));
let slices = MemorySlices::new(fdt, payload, payload_size)?;
@@ -369,7 +379,7 @@
scratch_end = in(reg) u64::try_from(scratch.end).unwrap(),
stack = in(reg) u64::try_from(stack.start).unwrap(),
stack_end = in(reg) u64::try_from(stack.end).unwrap(),
- dcache_line_size = in(reg) u64::try_from(helpers::min_dcache_line_size()).unwrap(),
+ dcache_line_size = in(reg) u64::try_from(min_dcache_line_size()).unwrap(),
in("x0") fdt_address,
in("x30") payload_start,
options(noreturn),
@@ -409,7 +419,7 @@
Some(Self::Config(config.unwrap()))
}
AppendedConfigType::NotFound if cfg!(feature = "legacy") => {
- const BCC_SIZE: usize = helpers::SIZE_4KB;
+ const BCC_SIZE: usize = SIZE_4KB;
warn!("Assuming the appended data at {:?} to be a raw BCC", data.as_ptr());
Some(Self::LegacyBcc(&mut data[..BCC_SIZE]))
}
diff --git a/pvmfw/src/exceptions.rs b/pvmfw/src/exceptions.rs
index 4b24d14..a10c300 100644
--- a/pvmfw/src/exceptions.rs
+++ b/pvmfw/src/exceptions.rs
@@ -14,11 +14,11 @@
//! Exception handlers.
-use crate::helpers::page_4kb_of;
use crate::memory::{MemoryTrackerError, MEMORY};
use core::fmt;
use vmbase::console;
use vmbase::logger;
+use vmbase::memory::page_4kb_of;
use vmbase::read_sysreg;
use vmbase::{eprintln, power::reboot};
diff --git a/pvmfw/src/fdt.rs b/pvmfw/src/fdt.rs
index c0241ca..ea002df 100644
--- a/pvmfw/src/fdt.rs
+++ b/pvmfw/src/fdt.rs
@@ -16,11 +16,7 @@
use crate::bootargs::BootArgsIterator;
use crate::cstr;
-use crate::helpers::flatten;
-use crate::helpers::RangeExt as _;
use crate::helpers::GUEST_PAGE_SIZE;
-use crate::helpers::SIZE_4KB;
-use crate::memory::BASE_ADDR;
use crate::memory::MAX_ADDR;
use crate::Box;
use crate::RebootReason;
@@ -43,6 +39,10 @@
use log::info;
use log::warn;
use tinyvec::ArrayVec;
+use vmbase::layout::crosvm::MEM_START;
+use vmbase::memory::SIZE_4KB;
+use vmbase::util::flatten;
+use vmbase::util::RangeExt as _;
/// Extract from /config the address range containing the pre-loaded kernel. Absence of /config is
/// not an error.
@@ -115,8 +115,8 @@
/// Check if memory range is ok
fn validate_memory_range(range: &Range<usize>) -> Result<(), RebootReason> {
let base = range.start;
- if base != BASE_ADDR {
- error!("Memory base address {:#x} is not {:#x}", base, BASE_ADDR);
+ if base != MEM_START {
+ error!("Memory base address {:#x} is not {:#x}", base, MEM_START);
return Err(RebootReason::InvalidFdt);
}
@@ -137,7 +137,7 @@
let size = memory_range.len() as u64;
fdt.node_mut(cstr!("/memory"))?
.ok_or(FdtError::NotFound)?
- .setprop_inplace(cstr!("reg"), flatten(&[BASE_ADDR.to_be_bytes(), size.to_be_bytes()]))
+ .setprop_inplace(cstr!("reg"), flatten(&[MEM_START.to_be_bytes(), size.to_be_bytes()]))
}
/// Read the number of CPUs from DT
diff --git a/pvmfw/src/gpt.rs b/pvmfw/src/gpt.rs
index c3ccb5a..e0ffdc3 100644
--- a/pvmfw/src/gpt.rs
+++ b/pvmfw/src/gpt.rs
@@ -14,7 +14,6 @@
//! Support for parsing GUID partition tables.
-use crate::helpers::ceiling_div;
use crate::virtio::pci::VirtIOBlk;
use core::cmp::min;
use core::fmt;
@@ -25,6 +24,7 @@
use static_assertions::const_assert_eq;
use uuid::Uuid;
use virtio_drivers::device::blk::SECTOR_SIZE;
+use vmbase::util::ceiling_div;
use zerocopy::FromBytes;
pub enum Error {
diff --git a/pvmfw/src/helpers.rs b/pvmfw/src/helpers.rs
index dcfd99c..5ad721e 100644
--- a/pvmfw/src/helpers.rs
+++ b/pvmfw/src/helpers.rs
@@ -14,136 +14,10 @@
//! Miscellaneous helper functions.
-use core::arch::asm;
-use core::ops::Range;
-use vmbase::read_sysreg;
-use zeroize::Zeroize;
-
-pub const SIZE_4KB: usize = 4 << 10;
-pub const SIZE_2MB: usize = 2 << 20;
-pub const SIZE_4MB: usize = 4 << 20;
+use vmbase::memory::{PAGE_SIZE, SIZE_4KB};
pub const GUEST_PAGE_SIZE: usize = SIZE_4KB;
-pub const PVMFW_PAGE_SIZE: usize = SIZE_4KB;
-
-/// Computes the largest multiple of the provided alignment smaller or equal to the address.
-///
-/// Note: the result is undefined if alignment isn't a power of two.
-pub const fn unchecked_align_down(addr: usize, alignment: usize) -> usize {
- addr & !(alignment - 1)
-}
-
-/// Computes the smallest multiple of the provided alignment larger or equal to the address.
-///
-/// Note: the result is undefined if alignment isn't a power of two and may wrap to 0.
-pub const fn unchecked_align_up(addr: usize, alignment: usize) -> usize {
- unchecked_align_down(addr + alignment - 1, alignment)
-}
-
-/// Safe wrapper around unchecked_align_up() that validates its assumptions and doesn't wrap.
-pub const fn align_up(addr: usize, alignment: usize) -> Option<usize> {
- if !alignment.is_power_of_two() {
- None
- } else if let Some(s) = addr.checked_add(alignment - 1) {
- Some(unchecked_align_down(s, alignment))
- } else {
- None
- }
-}
-
-/// Performs an integer division rounding up.
-///
-/// Note: Returns None if den isn't a power of two.
-pub const fn ceiling_div(num: usize, den: usize) -> Option<usize> {
- let Some(r) = align_up(num, den) else {
- return None;
- };
-
- r.checked_div(den)
-}
-
-/// Aligns the given address to the given alignment, if it is a power of two.
-///
-/// Returns `None` if the alignment isn't a power of two.
-#[allow(dead_code)] // Currently unused but might be needed again.
-pub const fn align_down(addr: usize, alignment: usize) -> Option<usize> {
- if !alignment.is_power_of_two() {
- None
- } else {
- Some(unchecked_align_down(addr, alignment))
- }
-}
-
-/// Computes the address of the 4KiB page containing a given address.
-pub const fn page_4kb_of(addr: usize) -> usize {
- unchecked_align_down(addr, SIZE_4KB)
-}
-
-#[inline]
-/// Read the number of words in the smallest cache line of all the data caches and unified caches.
-pub fn min_dcache_line_size() -> usize {
- const DMINLINE_SHIFT: usize = 16;
- const DMINLINE_MASK: usize = 0xf;
- let ctr_el0 = read_sysreg!("ctr_el0");
-
- // DminLine: log2 of the number of words in the smallest cache line of all the data caches.
- let dminline = (ctr_el0 >> DMINLINE_SHIFT) & DMINLINE_MASK;
-
- 1 << dminline
-}
-
-/// Flush `size` bytes of data cache by virtual address.
-#[inline]
-pub fn flush_region(start: usize, size: usize) {
- let line_size = min_dcache_line_size();
- let end = start + size;
- let start = unchecked_align_down(start, line_size);
-
- for line in (start..end).step_by(line_size) {
- // SAFETY - Clearing cache lines shouldn't have Rust-visible side effects.
- unsafe {
- asm!(
- "dc cvau, {x}",
- x = in(reg) line,
- options(nomem, nostack, preserves_flags),
- )
- }
- }
-}
-
-#[inline]
-/// Flushes the slice to the point of unification.
-pub fn flush(reg: &[u8]) {
- flush_region(reg.as_ptr() as usize, reg.len())
-}
-
-#[inline]
-/// Overwrites the slice with zeroes, to the point of unification.
-pub fn flushed_zeroize(reg: &mut [u8]) {
- reg.zeroize();
- flush(reg)
-}
-
-/// Flatten [[T; N]] into &[T]
-/// TODO: use slice::flatten when it graduates from experimental
-pub fn flatten<T, const N: usize>(original: &[[T; N]]) -> &[T] {
- // SAFETY: no overflow because original (whose size is len()*N) is already in memory
- let len = original.len() * N;
- // SAFETY: [T] has the same layout as [T;N]
- unsafe { core::slice::from_raw_parts(original.as_ptr().cast(), len) }
-}
-
-/// Trait to check containment of one range within another.
-pub(crate) trait RangeExt {
- /// Returns true if `self` is contained within the `other` range.
- fn is_within(&self, other: &Self) -> bool;
-}
-
-impl<T: PartialOrd> RangeExt for Range<T> {
- fn is_within(&self, other: &Self) -> bool {
- self.start >= other.start && self.end <= other.end
- }
-}
+pub const PVMFW_PAGE_SIZE: usize = PAGE_SIZE;
/// Create &CStr out of &str literal
#[macro_export]
diff --git a/pvmfw/src/instance.rs b/pvmfw/src/instance.rs
index 95ddefd..9a6a6e4 100644
--- a/pvmfw/src/instance.rs
+++ b/pvmfw/src/instance.rs
@@ -21,7 +21,6 @@
use crate::gpt;
use crate::gpt::Partition;
use crate::gpt::Partitions;
-use crate::helpers::ceiling_div;
use crate::rand;
use crate::virtio::pci::VirtIOBlkIterator;
use core::fmt;
@@ -32,6 +31,7 @@
use log::trace;
use uuid::Uuid;
use virtio_drivers::transport::pci::bus::PciRoot;
+use vmbase::util::ceiling_div;
use zerocopy::AsBytes;
use zerocopy::FromBytes;
diff --git a/pvmfw/src/main.rs b/pvmfw/src/main.rs
index 5108eb4..8d48098 100644
--- a/pvmfw/src/main.rs
+++ b/pvmfw/src/main.rs
@@ -40,7 +40,6 @@
use crate::dice::PartialInputs;
use crate::entry::RebootReason;
use crate::fdt::modify_for_next_stage;
-use crate::helpers::flush;
use crate::helpers::GUEST_PAGE_SIZE;
use crate::instance::get_or_generate_instance_salt;
use crate::memory::MEMORY;
@@ -55,6 +54,7 @@
use pvmfw_avb::Capability;
use pvmfw_avb::DebugLevel;
use pvmfw_embedded_key::PUBLIC_KEY;
+use vmbase::memory::flush;
const NEXT_BCC_SIZE: usize = GUEST_PAGE_SIZE;
diff --git a/pvmfw/src/memory.rs b/pvmfw/src/memory.rs
index 76950a2..989120d 100644
--- a/pvmfw/src/memory.rs
+++ b/pvmfw/src/memory.rs
@@ -16,16 +16,13 @@
#![deny(unsafe_op_in_unsafe_fn)]
-use crate::helpers::{self, page_4kb_of, RangeExt, PVMFW_PAGE_SIZE, SIZE_4MB};
-use aarch64_paging::idmap::IdMap;
+use crate::helpers::PVMFW_PAGE_SIZE;
use aarch64_paging::paging::{Attributes, Descriptor, MemoryRegion as VaRange};
use aarch64_paging::MapError;
use alloc::alloc::handle_alloc_error;
use alloc::boxed::Box;
use buddy_system_allocator::LockedFrameAllocator;
use core::alloc::Layout;
-use core::cmp::max;
-use core::cmp::min;
use core::fmt;
use core::iter::once;
use core::num::NonZeroUsize;
@@ -40,18 +37,17 @@
use tinyvec::ArrayVec;
use vmbase::{
dsb, isb, layout,
- memory::{set_dbm_enabled, MemorySharer, PageTable, MMIO_LAZY_MAP_FLAG},
+ memory::{
+ flush_dirty_range, is_leaf_pte, page_4kb_of, set_dbm_enabled, MemorySharer, PageTable,
+ MMIO_LAZY_MAP_FLAG, PT_ASID, SIZE_2MB, SIZE_4KB,
+ },
tlbi,
+ util::{align_up, RangeExt as _},
};
-/// Base of the system's contiguous "main" memory.
-pub const BASE_ADDR: usize = 0x8000_0000;
/// First address that can't be translated by a level 1 TTBR0_EL1.
pub const MAX_ADDR: usize = 1 << 40;
-const PT_ROOT_LEVEL: usize = 1;
-const PT_ASID: usize = 1;
-
pub type MemoryRange = Range<usize>;
pub static MEMORY: SpinMutex<Option<MemoryTracker>> = SpinMutex::new(None);
@@ -70,35 +66,14 @@
mem_type: MemoryType,
}
-impl MemoryRegion {
- /// True if the instance overlaps with the passed range.
- pub fn overlaps(&self, range: &MemoryRange) -> bool {
- overlaps(&self.range, range)
- }
-
- /// True if the instance is fully contained within the passed range.
- pub fn is_within(&self, range: &MemoryRange) -> bool {
- self.as_ref().is_within(range)
- }
-}
-
-impl AsRef<MemoryRange> for MemoryRegion {
- fn as_ref(&self) -> &MemoryRange {
- &self.range
- }
-}
-
-/// Returns true if one range overlaps with the other at all.
-fn overlaps<T: Copy + Ord>(a: &Range<T>, b: &Range<T>) -> bool {
- max(a.start, b.start) < min(a.end, b.end)
-}
-
/// Tracks non-overlapping slices of main memory.
pub struct MemoryTracker {
total: MemoryRange,
page_table: PageTable,
regions: ArrayVec<[MemoryRegion; MemoryTracker::CAPACITY]>,
mmio_regions: ArrayVec<[MemoryRange; MemoryTracker::MMIO_CAPACITY]>,
+ mmio_range: MemoryRange,
+ payload_range: MemoryRange,
}
/// Errors for MemoryTracker operations.
@@ -169,10 +144,19 @@
impl MemoryTracker {
const CAPACITY: usize = 5;
const MMIO_CAPACITY: usize = 5;
- const PVMFW_RANGE: MemoryRange = (BASE_ADDR - SIZE_4MB)..BASE_ADDR;
/// Create a new instance from an active page table, covering the maximum RAM size.
- pub fn new(mut page_table: PageTable) -> Self {
+ pub fn new(
+ mut page_table: PageTable,
+ total: MemoryRange,
+ mmio_range: MemoryRange,
+ payload_range: MemoryRange,
+ ) -> Self {
+ assert!(
+ !total.overlaps(&mmio_range),
+ "MMIO space should not overlap with the main memory region."
+ );
+
// Activate dirty state management first, otherwise we may get permission faults immediately
// after activating the new page table. This has no effect before the new page table is
// activated because none of the entries in the initial idmap have the DBM flag.
@@ -185,10 +169,12 @@
debug!("... Success!");
Self {
- total: BASE_ADDR..MAX_ADDR,
+ total,
page_table,
regions: ArrayVec::new(),
mmio_regions: ArrayVec::new(),
+ mmio_range,
+ payload_range,
}
}
@@ -202,7 +188,7 @@
if self.total.end < range.end {
return Err(MemoryTrackerError::SizeTooLarge);
}
- if !self.regions.iter().all(|r| r.is_within(range)) {
+ if !self.regions.iter().all(|r| r.range.is_within(range)) {
return Err(MemoryTrackerError::SizeTooSmall);
}
@@ -245,11 +231,10 @@
/// Checks that the given range of addresses is within the MMIO region, and then maps it
/// appropriately.
pub fn map_mmio_range(&mut self, range: MemoryRange) -> Result<()> {
- // MMIO space is below the main memory region.
- if range.end > self.total.start || overlaps(&Self::PVMFW_RANGE, &range) {
+ if !range.is_within(&self.mmio_range) {
return Err(MemoryTrackerError::OutOfRange);
}
- if self.mmio_regions.iter().any(|r| overlaps(r, &range)) {
+ if self.mmio_regions.iter().any(|r| range.overlaps(r)) {
return Err(MemoryTrackerError::Overlaps);
}
if self.mmio_regions.len() == self.mmio_regions.capacity() {
@@ -272,10 +257,10 @@
/// with any other previously allocated regions, and that the regions ArrayVec has capacity to
/// add it.
fn check(&self, region: &MemoryRegion) -> Result<()> {
- if !region.is_within(&self.total) {
+ if !region.range.is_within(&self.total) {
return Err(MemoryTrackerError::OutOfRange);
}
- if self.regions.iter().any(|r| r.overlaps(®ion.range)) {
+ if self.regions.iter().any(|r| region.range.overlaps(&r.range)) {
return Err(MemoryTrackerError::Overlaps);
}
if self.regions.len() == self.regions.capacity() {
@@ -289,7 +274,7 @@
return Err(MemoryTrackerError::Full);
}
- Ok(self.regions.last().unwrap().as_ref().clone())
+ Ok(self.regions.last().unwrap().range.clone())
}
/// Unmaps all tracked MMIO regions from the MMIO guard.
@@ -364,12 +349,11 @@
// Collect memory ranges for which dirty state is tracked.
let writable_regions =
self.regions.iter().filter(|r| r.mem_type == MemoryType::ReadWrite).map(|r| &r.range);
- let payload_range = appended_payload_range();
// Execute a barrier instruction to ensure all hardware updates to the page table have been
// observed before reading PTE flags to determine dirty state.
dsb!("ish");
// Now flush writable-dirty pages in those regions.
- for range in writable_regions.chain(once(&payload_range)) {
+ for range in writable_regions.chain(once(&self.payload_range)) {
self.page_table
.modify_range(range, &flush_dirty_range)
.map_err(|_| MemoryTrackerError::FlushRegionFailed)?;
@@ -435,17 +419,6 @@
Ok(())
}
-/// Checks whether a PTE at given level is a page or block descriptor.
-#[inline]
-fn is_leaf_pte(flags: &Attributes, level: usize) -> bool {
- const LEAF_PTE_LEVEL: usize = 3;
- if flags.contains(Attributes::TABLE_OR_PAGE) {
- level == LEAF_PTE_LEVEL
- } else {
- level < LEAF_PTE_LEVEL
- }
-}
-
/// Checks whether block flags indicate it should be MMIO guard mapped.
fn verify_lazy_mapped_block(
_range: &VaRange,
@@ -499,23 +472,6 @@
Ok(())
}
-/// Flushes a memory range the descriptor refers to, if the descriptor is in writable-dirty state.
-fn flush_dirty_range(
- va_range: &VaRange,
- desc: &mut Descriptor,
- level: usize,
-) -> result::Result<(), ()> {
- // Only flush ranges corresponding to dirty leaf PTEs.
- let flags = desc.flags().ok_or(())?;
- if !is_leaf_pte(&flags, level) {
- return Ok(());
- }
- if !flags.contains(Attributes::READ_ONLY) {
- helpers::flush_region(va_range.start().0, va_range.len());
- }
- Ok(())
-}
-
/// Clears read-only flag on a PTE, making it writable-dirty. Used when dirty state is managed
/// in software to handle permission faults on read-only descriptors.
fn mark_dirty_block(
@@ -545,22 +501,22 @@
}
/// Returns memory range reserved for the appended payload.
-pub fn appended_payload_range() -> Range<usize> {
- let start = helpers::align_up(layout::binary_end(), helpers::SIZE_4KB).unwrap();
+pub fn appended_payload_range() -> MemoryRange {
+ let start = align_up(layout::binary_end(), SIZE_4KB).unwrap();
// pvmfw is contained in a 2MiB region so the payload can't be larger than the 2MiB alignment.
- let end = helpers::align_up(start, helpers::SIZE_2MB).unwrap();
+ let end = align_up(start, SIZE_2MB).unwrap();
start..end
}
/// Region allocated for the stack.
-pub fn stack_range() -> Range<usize> {
+pub fn stack_range() -> MemoryRange {
const STACK_PAGES: usize = 8;
layout::stack_range(STACK_PAGES * PVMFW_PAGE_SIZE)
}
pub fn init_page_table() -> result::Result<PageTable, MapError> {
- let mut page_table: PageTable = IdMap::new(PT_ASID, PT_ROOT_LEVEL).into();
+ let mut page_table = PageTable::default();
// Stack and scratch ranges are explicitly zeroed and flushed before jumping to payload,
// so dirty state management can be omitted.
diff --git a/pvmfw/src/virtio/hal.rs b/pvmfw/src/virtio/hal.rs
index ce246b1..c8b279d 100644
--- a/pvmfw/src/virtio/hal.rs
+++ b/pvmfw/src/virtio/hal.rs
@@ -15,7 +15,6 @@
//! HAL for the virtio_drivers crate.
use super::pci::PCI_INFO;
-use crate::helpers::RangeExt as _;
use crate::memory::{alloc_shared, dealloc_shared};
use core::alloc::Layout;
use core::mem::size_of;
@@ -23,6 +22,7 @@
use log::trace;
use virtio_drivers::{BufferDirection, Hal, PhysAddr, PAGE_SIZE};
use vmbase::memory::{phys_to_virt, virt_to_phys};
+use vmbase::util::RangeExt as _;
/// The alignment to use for the temporary buffers allocated by `HalImpl::share`. There doesn't seem
/// to be any particular alignment required by VirtIO for these, so 16 bytes should be enough to
diff --git a/rialto/src/main.rs b/rialto/src/main.rs
index 30bc5b0..bc5ab2c 100644
--- a/rialto/src/main.rs
+++ b/rialto/src/main.rs
@@ -23,25 +23,22 @@
extern crate alloc;
use crate::error::{Error, Result};
-use aarch64_paging::idmap::IdMap;
use buddy_system_allocator::LockedHeap;
use core::slice;
use fdtpci::PciInfo;
use hyp::get_hypervisor;
use log::{debug, error, info};
-use vmbase::{layout, main, memory::PageTable, power::reboot};
+use vmbase::{
+ layout, main,
+ memory::{PageTable, PAGE_SIZE},
+ power::reboot,
+};
const SZ_1K: usize = 1024;
-const SZ_4K: usize = 4 * SZ_1K;
const SZ_64K: usize = 64 * SZ_1K;
const SZ_1M: usize = 1024 * SZ_1K;
const SZ_1G: usize = 1024 * SZ_1M;
-// Root level is given by the value of TCR_EL1.TG0 and TCR_EL1.T0SZ, set in
-// entry.S. For 4KB granule and 39-bit VA, the root level is 1.
-const PT_ROOT_LEVEL: usize = 1;
-const PT_ASID: usize = 1;
-
#[global_allocator]
static HEAP_ALLOCATOR: LockedHeap<32> = LockedHeap::<32>::new();
@@ -55,12 +52,12 @@
}
fn init_page_table() -> Result<()> {
- let mut page_table: PageTable = IdMap::new(PT_ASID, PT_ROOT_LEVEL).into();
+ let mut page_table = PageTable::default();
// The first 1 GiB of address space is used by crosvm for MMIO.
page_table.map_device(&(0..SZ_1G))?;
page_table.map_data(&layout::scratch_range())?;
- page_table.map_data(&layout::stack_range(40 * SZ_4K))?;
+ page_table.map_data(&layout::stack_range(40 * PAGE_SIZE))?;
page_table.map_code(&layout::text_range())?;
page_table.map_rodata(&layout::rodata_range())?;
page_table.map_device(&layout::console_uart_range())?;
diff --git a/tests/benchmark/src/java/com/android/microdroid/benchmark/MicrodroidBenchmarks.java b/tests/benchmark/src/java/com/android/microdroid/benchmark/MicrodroidBenchmarks.java
index c210ea6..aed28a8 100644
--- a/tests/benchmark/src/java/com/android/microdroid/benchmark/MicrodroidBenchmarks.java
+++ b/tests/benchmark/src/java/com/android/microdroid/benchmark/MicrodroidBenchmarks.java
@@ -24,6 +24,7 @@
import static androidx.test.platform.app.InstrumentationRegistry.getInstrumentation;
import static com.google.common.truth.Truth.assertThat;
+import static com.google.common.truth.Truth.assertWithMessage;
import static com.google.common.truth.TruthJUnit.assume;
import android.app.Instrumentation;
@@ -467,6 +468,8 @@
BenchmarkVmListener.create(listener).runToFinish(TAG, vm);
+ assertWithMessage("VM failed to start").that(listener.mCrosvm).isNotNull();
+
double mem_overall = 256.0;
double mem_total = (double) listener.mMemTotal / 1024.0;
double mem_free = (double) listener.mMemFree / 1024.0;
@@ -548,6 +551,8 @@
VirtualMachine vm = forceCreateNewVirtualMachine(vmName, config);
MemoryReclaimListener listener = new MemoryReclaimListener(this::executeCommand);
BenchmarkVmListener.create(listener).runToFinish(TAG, vm);
+ assertWithMessage("VM failed to start").that(listener.mPreCrosvm).isNotNull();
+ assertWithMessage("Post trim stats not available").that(listener.mPostCrosvm).isNotNull();
double mem_pre_crosvm_host_rss = (double) listener.mPreCrosvm.mHostRss / 1024.0;
double mem_pre_crosvm_host_pss = (double) listener.mPreCrosvm.mHostPss / 1024.0;
diff --git a/vmbase/Android.bp b/vmbase/Android.bp
index 72be0b0..bda3796 100644
--- a/vmbase/Android.bp
+++ b/vmbase/Android.bp
@@ -69,6 +69,7 @@
"liblog_rust_nostd",
"libsmccc",
"libspin_nostd",
+ "libzeroize_nostd",
],
whole_static_libs: [
"librust_baremetal",
diff --git a/vmbase/src/layout/crosvm.rs b/vmbase/src/layout/crosvm.rs
new file mode 100644
index 0000000..fbeb477
--- /dev/null
+++ b/vmbase/src/layout/crosvm.rs
@@ -0,0 +1,25 @@
+// Copyright 2023, The Android Open Source Project
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+//! Memory layout for crosvm for aarch64 architecture.
+//!
+//! https://crosvm.dev/book/appendix/memory_layout.html#common-layout
+
+/// The start address of MMIO space.
+pub const MMIO_START: usize = 0x0;
+/// The end address of MMIO space.
+pub const MMIO_END: usize = 0x4000_0000;
+
+/// The start of the system's contiguous "main" memory.
+pub const MEM_START: usize = 0x8000_0000;
diff --git a/vmbase/src/layout.rs b/vmbase/src/layout/mod.rs
similarity index 98%
rename from vmbase/src/layout.rs
rename to vmbase/src/layout/mod.rs
index 43bcb2c..dd3e0e0 100644
--- a/vmbase/src/layout.rs
+++ b/vmbase/src/layout/mod.rs
@@ -14,6 +14,8 @@
//! Memory layout.
+pub mod crosvm;
+
use crate::console::BASE_ADDRESS;
use core::ops::Range;
use core::ptr::addr_of;
diff --git a/vmbase/src/lib.rs b/vmbase/src/lib.rs
index 2541b8a..80cdf4e 100644
--- a/vmbase/src/lib.rs
+++ b/vmbase/src/lib.rs
@@ -28,6 +28,7 @@
pub mod memory;
pub mod power;
pub mod uart;
+pub mod util;
pub use bionic::STACK_CHK_GUARD;
diff --git a/vmbase/src/memory/dbm.rs b/vmbase/src/memory/dbm.rs
index 3a52c2d..235c0e0 100644
--- a/vmbase/src/memory/dbm.rs
+++ b/vmbase/src/memory/dbm.rs
@@ -14,7 +14,10 @@
//! Hardware management of the access flag and dirty state.
+use super::page_table::is_leaf_pte;
+use super::util::flush_region;
use crate::{isb, read_sysreg, write_sysreg};
+use aarch64_paging::paging::{Attributes, Descriptor, MemoryRegion};
/// Sets whether the hardware management of access and dirty state is enabled with
/// the given boolean.
@@ -45,3 +48,23 @@
const DBM_AVAILABLE: usize = 1 << 1;
read_sysreg!("id_aa64mmfr1_el1") & DBM_AVAILABLE != 0
}
+
+/// Flushes a memory range the descriptor refers to, if the descriptor is in writable-dirty state.
+/// As the return type is required by the crate `aarch64_paging`, we cannot address the lint
+/// issue `clippy::result_unit_err`.
+#[allow(clippy::result_unit_err)]
+pub fn flush_dirty_range(
+ va_range: &MemoryRegion,
+ desc: &mut Descriptor,
+ level: usize,
+) -> Result<(), ()> {
+ // Only flush ranges corresponding to dirty leaf PTEs.
+ let flags = desc.flags().ok_or(())?;
+ if !is_leaf_pte(&flags, level) {
+ return Ok(());
+ }
+ if !flags.contains(Attributes::READ_ONLY) {
+ flush_region(va_range.start().0, va_range.len());
+ }
+ Ok(())
+}
diff --git a/vmbase/src/memory/mod.rs b/vmbase/src/memory/mod.rs
index 5e1065a..bb9149c 100644
--- a/vmbase/src/memory/mod.rs
+++ b/vmbase/src/memory/mod.rs
@@ -19,7 +19,10 @@
mod shared;
mod util;
-pub use dbm::set_dbm_enabled;
-pub use page_table::{PageTable, MMIO_LAZY_MAP_FLAG};
+pub use dbm::{flush_dirty_range, set_dbm_enabled};
+pub use page_table::{is_leaf_pte, PageTable, MMIO_LAZY_MAP_FLAG, PT_ASID};
pub use shared::MemorySharer;
-pub use util::{phys_to_virt, virt_to_phys};
+pub use util::{
+ flush, flushed_zeroize, min_dcache_line_size, page_4kb_of, phys_to_virt, virt_to_phys,
+ PAGE_SIZE, SIZE_2MB, SIZE_4KB, SIZE_4MB,
+};
diff --git a/vmbase/src/memory/page_table.rs b/vmbase/src/memory/page_table.rs
index bc71e97..1a9d0f8 100644
--- a/vmbase/src/memory/page_table.rs
+++ b/vmbase/src/memory/page_table.rs
@@ -35,6 +35,12 @@
const RODATA: Attributes = DATA.union(Attributes::READ_ONLY);
const DATA_DBM: Attributes = RODATA.union(Attributes::DBM);
+/// Root level is given by the value of TCR_EL1.TG0 and TCR_EL1.T0SZ, set in
+/// entry.S. For 4KB granule and 39-bit VA, the root level is 1.
+const PT_ROOT_LEVEL: usize = 1;
+/// Page table ASID.
+pub const PT_ASID: usize = 1;
+
type Result<T> = result::Result<T, MapError>;
/// High-level API for managing MMU mappings.
@@ -48,6 +54,12 @@
}
}
+impl Default for PageTable {
+ fn default() -> Self {
+ IdMap::new(PT_ASID, PT_ROOT_LEVEL).into()
+ }
+}
+
impl PageTable {
/// Activates the page table.
///
@@ -108,3 +120,14 @@
self.idmap.modify_range(&MemoryRegion::new(range.start, range.end), f)
}
}
+
+/// Checks whether a PTE at given level is a page or block descriptor.
+#[inline]
+pub fn is_leaf_pte(flags: &Attributes, level: usize) -> bool {
+ const LEAF_PTE_LEVEL: usize = 3;
+ if flags.contains(Attributes::TABLE_OR_PAGE) {
+ level == LEAF_PTE_LEVEL
+ } else {
+ level < LEAF_PTE_LEVEL
+ }
+}
diff --git a/vmbase/src/memory/util.rs b/vmbase/src/memory/util.rs
index 5b89cd1..04d42cd 100644
--- a/vmbase/src/memory/util.rs
+++ b/vmbase/src/memory/util.rs
@@ -14,7 +14,71 @@
//! Utility functions for memory management.
+use crate::read_sysreg;
+use crate::util::unchecked_align_down;
+use core::arch::asm;
use core::ptr::NonNull;
+use zeroize::Zeroize;
+
+/// The size of a 4KB memory in bytes.
+pub const SIZE_4KB: usize = 4 << 10;
+/// The size of a 2MB memory in bytes.
+pub const SIZE_2MB: usize = 2 << 20;
+/// The size of a 4MB memory in bytes.
+pub const SIZE_4MB: usize = 4 << 20;
+
+/// The page size in bytes assumed by vmbase - 4 KiB.
+pub const PAGE_SIZE: usize = SIZE_4KB;
+
+/// Reads the number of words in the smallest cache line of all the data caches and unified caches.
+#[inline]
+pub fn min_dcache_line_size() -> usize {
+ const DMINLINE_SHIFT: usize = 16;
+ const DMINLINE_MASK: usize = 0xf;
+ let ctr_el0 = read_sysreg!("ctr_el0");
+
+ // DminLine: log2 of the number of words in the smallest cache line of all the data caches.
+ let dminline = (ctr_el0 >> DMINLINE_SHIFT) & DMINLINE_MASK;
+
+ 1 << dminline
+}
+
+/// Flush `size` bytes of data cache by virtual address.
+#[inline]
+pub(super) fn flush_region(start: usize, size: usize) {
+ let line_size = min_dcache_line_size();
+ let end = start + size;
+ let start = unchecked_align_down(start, line_size);
+
+ for line in (start..end).step_by(line_size) {
+ // SAFETY - Clearing cache lines shouldn't have Rust-visible side effects.
+ unsafe {
+ asm!(
+ "dc cvau, {x}",
+ x = in(reg) line,
+ options(nomem, nostack, preserves_flags),
+ )
+ }
+ }
+}
+
+/// Flushes the slice to the point of unification.
+#[inline]
+pub fn flush(reg: &[u8]) {
+ flush_region(reg.as_ptr() as usize, reg.len())
+}
+
+/// Overwrites the slice with zeroes, to the point of unification.
+#[inline]
+pub fn flushed_zeroize(reg: &mut [u8]) {
+ reg.zeroize();
+ flush(reg)
+}
+
+/// Computes the address of the 4KiB page containing a given address.
+pub const fn page_4kb_of(addr: usize) -> usize {
+ unchecked_align_down(addr, SIZE_4KB)
+}
/// Returns the intermediate physical address corresponding to the given virtual address.
///
diff --git a/vmbase/src/util.rs b/vmbase/src/util.rs
new file mode 100644
index 0000000..8c230a1
--- /dev/null
+++ b/vmbase/src/util.rs
@@ -0,0 +1,93 @@
+// Copyright 2023, The Android Open Source Project
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+//! Utility functions.
+
+use core::ops::Range;
+
+/// Flatten [[T; N]] into &[T]
+/// TODO: use slice::flatten when it graduates from experimental
+pub fn flatten<T, const N: usize>(original: &[[T; N]]) -> &[T] {
+ // SAFETY: no overflow because original (whose size is len()*N) is already in memory
+ let len = original.len() * N;
+ // SAFETY: [T] has the same layout as [T;N]
+ unsafe { core::slice::from_raw_parts(original.as_ptr().cast(), len) }
+}
+
+/// Computes the largest multiple of the provided alignment smaller or equal to the address.
+///
+/// Note: the result is undefined if alignment isn't a power of two.
+pub const fn unchecked_align_down(addr: usize, alignment: usize) -> usize {
+ addr & !(alignment - 1)
+}
+
+/// Computes the smallest multiple of the provided alignment larger or equal to the address.
+///
+/// Note: the result is undefined if alignment isn't a power of two and may wrap to 0.
+pub const fn unchecked_align_up(addr: usize, alignment: usize) -> usize {
+ unchecked_align_down(addr + alignment - 1, alignment)
+}
+
+/// Safe wrapper around unchecked_align_up() that validates its assumptions and doesn't wrap.
+pub const fn align_up(addr: usize, alignment: usize) -> Option<usize> {
+ if !alignment.is_power_of_two() {
+ None
+ } else if let Some(s) = addr.checked_add(alignment - 1) {
+ Some(unchecked_align_down(s, alignment))
+ } else {
+ None
+ }
+}
+
+/// Aligns the given address to the given alignment, if it is a power of two.
+///
+/// Returns `None` if the alignment isn't a power of two.
+#[allow(dead_code)] // Currently unused but might be needed again.
+const fn align_down(addr: usize, alignment: usize) -> Option<usize> {
+ if !alignment.is_power_of_two() {
+ None
+ } else {
+ Some(unchecked_align_down(addr, alignment))
+ }
+}
+
+/// Performs an integer division rounding up.
+///
+/// Note: Returns None if den isn't a power of two.
+pub const fn ceiling_div(num: usize, den: usize) -> Option<usize> {
+ let Some(r) = align_up(num, den) else {
+ return None;
+ };
+
+ r.checked_div(den)
+}
+
+/// Trait to check containment of one range within another.
+pub trait RangeExt {
+ /// Returns true if `self` is contained within the `other` range.
+ fn is_within(&self, other: &Self) -> bool;
+
+ /// Returns true if `self` overlaps with the `other` range.
+ fn overlaps(&self, other: &Self) -> bool;
+}
+
+impl<T: PartialOrd> RangeExt for Range<T> {
+ fn is_within(&self, other: &Self) -> bool {
+ self.start >= other.start && self.end <= other.end
+ }
+
+ fn overlaps(&self, other: &Self) -> bool {
+ self.start < other.end && other.start < self.end
+ }
+}