virtio-block: Port to EpollHelper

Port virtio-block device to the new EpollHelper. This required moving
the queue EventFd ownership to BlockEpollHandler.

Signed-off-by: Rob Bradford <robert.bradford@intel.com>
This commit is contained in:
Rob Bradford 2020-07-22 14:38:28 +01:00
parent 7d9dc4013e
commit 5cad2334dd

View File

@ -10,19 +10,18 @@
use super::Error as DeviceError;
use super::{
ActivateError, ActivateResult, DeviceEventT, Queue, VirtioDevice, VirtioDeviceType,
VirtioInterruptType,
ActivateError, ActivateResult, EpollHelper, EpollHelperError, EpollHelperHandler, Queue,
VirtioDevice, VirtioDeviceType, VirtioInterruptType, EPOLL_HELPER_EVENT_LAST,
};
use crate::VirtioInterrupt;
use anyhow::anyhow;
use block_util::{build_disk_image_id, Request, RequestType, VirtioBlockConfig};
use libc::EFD_NONBLOCK;
use std::collections::HashMap;
use std::fs::File;
use std::io::{self, Read, Seek, SeekFrom, Write};
use std::num::Wrapping;
use std::ops::DerefMut;
use std::os::unix::io::{AsRawFd, FromRawFd};
use std::os::unix::io::AsRawFd;
use std::path::PathBuf;
use std::result;
use std::sync::atomic::{AtomicBool, AtomicU64, Ordering};
@ -44,13 +43,7 @@ const SECTOR_SHIFT: u8 = 9;
pub const SECTOR_SIZE: u64 = (0x01 as u64) << SECTOR_SHIFT;
// New descriptors are pending on the virtio queue.
const QUEUE_AVAIL_EVENT: DeviceEventT = 0;
// The device has been dropped.
pub const KILL_EVENT: DeviceEventT = 1;
// Number of DeviceEventT events supported by this implementation.
pub const BLOCK_EVENTS_COUNT: usize = 2;
// The device should be paused.
const PAUSE_EVENT: DeviceEventT = 3;
const QUEUE_AVAIL_EVENT: u16 = EPOLL_HELPER_EVENT_LAST + 1;
#[derive(Debug)]
pub enum Error {
@ -95,6 +88,7 @@ struct BlockEpollHandler<T: DiskFile> {
event_idx: bool,
writeback: Arc<AtomicBool>,
counters: BlockCounters,
queue_evt: EventFd,
}
impl<T: DiskFile> BlockEpollHandler<T> {
@ -202,131 +196,57 @@ impl<T: DiskFile> BlockEpollHandler<T> {
Ok(())
}
fn run(
&mut self,
queue_evt: EventFd,
paused: Arc<AtomicBool>,
) -> result::Result<(), DeviceError> {
// Create the epoll file descriptor
let epoll_fd = epoll::create(true).map_err(DeviceError::EpollCreateFd)?;
// Use 'File' to enforce closing on 'epoll_fd'
let epoll_file = unsafe { File::from_raw_fd(epoll_fd) };
fn run(&mut self, paused: Arc<AtomicBool>) -> result::Result<(), EpollHelperError> {
let mut helper = EpollHelper::new(&self.kill_evt, &self.pause_evt)?;
helper.add_event(self.queue_evt.as_raw_fd(), QUEUE_AVAIL_EVENT)?;
helper.run(paused, self)?;
// Add events
epoll::ctl(
epoll_file.as_raw_fd(),
epoll::ControlOptions::EPOLL_CTL_ADD,
queue_evt.as_raw_fd(),
epoll::Event::new(epoll::Events::EPOLLIN, u64::from(QUEUE_AVAIL_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)?;
Ok(())
}
}
const EPOLL_EVENTS_LEN: usize = 100;
let mut events = vec![epoll::Event::new(epoll::Events::empty(), 0); EPOLL_EVENTS_LEN];
impl<T: DiskFile> EpollHelperHandler for BlockEpollHandler<T> {
fn handle_event(&mut self, _helper: &mut EpollHelper, event: u16) -> bool {
match event {
QUEUE_AVAIL_EVENT => {
if let Err(e) = self.queue_evt.read() {
error!("Failed to get queue event: {:?}", e);
return true;
} else if self.event_idx {
// vm-virtio's Queue implementation only checks avail_index
// once, so to properly support EVENT_IDX we need to keep
// calling process_queue() until it stops finding new
// requests on the queue.
loop {
if self.process_queue() {
self.queue.update_avail_event(&self.mem.memory());
// Before jumping into the epoll loop, check if the device is expected
// to be in a paused state. This is helpful for the restore code path
// as the device thread should not start processing anything before the
// device has been resumed.
while paused.load(Ordering::SeqCst) {
thread::park();
}
'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 {
QUEUE_AVAIL_EVENT => {
if let Err(e) = queue_evt.read() {
error!("Failed to get queue event: {:?}", e);
break 'epoll;
} else if self.event_idx {
// vm-virtio's Queue implementation only checks avail_index
// once, so to properly support EVENT_IDX we need to keep
// calling process_queue() until it stops finding new
// requests on the queue.
loop {
if self.process_queue() {
self.queue.update_avail_event(&self.mem.memory());
if self.queue.needs_notification(
&self.mem.memory(),
self.queue.next_used,
) {
if let Err(e) = self.signal_used_queue() {
error!("Failed to signal used queue: {:?}", e);
break 'epoll;
}
}
} else {
break;
if self
.queue
.needs_notification(&self.mem.memory(), self.queue.next_used)
{
if let Err(e) = self.signal_used_queue() {
error!("Failed to signal used queue: {:?}", e);
return true;
}
}
} else if self.process_queue() {
if let Err(e) = self.signal_used_queue() {
error!("Failed to signal used queue: {:?}", e);
break 'epoll;
}
} else {
break;
}
}
KILL_EVENT => {
debug!("KILL_EVENT received, stopping epoll loop");
break 'epoll;
}
PAUSE_EVENT => {
debug!("PAUSE_EVENT received, pausing virtio-block 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-block");
} else if self.process_queue() {
if let Err(e) = self.signal_used_queue() {
error!("Failed to signal used queue: {:?}", e);
return true;
}
}
}
_ => {
error!("Unexpected event: {}", event);
return true;
}
}
Ok(())
false
}
}
@ -342,7 +262,7 @@ pub struct Block<T: DiskFile> {
config: VirtioBlockConfig,
queue_evts: Option<Vec<EventFd>>,
interrupt_cb: Option<Arc<dyn VirtioInterrupt>>,
epoll_threads: Option<Vec<thread::JoinHandle<result::Result<(), DeviceError>>>>,
epoll_threads: Option<Vec<thread::JoinHandle<result::Result<(), EpollHelperError>>>>,
pause_evt: Option<EventFd>,
paused: Arc<AtomicBool>,
queue_size: Vec<u16>,
@ -588,6 +508,7 @@ impl<T: 'static + DiskFile + Send> VirtioDevice for Block<T> {
let mut epoll_threads = Vec::new();
for _ in 0..self.queue_size.len() {
let queue_evt = queue_evts.remove(0);
let mut handler = BlockEpollHandler {
queue: queues.remove(0),
mem: mem.clone(),
@ -600,15 +521,15 @@ impl<T: 'static + DiskFile + Send> VirtioDevice for Block<T> {
event_idx,
writeback: self.writeback.clone(),
counters: self.counters.clone(),
queue_evt,
};
handler.queue.set_event_idx(event_idx);
let queue_evt = queue_evts.remove(0);
let paused = self.paused.clone();
thread::Builder::new()
.name("virtio_blk".to_string())
.spawn(move || handler.run(queue_evt, paused))
.spawn(move || handler.run(paused))
.map(|thread| epoll_threads.push(thread))
.map_err(|e| {
error!("failed to clone the virtio-blk epoll thread: {}", e);