devices: vvu: virtio-iommu support in virtio-vhost-user

Make vvu use devices use VFIO to manage their virt queues. This
alleviates the need to use noiommu mode. However, it is still necessary
to use `vfio_iommu_type1.allow_unsafe_interrupts=1`.

BUG=b:202151642,b:215310597
TEST=launch sibling with vvu + virtio-iommu

Cq-Depend: chromium:3565728, chromium:3565260
Change-Id: If418524598c40a37d41c0ffaa1dcc0f8ee11fcb3
Reviewed-on: https://chromium-review.googlesource.com/c/chromiumos/platform/crosvm/+/3501052
Reviewed-by: Keiichi Watanabe <keiichiw@chromium.org>
Tested-by: kokoro <noreply+kokoro@google.com>
Commit-Queue: David Stevens <stevensd@chromium.org>
This commit is contained in:
Woody Chow 2022-03-03 10:56:26 +09:00 committed by Chromeos LUCI
parent 33af75be29
commit 91296bdee0
10 changed files with 331 additions and 208 deletions

View file

@ -16,18 +16,20 @@ use std::sync::Arc;
use std::u32;
use crate::IommuDevType;
use base::error;
use base::{
ioctl, ioctl_with_mut_ref, ioctl_with_ptr, ioctl_with_ref, ioctl_with_val, warn,
AsRawDescriptor, Error, Event, FromRawDescriptor, RawDescriptor, SafeDescriptor,
ioctl, ioctl_with_mut_ptr, ioctl_with_mut_ref, ioctl_with_ptr, ioctl_with_ref, ioctl_with_val,
warn, AsRawDescriptor, Error, Event, FromRawDescriptor, RawDescriptor, SafeDescriptor,
};
use data_model::{vec_with_array_field, DataInit};
use hypervisor::{DeviceKind, Vm};
use once_cell::sync::OnceCell;
use remain::sorted;
use resources::address_allocator::AddressAllocator;
use resources::{Alloc, Error as ResourcesError};
use sync::Mutex;
use thiserror::Error;
use vfio_sys::*;
use vm_memory::GuestMemory;
#[sorted]
#[derive(Error, Debug)]
@ -56,14 +58,20 @@ pub enum VfioError {
IommuDmaMap(Error),
#[error("failed to remove guest memory map from iommu table: {0}")]
IommuDmaUnmap(Error),
#[error("failed to get IOMMU cap info from host")]
IommuGetCapInfo,
#[error("failed to get IOMMU info from host: {0}")]
IommuGetInfo(Error),
#[error("failed to set KVM vfio device's attribute: {0}")]
KvmSetDeviceAttr(Error),
#[error("AddressAllocator is unavailable")]
NoRescAlloc,
#[error("failed to open /dev/vfio/vfio container: {0}")]
OpenContainer(io::Error),
#[error("failed to open /dev/vfio/$group_num group: {0}")]
OpenGroup(io::Error),
#[error("resources error: {0}")]
Resources(ResourcesError),
#[error(
"vfio API version doesn't match with VFIO_API_VERSION defined in vfio_sys/src/vfio.rs"
)]
@ -100,19 +108,24 @@ enum KvmVfioGroupOps {
#[repr(u32)]
enum IommuType {
Type1V2 = VFIO_TYPE1v2_IOMMU,
NoIommu = VFIO_NOIOMMU_IOMMU,
}
/// VfioContainer contain multi VfioGroup, and delegate an IOMMU domain table
pub struct VfioContainer {
container: File,
groups: HashMap<u32, Arc<Mutex<VfioGroup>>>,
host_iommu: bool,
}
fn extract_vfio_struct<T>(bytes: &[u8], offset: usize) -> T
where
T: DataInit,
{
T::from_reader(&bytes[offset..(offset + mem::size_of::<T>())]).expect("malformed kernel data")
}
const VFIO_API_VERSION: u8 = 0;
impl VfioContainer {
fn new_inner(host_iommu: bool) -> Result<Self> {
pub fn new() -> Result<Self> {
let container = OpenOptions::new()
.read(true)
.write(true)
@ -128,20 +141,9 @@ impl VfioContainer {
Ok(VfioContainer {
container,
groups: HashMap::new(),
host_iommu,
})
}
/// Open VfioContainer with IOMMU enabled.
pub fn new() -> Result<Self> {
Self::new_inner(true /* host_iommu */)
}
/// Open VfioContainer with IOMMU disabled.
pub fn new_noiommu() -> Result<Self> {
Self::new_inner(false /* host_iommu */)
}
// Construct a VfioContainer from an exist container file.
pub fn new_from_container(container: File) -> Result<Self> {
// Safe as file is vfio container descriptor and ioctl is defined by kernel.
@ -153,7 +155,6 @@ impl VfioContainer {
Ok(VfioContainer {
container,
groups: HashMap::new(),
host_iommu: true,
})
}
@ -236,7 +237,76 @@ impl VfioContainer {
Ok(iommu_info.iova_pgsizes)
}
fn init(&mut self, guest_mem: &GuestMemory, iommu_enabled: bool) -> Result<()> {
pub fn vfio_iommu_iova_get_iova_ranges(&self) -> Result<Vec<vfio_iova_range>> {
// Query the buffer size needed fetch the capabilities.
let mut iommu_info_argsz = vfio_iommu_type1_info {
argsz: mem::size_of::<vfio_iommu_type1_info>() as u32,
flags: 0,
iova_pgsizes: 0,
..Default::default()
};
// Safe as file is vfio container, iommu_info_argsz has valid values,
// and we check the return value
let ret = unsafe { ioctl_with_mut_ref(self, VFIO_IOMMU_GET_INFO(), &mut iommu_info_argsz) };
if ret != 0 {
return Err(VfioError::IommuGetInfo(get_error()));
}
if (iommu_info_argsz.flags & VFIO_IOMMU_INFO_CAPS) == 0 {
return Err(VfioError::IommuGetCapInfo);
}
let mut iommu_info = vec_with_array_field::<vfio_iommu_type1_info, u8>(
iommu_info_argsz.argsz as usize - mem::size_of::<vfio_iommu_type1_info>(),
);
iommu_info[0].argsz = iommu_info_argsz.argsz;
// Safe as file is vfio container, iommu_info has valid values,
// and we check the return value
let ret =
unsafe { ioctl_with_mut_ptr(self, VFIO_IOMMU_GET_INFO(), iommu_info.as_mut_ptr()) };
if ret != 0 {
return Err(VfioError::IommuGetInfo(get_error()));
}
// Safe because we initialized iommu_info with enough space, u8 has less strict
// alignment, and since it will no longer be mutated.
let info_bytes = unsafe {
std::slice::from_raw_parts(
iommu_info.as_ptr() as *const u8,
iommu_info_argsz.argsz as usize,
)
};
if (iommu_info[0].flags & VFIO_IOMMU_INFO_CAPS) == 0 {
return Err(VfioError::IommuGetCapInfo);
}
let mut offset = iommu_info[0].cap_offset as usize;
while offset != 0 {
let header = extract_vfio_struct::<vfio_info_cap_header>(info_bytes, offset);
if header.id == VFIO_IOMMU_TYPE1_INFO_CAP_IOVA_RANGE as u16 && header.version == 1 {
let iova_header = extract_vfio_struct::<vfio_iommu_type1_info_cap_iova_range_header>(
info_bytes, offset,
);
let range_offset = offset + mem::size_of::<vfio_iommu_type1_info_cap_iova_range>();
let mut ret = Vec::new();
for i in 0..iova_header.nr_iovas {
ret.push(extract_vfio_struct::<vfio_iova_range>(
info_bytes,
range_offset + i as usize * mem::size_of::<vfio_iova_range>(),
));
}
return Ok(ret);
}
offset = header.next as usize;
}
Err(VfioError::IommuGetCapInfo)
}
fn init_vfio_iommu(&mut self) -> Result<()> {
if !self.check_extension(IommuType::Type1V2) {
return Err(VfioError::VfioType1V2);
}
@ -245,15 +315,6 @@ impl VfioContainer {
return Err(VfioError::ContainerSetIOMMU(get_error()));
}
// Add all guest memory regions into vfio container's iommu table,
// then vfio kernel driver could access guest memory from gfn
if !iommu_enabled {
guest_mem.with_regions(|_index, guest_addr, size, host_addr, _mmap, _fd_offset| {
// Safe because the guest regions are guaranteed not to overlap
unsafe { self.vfio_dma_map(guest_addr.0, size as u64, host_addr as u64, true) }
})?;
}
Ok(())
}
@ -266,11 +327,27 @@ impl VfioContainer {
match self.groups.get(&id) {
Some(group) => Ok(group.clone()),
None => {
let group = Arc::new(Mutex::new(VfioGroup::new(self, self.host_iommu, id)?));
let group = Arc::new(Mutex::new(VfioGroup::new(self, id)?));
if self.groups.is_empty() {
// Before the first group is added into container, do once cotainer
// initialize for a vm
self.init(vm.get_memory(), iommu_enabled)?;
// Before the first group is added into container, do once per
// container initialization.
self.init_vfio_iommu()?;
if !iommu_enabled {
vm.get_memory().with_regions(
|_index, guest_addr, size, host_addr, _mmap, _fd_offset| {
// Safe because the guest regions are guaranteed not to overlap
unsafe {
self.vfio_dma_map(
guest_addr.0,
size as u64,
host_addr as u64,
true,
)
}
},
)?;
}
}
let kvm_vfio_file = KVM_VFIO_FILE
@ -291,12 +368,12 @@ impl VfioContainer {
match self.groups.get(&id) {
Some(group) => Ok(group.clone()),
None => {
let group = Arc::new(Mutex::new(VfioGroup::new(self, self.host_iommu, id)?));
let group = Arc::new(Mutex::new(VfioGroup::new(self, id)?));
if self.groups.is_empty() && !self.host_iommu {
if self.set_iommu(IommuType::NoIommu) < 0 {
return Err(VfioError::ContainerSetIOMMU(get_error()));
}
if self.groups.is_empty() {
// Before the first group is added into container, do once per
// container initialization.
self.init_vfio_iommu()?;
}
self.groups.insert(id, group.clone());
@ -352,12 +429,8 @@ struct VfioGroup {
}
impl VfioGroup {
fn new(container: &VfioContainer, host_iommu: bool, id: u32) -> Result<Self> {
let group_path = if host_iommu {
format!("/dev/vfio/{}", id)
} else {
format!("/dev/vfio/noiommu-{}", id)
};
fn new(container: &VfioContainer, id: u32) -> Result<Self> {
let group_path = format!("/dev/vfio/{}", id);
let group_file = OpenOptions::new()
.read(true)
.write(true)
@ -628,6 +701,8 @@ pub struct VfioDevice {
group_id: u32,
// vec for vfio device's regions
regions: Vec<VfioRegion>,
iova_alloc: Option<Arc<Mutex<AddressAllocator>>>,
}
impl VfioDevice {
@ -663,6 +738,7 @@ impl VfioDevice {
group_descriptor,
group_id,
regions,
iova_alloc: None,
})
}
@ -696,6 +772,14 @@ impl VfioDevice {
group.lock().add_device_num();
let group_descriptor = group.lock().as_raw_descriptor();
let iova_ranges = container
.lock()
.vfio_iommu_iova_get_iova_ranges()?
.into_iter()
.map(|r| std::ops::RangeInclusive::new(r.start, r.end));
let iova_alloc = AddressAllocator::new_from_list(iova_ranges, None, None)
.map_err(VfioError::Resources)?;
Ok(VfioDevice {
dev,
name,
@ -703,6 +787,7 @@ impl VfioDevice {
group_descriptor,
group_id,
regions,
iova_alloc: Some(Arc::new(Mutex::new(iova_alloc))),
})
}
@ -1262,6 +1347,20 @@ impl VfioDevice {
self.container.lock().vfio_dma_unmap(iova, size)
}
pub fn vfio_get_iommu_page_size_mask(&self) -> Result<u64> {
self.container.lock().vfio_get_iommu_page_size_mask()
}
pub fn alloc_iova(&self, size: u64, align_size: u64, alloc: Alloc) -> Result<u64> {
match &self.iova_alloc {
None => Err(VfioError::NoRescAlloc),
Some(iova_alloc) => iova_alloc
.lock()
.allocate_with_align(size, alloc, "alloc_iova".to_owned(), align_size)
.map_err(VfioError::Resources),
}
}
/// Gets the vfio device backing `File`.
pub fn device_file(&self) -> &File {
&self.dev

View file

@ -102,7 +102,7 @@ const TYPE_TPM: u32 = MAX_VIRTIO_DEVICE_ID - 1;
const TYPE_VHOST_USER: u32 = MAX_VIRTIO_DEVICE_ID - 2;
pub const VIRTIO_F_VERSION_1: u32 = 32;
const VIRTIO_F_ACCESS_PLATFORM: u32 = 33;
pub const VIRTIO_F_ACCESS_PLATFORM: u32 = 33;
const INTERRUPT_STATUS_USED_RING: u32 = 0x1;
const INTERRUPT_STATUS_CONFIG_CHANGED: u32 = 0x2;

View file

@ -41,8 +41,7 @@ pub fn open_vfio_device(pci_address: PciAddress) -> Result<VfioDevice> {
.context("failed to clear driver_override")?;
let vfio_path = format!("/sys/bus/pci/devices/{}", &addr_str);
// TODO(b/202151642): Use `VfioContainer::new()` once virtio-iommu for VFIO is implemented.
let vfio_container = Arc::new(Mutex::new(VfioContainer::new_noiommu()?));
let vfio_container = Arc::new(Mutex::new(VfioContainer::new()?));
let vfio = VfioDevice::new(&vfio_path, vfio_container)
.map_err(|e| anyhow!("failed to create VFIO device: {}", e))?;
Ok(vfio)

View file

@ -11,6 +11,7 @@ use anyhow::{anyhow, bail, Context, Result};
use base::{info, Event};
use data_model::DataInit;
use memoffset::offset_of;
use resources::Alloc;
use vfio_sys::*;
use virtio_sys::vhost::VIRTIO_F_VERSION_1;
@ -18,7 +19,7 @@ use crate::pci::{MsixCap, PciAddress, PciCapabilityID, CAPABILITY_LIST_HEAD_OFFS
use crate::vfio::{VfioDevice, VfioPciConfig, VfioRegionAddr};
use crate::virtio::vhost::user::device::vvu::{
bus::open_vfio_device,
queue::{DescTableAddrs, UserQueue},
queue::{DescTableAddrs, IovaAllocator, UserQueue},
};
use crate::virtio::{PciCapabilityType, VirtioPciCap};
@ -252,6 +253,13 @@ impl VvuPciDevice {
.check_device_info()
.context("failed to check VFIO device information")?;
let page_mask = vfio_dev
.vfio_get_iommu_page_size_mask()
.context("failed to get iommu page size mask")?;
if page_mask & (base::pagesize() as u64) == 0 {
bail!("Unsupported iommu page mask {:x}", page_mask);
}
let mut pci_dev = Self {
vfio_dev,
caps,
@ -309,7 +317,7 @@ impl VvuPciDevice {
QueueType::Rx => true,
QueueType::Tx => false,
};
let queue = UserQueue::new(queue_size, device_writable)?;
let queue = UserQueue::new(queue_size, device_writable, typ as u8, self)?;
let DescTableAddrs { desc, avail, used } = queue.desc_table_addrs()?;
let desc_lo = (desc & 0xffffffff) as u32;
@ -491,3 +499,17 @@ impl VvuPciDevice {
Ok(())
}
}
impl IovaAllocator for VvuPciDevice {
fn alloc_iova(&self, size: u64, tag: u8) -> Result<u64> {
self.vfio_dev
.alloc_iova(size, base::pagesize() as u64, Alloc::VvuQueue(tag))
.context("failed to find an iova region to map the gpa region to")
}
unsafe fn map_iova(&self, iova: u64, size: u64, addr: *const u8) -> Result<()> {
self.vfio_dev
.vfio_dma_map(iova, size, addr as u64, true)
.context("failed to map iova")
}
}

View file

@ -8,13 +8,11 @@
use std::mem;
use std::num::Wrapping;
use std::sync::atomic::{fence, Ordering};
#[cfg(not(test))]
use std::{collections::BTreeMap, fs::File};
use anyhow::{anyhow, bail, Context, Result};
use data_model::{DataInit, Le16, Le32, Le64, VolatileSlice};
use virtio_sys::vhost::VRING_DESC_F_WRITE;
use vm_memory::{GuestAddress, GuestMemory};
use vm_memory::{GuestAddress as IOVA, GuestMemory as QueueMemory};
use crate::virtio::Desc;
@ -36,19 +34,17 @@ pub struct DescTableAddrs {
}
struct MemLayout {
/// Address of the descriptor table.
/// Since the vvu driver runs in the guest user space, `GuestAddress` here stores the guest
/// virtual address.
desc_table: GuestAddress,
/// Address of the descriptor table in UserQueue.mem.
desc_table: IOVA,
/// Virtual address of the available ring
avail_ring: GuestAddress,
/// Address of the available ring in UserQueue.mem.
avail_ring: IOVA,
/// Virtual address of the used ring
used_ring: GuestAddress,
/// Address of the used ring in UserQueue.mem.
used_ring: IOVA,
/// Virtual address of the start of buffers.
buffer_addr: GuestAddress,
/// Address of the start of buffers in UserQueue.mem.
buffer_addr: IOVA,
}
/// Represents a virtqueue that is allocated in the guest userspace and manipulated from a VFIO
@ -59,8 +55,12 @@ struct MemLayout {
///
/// # Memory Layout
///
/// `mem` is a continuous memory allocated in the guest userspace and used to have a virtqueue.
/// Its layout is defined in the following table and stored in `mem_layout`.
/// `mem` is the memory allocated in the guest userspace for the virtqueue, which is mapped into
/// the vvu device via VFIO. The GuestAddresses of `mem` are the IOVAs that should be used when
/// communicating with the vvu device. All accesses to the shared memory from the device backend
/// must be done through the GuestMemory read/write functions.
///
/// The layout `mem` is defined in the following table and stored in `mem_layout`.
///
/// | | Alignment | Size |
/// |-----------------------------------------------------------------|
@ -81,7 +81,7 @@ pub struct UserQueue {
size: Wrapping<u16>,
/// The underlying memory.
mem: GuestMemory,
mem: QueueMemory,
/// Virtqueue layout on `mem`.
mem_layout: MemLayout,
@ -100,21 +100,30 @@ pub struct UserQueue {
/// one virtqueue. Also, it's better to use `crate::virtio::DescriptorChain` for descirptors as
/// a part of b/215153367.
device_writable: bool,
}
/// Mapping from a virtual address to the physical address.
/// This mapping is initialized by reading `/proc/self/pagemap`.
/// TODO(b/215310597): This workaround won't work if memory mapping is changed. Currently, we
/// are assuming that memory mapping is fixed during the vvu negotiation.
/// Once virtio-iommu supports VFIO usage, we can remove this workaround and we should use
/// VFIO_IOMMU_MAP_DMA call to get physical addresses.
#[cfg(not(test))]
addr_table: BTreeMap<GuestAddress, u64>,
/// Interface used by UserQueue to interact with the IOMMU.
pub trait IovaAllocator {
/// Allocates an IO virtual address region of the requested size.
fn alloc_iova(&self, size: u64, tag: u8) -> Result<u64>;
/// Maps the given address at the given IOVA.
///
/// # Safety
///
/// `addr` must reference a region of at least length `size`. Memory passed
/// to this function may be mutated at any time, so `addr` must not be memory
/// that is directly managed by rust.
unsafe fn map_iova(&self, iova: u64, size: u64, addr: *const u8) -> Result<()>;
}
impl UserQueue {
/// Creats a `UserQueue` instance.
pub fn new(queue_size: u16, device_writable: bool) -> Result<Self> {
let (mem, size, mem_layout) = Self::init_memory(queue_size)?;
pub fn new<I>(queue_size: u16, device_writable: bool, tag: u8, iova_alloc: &I) -> Result<Self>
where
I: IovaAllocator,
{
let (mem, size, mem_layout) = Self::init_memory(queue_size, tag, iova_alloc)?;
let mut queue = Self {
mem,
size: Wrapping(size),
@ -123,8 +132,6 @@ impl UserQueue {
used_count: Wrapping(0),
free_count: Wrapping(size),
device_writable,
#[cfg(not(test))]
addr_table: Default::default(),
};
queue.init_descriptor_table()?;
@ -133,7 +140,14 @@ impl UserQueue {
}
/// Allocates memory region and returns addresses on the regions for (`desc_table`, `avail_ring`, `used_ring`, `buffer``).
fn init_memory(max_queue_size: u16) -> Result<(GuestMemory, u16, MemLayout)> {
fn init_memory<I>(
max_queue_size: u16,
tag: u8,
iova_alloc: &I,
) -> Result<(QueueMemory, u16, MemLayout)>
where
I: IovaAllocator,
{
// Since vhost-user negotiation finishes within ~20 messages, queue size 32 is enough.
const MAX_QUEUE_SIZE: u16 = 256;
@ -149,51 +163,44 @@ impl UserQueue {
((n + m - 1) / m) * m
}
let desc_table = GuestAddress(0);
let desc_table = IOVA(0);
let desc_size = 16u64 * u64::from(queue_size);
let desc_end = desc_table.0 + desc_size;
let avail_ring = GuestAddress(align(desc_end, 2));
let avail_ring = IOVA(align(desc_end, 2));
let avail_size = 6 + 2 * u64::from(queue_size);
let avail_end = avail_ring.0 + avail_size;
let used_ring = GuestAddress(align(avail_end, 4));
let used_ring = IOVA(align(avail_end, 4));
let used_size = 6 + 8 * u64::from(queue_size);
let used_end = used_ring.0 + used_size;
let buffer_addr = GuestAddress(align(used_end, BUF_SIZE));
let buffer_addr = IOVA(align(used_end, BUF_SIZE));
let buffer_size = BUF_SIZE * u64::from(queue_size);
let mem_size = align(buffer_addr.0 + buffer_size, base::pagesize() as u64);
let iova_start = iova_alloc
.alloc_iova(mem_size, tag)
.context("failed to allocate queue iova")?;
let mem = GuestMemory::new(&[(desc_table, mem_size)])
.map_err(|e| anyhow!("failed to create GuestMemory for virtqueue: {}", e))?;
let mem = QueueMemory::new(&[(IOVA(iova_start), mem_size)])
.map_err(|e| anyhow!("failed to create QueueMemory for virtqueue: {}", e))?;
// Call `mlock()` to guarantees that pages will stay in RAM.
// Note that this can't ensure that physical address mapping is consistent.
// TODO(b/215310597) We're assume that the kernel won't swap these memory region at least
// during the vvu negotiation. Although this assumption is risky, it'll be resolved once
// virtio-iommu for virtio devices is supported.
mem.with_regions(|_, _, size, ptr, _, _| {
let ret = unsafe { libc::mlock(ptr as *const libc::c_void, size) };
if ret == -1 {
bail!("failed to mlock(): {}", base::Error::last());
}
Ok(())
})?;
// To ensure the GuestMemory is mapped to physical memory, read the entire buffer first.
// Otherwise, reading `/proc/self/pagemap` returns invalid values.
// TODO(b/215310597): Once we use iommu for VFIO, we can probably remove this workaround.
let mut buf = vec![0; mem_size as usize];
mem.read_at_addr(&mut buf, desc_table)
.map_err(|e| anyhow!("failed to read_slice: {}", e))?;
let host_addr = mem
.get_host_address_range(IOVA(iova_start), mem_size as usize)
.context("failed to get host address")?;
// Safe because the region being mapped is managed via the GuestMemory interface.
unsafe {
iova_alloc
.map_iova(iova_start, mem_size, host_addr)
.context("failed to map queue")?;
}
let mem_layout = MemLayout {
desc_table,
avail_ring,
used_ring,
buffer_addr,
desc_table: desc_table.unchecked_add(iova_start),
avail_ring: avail_ring.unchecked_add(iova_start),
used_ring: used_ring.unchecked_add(iova_start),
buffer_addr: buffer_addr.unchecked_add(iova_start),
};
Ok((mem, queue_size, mem_layout))
@ -201,8 +208,6 @@ impl UserQueue {
/// Initialize the descriptor table.
fn init_descriptor_table(&mut self) -> Result<()> {
self.init_addr_table()?;
let flags = if self.device_writable {
Le16::from(VRING_DESC_F_WRITE as u16)
} else {
@ -214,9 +219,9 @@ impl UserQueue {
// Register pre-allocated buffers to the descriptor area.
for i in 0..self.size.0 {
let idx = Wrapping(i);
let addr = Le64::from(self.to_phys_addr(&self.buffer_guest_addr(idx)?)?);
let iova = self.buffer_address(idx)?.offset();
let desc = Desc {
addr,
addr: iova.into(),
len,
flags,
next,
@ -247,91 +252,16 @@ impl UserQueue {
Ok(())
}
#[cfg(not(test))]
/// Reads `/proc/self/pagemap` and stores mapping from virtual addresses for virtqueue
/// information and buffers to physical addresses.
fn init_addr_table(&mut self) -> Result<()> {
let pagemap = File::open("/proc/self/pagemap").context("failed to open pagemap")?;
self.register_addr(&pagemap, &self.mem_layout.desc_table.clone())?;
self.register_addr(&pagemap, &self.mem_layout.avail_ring.clone())?;
self.register_addr(&pagemap, &self.mem_layout.used_ring.clone())?;
self.register_addr(&pagemap, &self.mem_layout.buffer_addr.clone())?;
// Register addresses of buffers.
for i in 0..self.size.0 {
self.register_addr(&pagemap, &self.buffer_guest_addr(Wrapping(i))?)?;
}
Ok(())
}
#[cfg(test)]
fn init_addr_table(&mut self) -> Result<()> {
Ok(())
}
/// Registers an address mapping for the given virtual address to `self.addr_table`.
// TODO(b/215310597): This function reads `/proc/self/pagemap`, which requires root
// privileges. Instead, we should use VFIO_IOMMU_MAP_DMA call with virtio-iommu to get
// physical addresses.
#[cfg(not(test))]
fn register_addr(&mut self, pagemap_file: &File, addr: &GuestAddress) -> Result<u64> {
use std::os::unix::fs::FileExt;
let vaddr = self
.mem
.get_slice_at_addr(*addr, 1)
.context("failed to get slice")?
.as_ptr() as u64;
let page_size = base::pagesize() as u64;
let virt_page_number = vaddr / page_size;
let offset = std::mem::size_of::<u64>() as u64 * virt_page_number;
let mut buf = [0u8; 8];
pagemap_file
.read_exact_at(&mut buf, offset)
.context("failed to read pagemap")?;
let pagemap = u64::from_le_bytes(buf);
// Bit 55 is soft-dirty.
if (pagemap & (1u64 << 55)) != 0 {
bail!("page table entry is soft-dirty")
}
// page frame numbers are bits 0-54
let page = pagemap & 0x7f_ffff_ffff_ffffu64;
if page == 0 {
bail!("failed to get page frame number: page={:x}", page);
}
let paddr = page * page_size + (vaddr % page_size);
self.addr_table.insert(*addr, paddr);
Ok(paddr)
}
/// Translate a virtual address to the physical address.
#[cfg(not(test))]
fn to_phys_addr(&self, addr: &GuestAddress) -> Result<u64> {
self.addr_table
.get(addr)
.context(anyhow!("addr {} not found", addr))
.map(|v| *v)
}
#[cfg(test)]
fn to_phys_addr(&self, addr: &GuestAddress) -> Result<u64> {
Ok(addr.0)
}
/// Returns physical addresses of the descriptor table, the avail ring and the used ring.
pub fn desc_table_addrs(&self) -> Result<DescTableAddrs> {
let desc = self.to_phys_addr(&self.mem_layout.desc_table)?;
let avail = self.to_phys_addr(&self.mem_layout.avail_ring)?;
let used = self.to_phys_addr(&self.mem_layout.used_ring)?;
Ok(DescTableAddrs { desc, avail, used })
Ok(DescTableAddrs {
desc: self.mem_layout.desc_table.offset(),
avail: self.mem_layout.avail_ring.offset(),
used: self.mem_layout.used_ring.offset(),
})
}
/// Returns a virtual address of the buffer for the given `index`.
fn buffer_guest_addr(&self, index: Wrapping<u16>) -> Result<GuestAddress> {
/// Returns the IOVA of the buffer for the given `index`.
fn buffer_address(&self, index: Wrapping<u16>) -> Result<IOVA> {
let offset = u64::from((index % self.size).0) * BUF_SIZE;
self.mem_layout
.buffer_addr
@ -341,7 +271,10 @@ impl UserQueue {
/// Writes the given descriptor table entry.
fn write_desc_entry(&self, index: Wrapping<u16>, desc: Desc) -> Result<()> {
let addr = GuestAddress(u64::from((index % self.size).0) * mem::size_of::<Desc>() as u64);
let addr = self
.mem_layout
.desc_table
.unchecked_add(u64::from((index % self.size).0) * mem::size_of::<Desc>() as u64);
fence(Ordering::SeqCst);
self.mem
.write_obj_at_addr(desc, addr)
@ -402,7 +335,7 @@ impl UserQueue {
let id = Wrapping(u32::from(elem.id) as u16);
let len = u32::from(elem.len) as usize;
let addr = self.buffer_guest_addr(id)?;
let addr = self.buffer_address(id)?;
fence(Ordering::SeqCst);
let s = self
@ -419,7 +352,7 @@ impl UserQueue {
/// Writes data into virtqueue's buffer and returns its address.
///
/// TODO: Use `descriptor_utils::Writer`.
fn write_to_buffer(&self, index: Wrapping<u16>, data: &[u8]) -> Result<GuestAddress> {
fn write_to_buffer(&self, index: Wrapping<u16>, data: &[u8]) -> Result<IOVA> {
if data.len() as u64 > BUF_SIZE {
bail!(
"data size {} is larger than the buffer size {}",
@ -428,7 +361,7 @@ impl UserQueue {
);
}
let addr = self.buffer_guest_addr(index)?;
let addr = self.buffer_address(index)?;
fence(Ordering::SeqCst);
let written = self
.mem
@ -473,7 +406,7 @@ impl UserQueue {
.context("failed to write data to virtqueue")?;
let desc = Desc {
addr: Le64::from(self.to_phys_addr(&addr)?),
addr: Le64::from(addr.offset()),
len: Le32::from(data.len() as u32),
flags: Le16::from(0),
next: Le16::from(0),
@ -492,19 +425,40 @@ impl UserQueue {
mod test {
use super::*;
use std::cell::RefCell;
use std::io::Read;
use std::io::Write;
use crate::virtio::{Queue as DeviceQueue, Reader, Writer};
// An allocator that just allocates 0 as an IOVA.
struct SimpleIovaAllocator(RefCell<bool>);
impl IovaAllocator for SimpleIovaAllocator {
fn alloc_iova(&self, _size: u64, _tag: u8) -> Result<u64> {
if *self.0.borrow() {
bail!("exhaused");
}
*self.0.borrow_mut() = true;
Ok(0)
}
unsafe fn map_iova(&self, _iova: u64, _size: u64, _addr: *const u8) -> Result<()> {
if !*self.0.borrow() {
bail!("not allocated");
}
Ok(())
}
}
fn setup_vq(queue: &mut DeviceQueue, addrs: DescTableAddrs) {
queue.desc_table = GuestAddress(addrs.desc);
queue.avail_ring = GuestAddress(addrs.avail);
queue.used_ring = GuestAddress(addrs.used);
queue.desc_table = IOVA(addrs.desc);
queue.avail_ring = IOVA(addrs.avail);
queue.used_ring = IOVA(addrs.used);
queue.ready = true;
}
fn device_write(mem: &GuestMemory, q: &mut DeviceQueue, data: &[u8]) -> usize {
fn device_write(mem: &QueueMemory, q: &mut DeviceQueue, data: &[u8]) -> usize {
let desc_chain = q.pop(mem).unwrap();
let index = desc_chain.index;
@ -514,7 +468,7 @@ mod test {
written
}
fn device_read(mem: &GuestMemory, q: &mut DeviceQueue, len: usize) -> Vec<u8> {
fn device_read(mem: &QueueMemory, q: &mut DeviceQueue, len: usize) -> Vec<u8> {
let desc_chain = q.pop(mem).unwrap();
let desc_index = desc_chain.index;
let mut reader = Reader::new(mem.clone(), desc_chain).unwrap();
@ -538,7 +492,9 @@ mod test {
// Send an array from the driver to the device `count` times.
fn drv_to_dev(queue_size: u16, count: u32) {
let mut drv_queue = UserQueue::new(queue_size, false /* device_writable */).unwrap();
let iova_alloc = SimpleIovaAllocator(RefCell::new(false));
let mut drv_queue =
UserQueue::new(queue_size, false /* device_writable */, 0, &iova_alloc).unwrap();
let mut dev_queue = DeviceQueue::new(queue_size);
setup_vq(&mut dev_queue, drv_queue.desc_table_addrs().unwrap());
@ -582,7 +538,9 @@ mod test {
// Send an array from the device to the driver `count` times.
fn dev_to_drv(queue_size: u16, count: u32) {
let mut drv_queue = UserQueue::new(queue_size, true /* device_writable */).unwrap();
let iova_alloc = SimpleIovaAllocator(RefCell::new(false));
let mut drv_queue =
UserQueue::new(queue_size, true /* device_writable */, 0, &iova_alloc).unwrap();
let mut dev_queue = DeviceQueue::new(queue_size);
setup_vq(&mut dev_queue, drv_queue.desc_table_addrs().unwrap());

View file

@ -46,7 +46,7 @@ use crate::{
PciBarConfiguration, PciBarIndex, PciBarPrefetchable, PciBarRegionType, PciCapability,
PciCapabilityID,
},
virtio::VIRTIO_MSI_NO_VECTOR,
virtio::{VIRTIO_F_ACCESS_PLATFORM, VIRTIO_MSI_NO_VECTOR},
};
use remain::sorted;
@ -997,7 +997,7 @@ impl VirtioVhostUser {
uuid: Option<Uuid>,
) -> Result<VirtioVhostUser> {
Ok(VirtioVhostUser {
base_features,
base_features: base_features | 1 << VIRTIO_F_ACCESS_PLATFORM,
listener: Some(listener),
config: VirtioVhostUserConfig {
status: Le32::from(0),

View file

@ -12,7 +12,7 @@ use thiserror::Error;
pub use crate::system_allocator::{MemRegion, MmioType, SystemAllocator, SystemAllocatorConfig};
mod address_allocator;
pub mod address_allocator;
mod system_allocator;
/// Used to tag SystemAllocator allocations.
@ -36,6 +36,8 @@ pub enum Alloc {
PciBridgePrefetchWindow { bus: u8, dev: u8, func: u8 },
/// File-backed memory mapping.
FileBacked(u64),
/// virtio vhost user queue with queue id
VvuQueue(u8),
}
#[sorted]

View file

@ -6,3 +6,4 @@ edition = "2021"
[dependencies]
base = { path = "../base" }
data_model = { path = "../common/data_model" }

View file

@ -17,12 +17,33 @@ fix_vfio_type() {
}
VFIO_EXTRA="// Added by vfio_sys/bindgen.sh
use data_model::DataInit;
#[repr(C)]
#[derive(Debug, Default)]
pub struct vfio_region_info_with_cap {
pub region_info: vfio_region_info,
pub cap_info: __IncompleteArrayField<u8>,
}"
}
// vfio_iommu_type1_info_cap_iova_range minus the incomplete iova_ranges
// array, so that Copy/DataInit can be implemented.
#[repr(C)]
#[derive(Debug, Default, Copy, Clone)]
pub struct vfio_iommu_type1_info_cap_iova_range_header {
pub header: vfio_info_cap_header,
pub nr_iovas: u32,
pub reserved: u32,
}
// Safe because it only has data and no implicit padding.
unsafe impl DataInit for vfio_info_cap_header {}
// Safe because it only has data and no implicit padding.
unsafe impl DataInit for vfio_iommu_type1_info_cap_iova_range_header {}
// Safe because it only has data and no implicit padding.
unsafe impl DataInit for vfio_iova_range {}"
bindgen_generate \
--raw-line "${VFIO_EXTRA}" \

View file

@ -6,6 +6,8 @@
#![allow(dead_code)]
// Added by vfio_sys/bindgen.sh
use data_model::DataInit;
#[repr(C)]
#[derive(Debug, Default)]
pub struct vfio_region_info_with_cap {
@ -13,6 +15,25 @@ pub struct vfio_region_info_with_cap {
pub cap_info: __IncompleteArrayField<u8>,
}
// vfio_iommu_type1_info_cap_iova_range minus the incomplete iova_ranges
// array, so that Copy/DataInit can be implemented.
#[repr(C)]
#[derive(Debug, Default, Copy, Clone)]
pub struct vfio_iommu_type1_info_cap_iova_range_header {
pub header: vfio_info_cap_header,
pub nr_iovas: u32,
pub reserved: u32,
}
// Safe because it only has data and no implicit padding.
unsafe impl DataInit for vfio_info_cap_header {}
// Safe because it only has data and no implicit padding.
unsafe impl DataInit for vfio_iommu_type1_info_cap_iova_range_header {}
// Safe because it only has data and no implicit padding.
unsafe impl DataInit for vfio_iova_range {}
#[repr(C)]
#[derive(Default)]
pub struct __IncompleteArrayField<T>(::std::marker::PhantomData<T>, [T; 0]);