virtio-devices: iommu: Port to EpollHelper

Migrate to EpollHelper so as to remove code that is duplicated between
multiple virtio devices.

Signed-off-by: Rob Bradford <robert.bradford@intel.com>
This commit is contained in:
Rob Bradford 2020-08-04 12:16:44 +01:00 committed by Sebastien Boeuf
parent 2e98208af5
commit b5d64be479

View File

@ -4,19 +4,19 @@
use super::Error as DeviceError; use super::Error as DeviceError;
use super::{ use super::{
ActivateError, ActivateResult, DescriptorChain, DeviceEventT, Queue, VirtioDevice, ActivateError, ActivateResult, DescriptorChain, EpollHelper, EpollHelperError,
VirtioDeviceType, VIRTIO_F_VERSION_1, EpollHelperHandler, Queue, VirtioDevice, VirtioDeviceType, EPOLL_HELPER_EVENT_LAST,
VIRTIO_F_VERSION_1,
}; };
use crate::{DmaRemapping, VirtioInterrupt, VirtioInterruptType}; use crate::{DmaRemapping, VirtioInterrupt, VirtioInterruptType};
use anyhow::anyhow; use anyhow::anyhow;
use libc::EFD_NONBLOCK; use libc::EFD_NONBLOCK;
use std::collections::BTreeMap; use std::collections::BTreeMap;
use std::fmt::{self, Display}; use std::fmt::{self, Display};
use std::fs::File;
use std::io; use std::io;
use std::mem::size_of; use std::mem::size_of;
use std::ops::Bound::Included; use std::ops::Bound::Included;
use std::os::unix::io::{AsRawFd, FromRawFd}; use std::os::unix::io::AsRawFd;
use std::result; use std::result;
use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::atomic::{AtomicBool, Ordering};
use std::sync::{Arc, RwLock}; use std::sync::{Arc, RwLock};
@ -40,15 +40,11 @@ const QUEUE_SIZES: &[u16] = &[QUEUE_SIZE; NUM_QUEUES];
/// New descriptors are pending on the request queue. /// New descriptors are pending on the request queue.
/// "requestq" is meant to be used anytime an action is required to be /// "requestq" is meant to be used anytime an action is required to be
/// performed on behalf of the guest driver. /// performed on behalf of the guest driver.
const REQUEST_Q_EVENT: DeviceEventT = 0; const REQUEST_Q_EVENT: u16 = EPOLL_HELPER_EVENT_LAST + 1;
/// New descriptors are pending on the event queue. /// New descriptors are pending on the event queue.
/// "eventq" lets the device report any fault or other asynchronous event to /// "eventq" lets the device report any fault or other asynchronous event to
/// the guest driver. /// the guest driver.
const EVENT_Q_EVENT: DeviceEventT = 1; const EVENT_Q_EVENT: u16 = EPOLL_HELPER_EVENT_LAST + 2;
/// The device has been dropped.
const KILL_EVENT: DeviceEventT = 2;
/// The device should be paused.
const PAUSE_EVENT: DeviceEventT = 3;
/// PROBE properties size. /// PROBE properties size.
/// This is the minimal size to provide at least one RESV_MEM property. /// This is the minimal size to provide at least one RESV_MEM property.
@ -650,126 +646,47 @@ impl IommuEpollHandler {
}) })
} }
fn run(&mut self, paused: Arc<AtomicBool>) -> result::Result<(), DeviceError> { fn run(&mut self, paused: Arc<AtomicBool>) -> result::Result<(), EpollHelperError> {
// Create the epoll file descriptor let mut helper = EpollHelper::new(&self.kill_evt, &self.pause_evt)?;
let epoll_fd = epoll::create(true).map_err(DeviceError::EpollCreateFd)?; helper.add_event(self.queue_evts[0].as_raw_fd(), REQUEST_Q_EVENT)?;
// Use 'File' to enforce closing on 'epoll_fd' helper.add_event(self.queue_evts[1].as_raw_fd(), EVENT_Q_EVENT)?;
let epoll_file = unsafe { File::from_raw_fd(epoll_fd) }; helper.run(paused, self)?;
// Add events Ok(())
epoll::ctl( }
epoll_file.as_raw_fd(), }
epoll::ControlOptions::EPOLL_CTL_ADD,
self.queue_evts[0].as_raw_fd(),
epoll::Event::new(epoll::Events::EPOLLIN, u64::from(REQUEST_Q_EVENT)),
)
.map_err(DeviceError::EpollCtl)?;
epoll::ctl(
epoll_file.as_raw_fd(),
epoll::ControlOptions::EPOLL_CTL_ADD,
self.queue_evts[1].as_raw_fd(),
epoll::Event::new(epoll::Events::EPOLLIN, u64::from(EVENT_Q_EVENT)),
)
.map_err(DeviceError::EpollCtl)?;
epoll::ctl(
epoll_file.as_raw_fd(),
epoll::ControlOptions::EPOLL_CTL_ADD,
self.kill_evt.as_raw_fd(),
epoll::Event::new(epoll::Events::EPOLLIN, u64::from(KILL_EVENT)),
)
.map_err(DeviceError::EpollCtl)?;
epoll::ctl(
epoll_file.as_raw_fd(),
epoll::ControlOptions::EPOLL_CTL_ADD,
self.pause_evt.as_raw_fd(),
epoll::Event::new(epoll::Events::EPOLLIN, u64::from(PAUSE_EVENT)),
)
.map_err(DeviceError::EpollCtl)?;
const EPOLL_EVENTS_LEN: usize = 100; impl EpollHelperHandler for IommuEpollHandler {
let mut events = vec![epoll::Event::new(epoll::Events::empty(), 0); EPOLL_EVENTS_LEN]; fn handle_event(&mut self, _helper: &mut EpollHelper, event: u16) -> bool {
match event {
// Before jumping into the epoll loop, check if the device is expected REQUEST_Q_EVENT => {
// to be in a paused state. This is helpful for the restore code path if let Err(e) = self.queue_evts[0].read() {
// as the device thread should not start processing anything before the error!("Failed to get queue event: {:?}", e);
// device has been resumed. return true;
while paused.load(Ordering::SeqCst) { } else if self.request_queue() {
thread::park(); if let Err(e) = self.signal_used_queue(&self.queues[0]) {
} error!("Failed to signal used queue: {:?}", e);
return true;
'epoll: loop {
let num_events = match epoll::wait(epoll_file.as_raw_fd(), -1, &mut events[..]) {
Ok(res) => res,
Err(e) => {
if e.kind() == io::ErrorKind::Interrupted {
// It's well defined from the epoll_wait() syscall
// documentation that the epoll loop can be interrupted
// before any of the requested events occurred or the
// timeout expired. In both those cases, epoll_wait()
// returns an error of type EINTR, but this should not
// be considered as a regular error. Instead it is more
// appropriate to retry, by calling into epoll_wait().
continue;
}
return Err(DeviceError::EpollWait(e));
}
};
for event in events.iter().take(num_events) {
let ev_type = event.data as u16;
match ev_type {
REQUEST_Q_EVENT => {
if let Err(e) = self.queue_evts[0].read() {
error!("Failed to get queue event: {:?}", e);
break 'epoll;
} else if self.request_queue() {
if let Err(e) = self.signal_used_queue(&self.queues[0]) {
error!("Failed to signal used queue: {:?}", e);
break 'epoll;
}
}
}
EVENT_Q_EVENT => {
if let Err(e) = self.queue_evts[1].read() {
error!("Failed to get queue event: {:?}", e);
break 'epoll;
} else if self.event_queue() {
if let Err(e) = self.signal_used_queue(&self.queues[1]) {
error!("Failed to signal used queue: {:?}", e);
break 'epoll;
}
}
}
KILL_EVENT => {
debug!("kill_evt received, stopping epoll loop");
break 'epoll;
}
PAUSE_EVENT => {
debug!("PAUSE_EVENT received, pausing virtio-iommu epoll loop");
// We loop here to handle spurious park() returns.
// Until we have not resumed, the paused boolean will
// be true.
while paused.load(Ordering::SeqCst) {
thread::park();
}
// Drain pause event after the device has been resumed.
// This ensures the pause event has been seen by each
// and every thread related to this virtio device.
let _ = self.pause_evt.read();
}
_ => {
error!("Unknown event for virtio-iommu");
break 'epoll;
} }
} }
} }
EVENT_Q_EVENT => {
info!("Exit epoll loop"); if let Err(e) = self.queue_evts[1].read() {
error!("Failed to get queue event: {:?}", e);
return true;
} else if self.event_queue() {
if let Err(e) = self.signal_used_queue(&self.queues[1]) {
error!("Failed to signal used queue: {:?}", e);
return true;
}
}
}
_ => {
error!("Unexpected event: {}", event);
return true;
}
} }
false
Ok(())
} }
} }
@ -823,7 +740,7 @@ pub struct Iommu {
ext_mapping: BTreeMap<u32, Arc<dyn ExternalDmaMapping>>, ext_mapping: BTreeMap<u32, Arc<dyn ExternalDmaMapping>>,
queue_evts: Option<Vec<EventFd>>, queue_evts: Option<Vec<EventFd>>,
interrupt_cb: Option<Arc<dyn VirtioInterrupt>>, interrupt_cb: Option<Arc<dyn VirtioInterrupt>>,
epoll_threads: Option<Vec<thread::JoinHandle<result::Result<(), DeviceError>>>>, epoll_threads: Option<Vec<thread::JoinHandle<result::Result<(), EpollHelperError>>>>,
paused: Arc<AtomicBool>, paused: Arc<AtomicBool>,
} }