#[cfg(test)] use crate::hal::VirtAddr; use crate::hal::{BufferDirection, Dma, Hal, PhysAddr}; use crate::transport::Transport; use crate::{align_up, nonnull_slice_from_raw_parts, pages, Error, Result, PAGE_SIZE}; use bitflags::bitflags; #[cfg(test)] use core::cmp::min; use core::hint::spin_loop; use core::mem::size_of; #[cfg(test)] use core::ptr; use core::ptr::{addr_of_mut, NonNull}; use core::sync::atomic::{fence, Ordering}; /// The mechanism for bulk data transport on virtio devices. /// /// Each device can have zero or more virtqueues. /// /// * `SIZE`: The size of the queue. This is both the number of descriptors, and the number of slots /// in the available and used rings. #[derive(Debug)] pub struct VirtQueue { /// DMA guard layout: VirtQueueLayout, /// Descriptor table desc: NonNull<[Descriptor]>, /// Available ring avail: NonNull>, /// Used ring used: NonNull>, /// The index of queue queue_idx: u16, /// The number of descriptors currently in use. num_used: u16, /// The head desc index of the free list. free_head: u16, avail_idx: u16, last_used_idx: u16, } impl VirtQueue { /// Create a new VirtQueue. pub fn new(transport: &mut T, idx: u16) -> Result { if transport.queue_used(idx) { return Err(Error::AlreadyUsed); } if !SIZE.is_power_of_two() || SIZE > u16::MAX.into() || transport.max_queue_size() < SIZE as u32 { return Err(Error::InvalidParam); } let size = SIZE as u16; let layout = if transport.requires_legacy_layout() { VirtQueueLayout::allocate_legacy(size)? } else { VirtQueueLayout::allocate_flexible(size)? }; transport.queue_set( idx, size.into(), layout.descriptors_paddr(), layout.driver_area_paddr(), layout.device_area_paddr(), ); let desc = nonnull_slice_from_raw_parts(layout.descriptors_vaddr().cast::(), SIZE); let avail = layout.avail_vaddr().cast(); let used = layout.used_vaddr().cast(); // Link descriptors together. for i in 0..(size - 1) { // Safe because `desc` is properly aligned, dereferenceable, initialised, and the device // won't access the descriptors for the duration of this unsafe block. unsafe { (*desc.as_ptr())[i as usize].next = i + 1; } } Ok(VirtQueue { layout, desc, avail, used, queue_idx: idx, num_used: 0, free_head: 0, avail_idx: 0, last_used_idx: 0, }) } /// Add buffers to the virtqueue, return a token. /// /// Ref: linux virtio_ring.c virtqueue_add /// /// # Safety /// /// The input and output buffers must remain valid until the token is returned by `pop_used`. pub unsafe fn add(&mut self, inputs: &[*const [u8]], outputs: &[*mut [u8]]) -> Result { if inputs.is_empty() && outputs.is_empty() { return Err(Error::InvalidParam); } if inputs.len() + outputs.len() + self.num_used as usize > SIZE { return Err(Error::QueueFull); } // allocate descriptors from free list let head = self.free_head; let mut last = self.free_head; // Safe because self.desc is properly aligned, dereferenceable and initialised, and nothing // else reads or writes the free descriptors during this block. unsafe { for (buffer, direction) in input_output_iter(inputs, outputs) { let desc = self.desc_ptr(self.free_head); (*desc).set_buf::(buffer, direction, DescFlags::NEXT); last = self.free_head; self.free_head = (*desc).next; } // set last_elem.next = NULL (*self.desc_ptr(last)).flags.remove(DescFlags::NEXT); } self.num_used += (inputs.len() + outputs.len()) as u16; let avail_slot = self.avail_idx & (SIZE as u16 - 1); // Safe because self.avail is properly aligned, dereferenceable and initialised. unsafe { (*self.avail.as_ptr()).ring[avail_slot as usize] = head; } // Write barrier so that device sees changes to descriptor table and available ring before // change to available index. fence(Ordering::SeqCst); // increase head of avail ring self.avail_idx = self.avail_idx.wrapping_add(1); // Safe because self.avail is properly aligned, dereferenceable and initialised. unsafe { (*self.avail.as_ptr()).idx = self.avail_idx; } // Write barrier so that device can see change to available index after this method returns. fence(Ordering::SeqCst); Ok(head) } /// Add the given buffers to the virtqueue, notifies the device, blocks until the device uses /// them, then pops them. /// /// This assumes that the device isn't processing any other buffers at the same time. pub fn add_notify_wait_pop( &mut self, inputs: &[*const [u8]], outputs: &[*mut [u8]], transport: &mut impl Transport, ) -> Result { // Safe because we don't return until the same token has been popped, so they remain valid // until then. let token = unsafe { self.add(inputs, outputs) }?; // Notify the queue. transport.notify(self.queue_idx); // Wait until there is at least one element in the used ring. while !self.can_pop() { spin_loop(); } self.pop_used(token, inputs, outputs) } /// Returns a non-null pointer to the descriptor at the given index. fn desc_ptr(&mut self, index: u16) -> *mut Descriptor { // Safe because self.desc is properly aligned and dereferenceable. unsafe { addr_of_mut!((*self.desc.as_ptr())[index as usize]) } } /// Returns whether there is a used element that can be popped. pub fn can_pop(&self) -> bool { // Read barrier, so we read a fresh value from the device. fence(Ordering::SeqCst); // Safe because self.used points to a valid, aligned, initialised, dereferenceable, readable // instance of UsedRing. self.last_used_idx != unsafe { (*self.used.as_ptr()).idx } } /// Returns the descriptor index (a.k.a. token) of the next used element without popping it, or /// `None` if the used ring is empty. pub fn peek_used(&self) -> Option { if self.can_pop() { let last_used_slot = self.last_used_idx & (SIZE as u16 - 1); // Safe because self.used points to a valid, aligned, initialised, dereferenceable, // readable instance of UsedRing. Some(unsafe { (*self.used.as_ptr()).ring[last_used_slot as usize].id as u16 }) } else { None } } /// Returns the number of free descriptors. pub fn available_desc(&self) -> usize { SIZE - self.num_used as usize } /// Unshares buffers in the list starting at descriptor index `head` and adds them to the free /// list. Unsharing may involve copying data back to the original buffers, so they must be /// passed in too. /// /// This will push all linked descriptors at the front of the free list. fn recycle_descriptors(&mut self, head: u16, inputs: &[*const [u8]], outputs: &[*mut [u8]]) { let original_free_head = self.free_head; self.free_head = head; let mut next = Some(head); for (buffer, direction) in input_output_iter(inputs, outputs) { let desc = self.desc_ptr(next.expect("Descriptor chain was shorter than expected.")); // Safe because self.desc is properly aligned, dereferenceable and initialised, and // nothing else reads or writes the descriptor during this block. let paddr = unsafe { let paddr = (*desc).addr; (*desc).unset_buf(); self.num_used -= 1; next = (*desc).next(); if next.is_none() { (*desc).next = original_free_head; } paddr }; // Unshare the buffer (and perhaps copy its contents back to the original buffer). H::unshare(paddr as usize, buffer, direction); } if next.is_some() { panic!("Descriptor chain was longer than expected."); } } /// If the given token is next on the device used queue, pops it and returns the total buffer /// length which was used (written) by the device. /// /// Ref: linux virtio_ring.c virtqueue_get_buf_ctx pub fn pop_used( &mut self, token: u16, inputs: &[*const [u8]], outputs: &[*mut [u8]], ) -> Result { if !self.can_pop() { return Err(Error::NotReady); } // Read barrier not necessary, as can_pop already has one. // Get the index of the start of the descriptor chain for the next element in the used ring. let last_used_slot = self.last_used_idx & (SIZE as u16 - 1); let index; let len; // Safe because self.used points to a valid, aligned, initialised, dereferenceable, readable // instance of UsedRing. unsafe { index = (*self.used.as_ptr()).ring[last_used_slot as usize].id as u16; len = (*self.used.as_ptr()).ring[last_used_slot as usize].len; } if index != token { // The device used a different descriptor chain to the one we were expecting. return Err(Error::WrongToken); } self.recycle_descriptors(index, inputs, outputs); self.last_used_idx = self.last_used_idx.wrapping_add(1); Ok(len) } } /// The inner layout of a VirtQueue. /// /// Ref: 2.6 Split Virtqueues #[derive(Debug)] enum VirtQueueLayout { Legacy { dma: Dma, avail_offset: usize, used_offset: usize, }, Modern { /// The region used for the descriptor area and driver area. driver_to_device_dma: Dma, /// The region used for the device area. device_to_driver_dma: Dma, /// The offset from the start of the `driver_to_device_dma` region to the driver area /// (available ring). avail_offset: usize, }, } impl VirtQueueLayout { /// Allocates a single DMA region containing all parts of the virtqueue, following the layout /// required by legacy interfaces. /// /// Ref: 2.6.2 Legacy Interfaces: A Note on Virtqueue Layout fn allocate_legacy(queue_size: u16) -> Result { let (desc, avail, used) = queue_part_sizes(queue_size); let size = align_up(desc + avail) + align_up(used); // Allocate contiguous pages. let dma = Dma::new(size / PAGE_SIZE, BufferDirection::Both)?; Ok(Self::Legacy { dma, avail_offset: desc, used_offset: align_up(desc + avail), }) } /// Allocates separate DMA regions for the the different parts of the virtqueue, as supported by /// non-legacy interfaces. /// /// This is preferred over `allocate_legacy` where possible as it reduces memory fragmentation /// and allows the HAL to know which DMA regions are used in which direction. fn allocate_flexible(queue_size: u16) -> Result { let (desc, avail, used) = queue_part_sizes(queue_size); let driver_to_device_dma = Dma::new(pages(desc + avail), BufferDirection::DriverToDevice)?; let device_to_driver_dma = Dma::new(pages(used), BufferDirection::DeviceToDriver)?; Ok(Self::Modern { driver_to_device_dma, device_to_driver_dma, avail_offset: desc, }) } /// Returns the physical address of the descriptor area. fn descriptors_paddr(&self) -> PhysAddr { match self { Self::Legacy { dma, .. } => dma.paddr(), Self::Modern { driver_to_device_dma, .. } => driver_to_device_dma.paddr(), } } /// Returns a pointer to the descriptor table (in the descriptor area). fn descriptors_vaddr(&self) -> NonNull { match self { Self::Legacy { dma, .. } => dma.vaddr(0), Self::Modern { driver_to_device_dma, .. } => driver_to_device_dma.vaddr(0), } } /// Returns the physical address of the driver area. fn driver_area_paddr(&self) -> PhysAddr { match self { Self::Legacy { dma, avail_offset, .. } => dma.paddr() + avail_offset, Self::Modern { driver_to_device_dma, avail_offset, .. } => driver_to_device_dma.paddr() + avail_offset, } } /// Returns a pointer to the available ring (in the driver area). fn avail_vaddr(&self) -> NonNull { match self { Self::Legacy { dma, avail_offset, .. } => dma.vaddr(*avail_offset), Self::Modern { driver_to_device_dma, avail_offset, .. } => driver_to_device_dma.vaddr(*avail_offset), } } /// Returns the physical address of the device area. fn device_area_paddr(&self) -> PhysAddr { match self { Self::Legacy { used_offset, dma, .. } => dma.paddr() + used_offset, Self::Modern { device_to_driver_dma, .. } => device_to_driver_dma.paddr(), } } /// Returns a pointer to the used ring (in the driver area). fn used_vaddr(&self) -> NonNull { match self { Self::Legacy { dma, used_offset, .. } => dma.vaddr(*used_offset), Self::Modern { device_to_driver_dma, .. } => device_to_driver_dma.vaddr(0), } } } /// Returns the size in bytes of the descriptor table, available ring and used ring for a given /// queue size. /// /// Ref: 2.6 Split Virtqueues fn queue_part_sizes(queue_size: u16) -> (usize, usize, usize) { assert!( queue_size.is_power_of_two(), "queue size should be a power of 2" ); let queue_size = queue_size as usize; let desc = size_of::() * queue_size; let avail = size_of::() * (3 + queue_size); let used = size_of::() * 3 + size_of::() * queue_size; (desc, avail, used) } #[repr(C, align(16))] #[derive(Debug)] pub(crate) struct Descriptor { addr: u64, len: u32, flags: DescFlags, next: u16, } impl Descriptor { /// Sets the buffer address, length and flags, and shares it with the device. /// /// # Safety /// /// The caller must ensure that the buffer lives at least as long as the descriptor is active. unsafe fn set_buf( &mut self, buf: NonNull<[u8]>, direction: BufferDirection, extra_flags: DescFlags, ) { self.addr = H::share(buf, direction) as u64; self.len = buf.len() as u32; self.flags = extra_flags | match direction { BufferDirection::DeviceToDriver => DescFlags::WRITE, BufferDirection::DriverToDevice => DescFlags::empty(), BufferDirection::Both => { panic!("Buffer passed to device should never use BufferDirection::Both.") } }; } /// Sets the buffer address and length to 0. /// /// This must only be called once the device has finished using the descriptor. fn unset_buf(&mut self) { self.addr = 0; self.len = 0; } /// Returns the index of the next descriptor in the chain if the `NEXT` flag is set, or `None` /// if it is not (and thus this descriptor is the end of the chain). fn next(&self) -> Option { if self.flags.contains(DescFlags::NEXT) { Some(self.next) } else { None } } } bitflags! { /// Descriptor flags struct DescFlags: u16 { const NEXT = 1; const WRITE = 2; const INDIRECT = 4; } } /// The driver uses the available ring to offer buffers to the device: /// each ring entry refers to the head of a descriptor chain. /// It is only written by the driver and read by the device. #[repr(C)] #[derive(Debug)] struct AvailRing { flags: u16, /// A driver MUST NOT decrement the idx. idx: u16, ring: [u16; SIZE], used_event: u16, // unused } /// The used ring is where the device returns buffers once it is done with them: /// it is only written to by the device, and read by the driver. #[repr(C)] #[derive(Debug)] struct UsedRing { flags: u16, idx: u16, ring: [UsedElem; SIZE], avail_event: u16, // unused } #[repr(C)] #[derive(Debug)] struct UsedElem { id: u32, len: u32, } /// Simulates the device writing to a VirtIO queue, for use in tests. /// /// The fake device always uses descriptors in order. #[cfg(test)] pub(crate) fn fake_write_to_queue( receive_queue_descriptors: *const Descriptor, receive_queue_driver_area: VirtAddr, receive_queue_device_area: VirtAddr, data: &[u8], ) { let descriptors = ptr::slice_from_raw_parts(receive_queue_descriptors, QUEUE_SIZE); let available_ring = receive_queue_driver_area as *const AvailRing; let used_ring = receive_queue_device_area as *mut UsedRing; // Safe because the various pointers are properly aligned, dereferenceable, initialised, and // nothing else accesses them during this block. unsafe { // Make sure there is actually at least one descriptor available to write to. assert_ne!((*available_ring).idx, (*used_ring).idx); // The fake device always uses descriptors in order, like VIRTIO_F_IN_ORDER, so // `used_ring.idx` marks the next descriptor we should take from the available ring. let next_slot = (*used_ring).idx & (QUEUE_SIZE as u16 - 1); let head_descriptor_index = (*available_ring).ring[next_slot as usize]; let mut descriptor = &(*descriptors)[head_descriptor_index as usize]; // Loop through all descriptors in the chain, writing data to them. let mut remaining_data = data; loop { // Check the buffer and write to it. let flags = descriptor.flags; assert!(flags.contains(DescFlags::WRITE)); let buffer_length = descriptor.len as usize; let length_to_write = min(remaining_data.len(), buffer_length); ptr::copy( remaining_data.as_ptr(), descriptor.addr as *mut u8, length_to_write, ); remaining_data = &remaining_data[length_to_write..]; if let Some(next) = descriptor.next() { descriptor = &(*descriptors)[next as usize]; } else { assert_eq!(remaining_data.len(), 0); break; } } // Mark the buffer as used. (*used_ring).ring[next_slot as usize].id = head_descriptor_index as u32; (*used_ring).ring[next_slot as usize].len = data.len() as u32; (*used_ring).idx += 1; } } #[cfg(test)] mod tests { use super::*; use crate::{ hal::fake::FakeHal, transport::mmio::{MmioTransport, VirtIOHeader, MODERN_VERSION}, }; use core::ptr::NonNull; #[test] fn invalid_queue_size() { let mut header = VirtIOHeader::make_fake_header(MODERN_VERSION, 1, 0, 0, 4); let mut transport = unsafe { MmioTransport::new(NonNull::from(&mut header)) }.unwrap(); // Size not a power of 2. assert_eq!( VirtQueue::::new(&mut transport, 0).unwrap_err(), Error::InvalidParam ); } #[test] fn queue_too_big() { let mut header = VirtIOHeader::make_fake_header(MODERN_VERSION, 1, 0, 0, 4); let mut transport = unsafe { MmioTransport::new(NonNull::from(&mut header)) }.unwrap(); assert_eq!( VirtQueue::::new(&mut transport, 0).unwrap_err(), Error::InvalidParam ); } #[test] fn queue_already_used() { let mut header = VirtIOHeader::make_fake_header(MODERN_VERSION, 1, 0, 0, 4); let mut transport = unsafe { MmioTransport::new(NonNull::from(&mut header)) }.unwrap(); VirtQueue::::new(&mut transport, 0).unwrap(); assert_eq!( VirtQueue::::new(&mut transport, 0).unwrap_err(), Error::AlreadyUsed ); } #[test] fn add_empty() { let mut header = VirtIOHeader::make_fake_header(MODERN_VERSION, 1, 0, 0, 4); let mut transport = unsafe { MmioTransport::new(NonNull::from(&mut header)) }.unwrap(); let mut queue = VirtQueue::::new(&mut transport, 0).unwrap(); assert_eq!( unsafe { queue.add(&[], &[]) }.unwrap_err(), Error::InvalidParam ); } #[test] fn add_too_many() { let mut header = VirtIOHeader::make_fake_header(MODERN_VERSION, 1, 0, 0, 4); let mut transport = unsafe { MmioTransport::new(NonNull::from(&mut header)) }.unwrap(); let mut queue = VirtQueue::::new(&mut transport, 0).unwrap(); assert_eq!(queue.available_desc(), 4); assert_eq!( unsafe { queue.add(&[&[], &[], &[]], &[&mut [], &mut []]) }.unwrap_err(), Error::QueueFull ); } #[test] fn add_buffers() { let mut header = VirtIOHeader::make_fake_header(MODERN_VERSION, 1, 0, 0, 4); let mut transport = unsafe { MmioTransport::new(NonNull::from(&mut header)) }.unwrap(); let mut queue = VirtQueue::::new(&mut transport, 0).unwrap(); assert_eq!(queue.available_desc(), 4); // Add a buffer chain consisting of two device-readable parts followed by two // device-writable parts. let token = unsafe { queue.add(&[&[1, 2], &[3]], &[&mut [0, 0], &mut [0]]) }.unwrap(); assert_eq!(queue.available_desc(), 0); assert!(!queue.can_pop()); // Safe because the various parts of the queue are properly aligned, dereferenceable and // initialised, and nothing else is accessing them at the same time. unsafe { let first_descriptor_index = (*queue.avail.as_ptr()).ring[0]; assert_eq!(first_descriptor_index, token); assert_eq!( (*queue.desc.as_ptr())[first_descriptor_index as usize].len, 2 ); assert_eq!( (*queue.desc.as_ptr())[first_descriptor_index as usize].flags, DescFlags::NEXT ); let second_descriptor_index = (*queue.desc.as_ptr())[first_descriptor_index as usize].next; assert_eq!( (*queue.desc.as_ptr())[second_descriptor_index as usize].len, 1 ); assert_eq!( (*queue.desc.as_ptr())[second_descriptor_index as usize].flags, DescFlags::NEXT ); let third_descriptor_index = (*queue.desc.as_ptr())[second_descriptor_index as usize].next; assert_eq!( (*queue.desc.as_ptr())[third_descriptor_index as usize].len, 2 ); assert_eq!( (*queue.desc.as_ptr())[third_descriptor_index as usize].flags, DescFlags::NEXT | DescFlags::WRITE ); let fourth_descriptor_index = (*queue.desc.as_ptr())[third_descriptor_index as usize].next; assert_eq!( (*queue.desc.as_ptr())[fourth_descriptor_index as usize].len, 1 ); assert_eq!( (*queue.desc.as_ptr())[fourth_descriptor_index as usize].flags, DescFlags::WRITE ); } } } /// Returns an iterator over the buffers of first `inputs` and then `outputs`, paired with the /// corresponding `BufferDirection`. /// /// Panics if any of the buffer pointers is null. fn input_output_iter<'a>( inputs: &'a [*const [u8]], outputs: &'a [*mut [u8]], ) -> impl Iterator, BufferDirection)> + 'a { inputs .iter() .map(|input| { ( NonNull::new(*input as *mut [u8]).unwrap(), BufferDirection::DriverToDevice, ) }) .chain(outputs.iter().map(|output| { ( NonNull::new(*output).unwrap(), BufferDirection::DeviceToDriver, ) })) }