mirror of
https://github.com/cloud-hypervisor/cloud-hypervisor.git
synced 2024-12-22 13:45:20 +00:00
virtio-devices: vhost-user: Port to EpollHelper
Migrate all vhost-user devices to EpollHelper so as to remove code that is duplicated between multiple virtio devices. Signed-off-by: Sebastien Boeuf <sebastien.boeuf@intel.com>
This commit is contained in:
parent
d749aa2e6b
commit
821cf3add9
@ -1,10 +1,11 @@
|
||||
// Copyright 2019 Intel Corporation. All Rights Reserved.
|
||||
// SPDX-License-Identifier: Apache-2.0
|
||||
|
||||
use super::super::{ActivateError, ActivateResult, Queue, VirtioDevice, VirtioDeviceType};
|
||||
use super::super::{
|
||||
ActivateError, ActivateResult, EpollHelperError, Queue, VirtioDevice, VirtioDeviceType,
|
||||
};
|
||||
use super::handler::*;
|
||||
use super::vu_common_ctrl::*;
|
||||
use super::Error as DeviceError;
|
||||
use super::{Error, Result};
|
||||
use crate::VirtioInterrupt;
|
||||
use block_util::VirtioBlockConfig;
|
||||
@ -41,7 +42,7 @@ pub struct Blk {
|
||||
queue_sizes: Vec<u16>,
|
||||
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>>>>,
|
||||
paused: Arc<AtomicBool>,
|
||||
}
|
||||
|
||||
|
@ -2,12 +2,11 @@
|
||||
// SPDX-License-Identifier: Apache-2.0
|
||||
|
||||
use super::vu_common_ctrl::{reset_vhost_user, setup_vhost_user, update_mem_table};
|
||||
use super::Error as DeviceError;
|
||||
use super::{Error, Result};
|
||||
use crate::vhost_user::handler::{VhostUserEpollConfig, VhostUserEpollHandler};
|
||||
use crate::{
|
||||
ActivateError, ActivateResult, Queue, UserspaceMapping, VirtioDevice, VirtioDeviceType,
|
||||
VirtioInterrupt, VirtioSharedMemoryList, VIRTIO_F_VERSION_1,
|
||||
ActivateError, ActivateResult, EpollHelperError, Queue, UserspaceMapping, VirtioDevice,
|
||||
VirtioDeviceType, VirtioInterrupt, VirtioSharedMemoryList, VIRTIO_F_VERSION_1,
|
||||
};
|
||||
use libc::{self, c_void, off64_t, pread64, pwrite64, EFD_NONBLOCK};
|
||||
use std::io;
|
||||
@ -279,7 +278,7 @@ pub struct Fs {
|
||||
slave_req_support: bool,
|
||||
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>>>>,
|
||||
paused: Arc<AtomicBool>,
|
||||
}
|
||||
|
||||
|
@ -7,17 +7,17 @@
|
||||
//
|
||||
// SPDX-License-Identifier: Apache-2.0 AND BSD-3-Clause
|
||||
|
||||
use super::super::{Queue, VirtioInterruptType};
|
||||
use super::super::{
|
||||
EpollHelper, EpollHelperError, EpollHelperHandler, Queue, VirtioInterruptType,
|
||||
EPOLL_HELPER_EVENT_LAST,
|
||||
};
|
||||
use super::{Error, Result};
|
||||
use vmm_sys_util::eventfd::EventFd;
|
||||
|
||||
use crate::VirtioInterrupt;
|
||||
use std::fs::File;
|
||||
use std::io;
|
||||
use std::os::unix::io::{AsRawFd, FromRawFd};
|
||||
use std::sync::atomic::{AtomicBool, Ordering};
|
||||
use std::os::unix::io::AsRawFd;
|
||||
use std::sync::atomic::AtomicBool;
|
||||
use std::sync::Arc;
|
||||
use std::thread;
|
||||
use vhost_rs::vhost_user::{MasterReqHandler, VhostUserMasterReqHandler};
|
||||
|
||||
/// Collection of common parameters required by vhost-user devices while
|
||||
@ -37,6 +37,8 @@ pub struct VhostUserEpollConfig<S: VhostUserMasterReqHandler> {
|
||||
|
||||
pub struct VhostUserEpollHandler<S: VhostUserMasterReqHandler> {
|
||||
vu_epoll_cfg: VhostUserEpollConfig<S>,
|
||||
queue_evt_start_idx: u16,
|
||||
slave_evt_idx: u16,
|
||||
}
|
||||
|
||||
impl<S: VhostUserMasterReqHandler> VhostUserEpollHandler<S> {
|
||||
@ -48,7 +50,14 @@ impl<S: VhostUserMasterReqHandler> VhostUserEpollHandler<S> {
|
||||
/// # Return
|
||||
/// * `VhostUserEpollHandler` - epoll handler for vhost-user based devices
|
||||
pub fn new(vu_epoll_cfg: VhostUserEpollConfig<S>) -> VhostUserEpollHandler<S> {
|
||||
VhostUserEpollHandler { vu_epoll_cfg }
|
||||
let queue_evt_start_idx = EPOLL_HELPER_EVENT_LAST + 1;
|
||||
let slave_evt_idx = queue_evt_start_idx + vu_epoll_cfg.vu_interrupt_list.len() as u16;
|
||||
|
||||
VhostUserEpollHandler {
|
||||
vu_epoll_cfg,
|
||||
queue_evt_start_idx,
|
||||
slave_evt_idx,
|
||||
}
|
||||
}
|
||||
|
||||
fn signal_used_queue(&self, queue: &Queue) -> Result<()> {
|
||||
@ -58,139 +67,59 @@ impl<S: VhostUserMasterReqHandler> VhostUserEpollHandler<S> {
|
||||
.map_err(Error::FailedSignalingUsedQueue)
|
||||
}
|
||||
|
||||
pub fn run(&mut self, paused: Arc<AtomicBool>) -> Result<()> {
|
||||
// Create the epoll file descriptor
|
||||
let epoll_fd = epoll::create(true).map_err(Error::EpollCreateFd)?;
|
||||
// Use 'File' to enforce closing on 'epoll_fd'
|
||||
let epoll_file = unsafe { File::from_raw_fd(epoll_fd) };
|
||||
pub fn run(&mut self, paused: Arc<AtomicBool>) -> std::result::Result<(), EpollHelperError> {
|
||||
let mut helper =
|
||||
EpollHelper::new(&self.vu_epoll_cfg.kill_evt, &self.vu_epoll_cfg.pause_evt)?;
|
||||
|
||||
for (index, vhost_user_interrupt) in self.vu_epoll_cfg.vu_interrupt_list.iter().enumerate()
|
||||
{
|
||||
for (i, vhost_user_interrupt) in self.vu_epoll_cfg.vu_interrupt_list.iter().enumerate() {
|
||||
if let Some(eventfd) = &vhost_user_interrupt.0 {
|
||||
// Add events
|
||||
epoll::ctl(
|
||||
epoll_file.as_raw_fd(),
|
||||
epoll::ControlOptions::EPOLL_CTL_ADD,
|
||||
eventfd.as_raw_fd(),
|
||||
epoll::Event::new(epoll::Events::EPOLLIN, index as u64),
|
||||
)
|
||||
.map_err(Error::EpollCtl)?;
|
||||
helper.add_event(eventfd.as_raw_fd(), self.queue_evt_start_idx + i as u16)?;
|
||||
}
|
||||
}
|
||||
|
||||
let kill_evt_index = self.vu_epoll_cfg.vu_interrupt_list.len();
|
||||
|
||||
epoll::ctl(
|
||||
epoll_file.as_raw_fd(),
|
||||
epoll::ControlOptions::EPOLL_CTL_ADD,
|
||||
self.vu_epoll_cfg.kill_evt.as_raw_fd(),
|
||||
epoll::Event::new(epoll::Events::EPOLLIN, kill_evt_index as u64),
|
||||
)
|
||||
.map_err(Error::EpollCtl)?;
|
||||
|
||||
let pause_evt_index = kill_evt_index + 1;
|
||||
|
||||
epoll::ctl(
|
||||
epoll_file.as_raw_fd(),
|
||||
epoll::ControlOptions::EPOLL_CTL_ADD,
|
||||
self.vu_epoll_cfg.pause_evt.as_raw_fd(),
|
||||
epoll::Event::new(epoll::Events::EPOLLIN, pause_evt_index as u64),
|
||||
)
|
||||
.map_err(Error::EpollCtl)?;
|
||||
|
||||
let mut index = pause_evt_index;
|
||||
|
||||
let slave_evt_index = if let Some(self_req_handler) = &self.vu_epoll_cfg.slave_req_handler {
|
||||
index = pause_evt_index + 1;
|
||||
epoll::ctl(
|
||||
epoll_file.as_raw_fd(),
|
||||
epoll::ControlOptions::EPOLL_CTL_ADD,
|
||||
self_req_handler.as_raw_fd(),
|
||||
epoll::Event::new(epoll::Events::EPOLLIN, index as u64),
|
||||
)
|
||||
.map_err(Error::EpollCtl)?;
|
||||
|
||||
Some(index)
|
||||
} else {
|
||||
None
|
||||
};
|
||||
|
||||
let mut events = vec![epoll::Event::new(epoll::Events::empty(), 0); index + 1];
|
||||
|
||||
// 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();
|
||||
if let Some(self_req_handler) = &self.vu_epoll_cfg.slave_req_handler {
|
||||
helper.add_event(self_req_handler.as_raw_fd(), self.slave_evt_idx)?;
|
||||
}
|
||||
|
||||
'poll: 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(Error::EpollWait(e));
|
||||
}
|
||||
};
|
||||
helper.run(paused, self)?;
|
||||
|
||||
for event in events.iter().take(num_events) {
|
||||
let ev_type = event.data as usize;
|
||||
|
||||
match ev_type {
|
||||
x if x < kill_evt_index => {
|
||||
if let Some(eventfd) = &self.vu_epoll_cfg.vu_interrupt_list[x].0 {
|
||||
eventfd.read().map_err(Error::FailedReadingQueue)?;
|
||||
if let Err(e) =
|
||||
self.signal_used_queue(&self.vu_epoll_cfg.vu_interrupt_list[x].1)
|
||||
{
|
||||
error!("Failed to signal used queue: {:?}", e);
|
||||
break 'poll;
|
||||
}
|
||||
}
|
||||
}
|
||||
x if kill_evt_index == x => {
|
||||
debug!("KILL_EVENT received, stopping epoll loop");
|
||||
break 'poll;
|
||||
}
|
||||
x if pause_evt_index == x => {
|
||||
debug!("PAUSE_EVENT received, pausing vhost-user 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.vu_epoll_cfg.pause_evt.read();
|
||||
}
|
||||
x if (slave_evt_index.is_some() && slave_evt_index.unwrap() == x) => {
|
||||
if let Some(slave_req_handler) =
|
||||
self.vu_epoll_cfg.slave_req_handler.as_mut()
|
||||
{
|
||||
slave_req_handler
|
||||
.handle_request()
|
||||
.map_err(Error::VhostUserSlaveRequest)?;
|
||||
}
|
||||
}
|
||||
_ => {
|
||||
error!("Unknown event for vhost-user");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
impl<S: VhostUserMasterReqHandler> EpollHelperHandler for VhostUserEpollHandler<S> {
|
||||
fn handle_event(&mut self, _helper: &mut EpollHelper, event: &epoll::Event) -> bool {
|
||||
let ev_type = event.data as u16;
|
||||
match ev_type {
|
||||
x if (x >= self.queue_evt_start_idx && x < self.slave_evt_idx) => {
|
||||
let idx = (x - self.queue_evt_start_idx) as usize;
|
||||
if let Some(eventfd) = &self.vu_epoll_cfg.vu_interrupt_list[idx].0 {
|
||||
if let Err(e) = eventfd.read() {
|
||||
error!("Failed to read queue: {:?}", e);
|
||||
return true;
|
||||
}
|
||||
if let Err(e) =
|
||||
self.signal_used_queue(&self.vu_epoll_cfg.vu_interrupt_list[idx].1)
|
||||
{
|
||||
error!("Failed to signal used queue: {:?}", e);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
x if x == self.slave_evt_idx => {
|
||||
if let Some(slave_req_handler) = self.vu_epoll_cfg.slave_req_handler.as_mut() {
|
||||
if let Err(e) = slave_req_handler.handle_request() {
|
||||
error!("Failed to handle vhost-user request: {:?}", e);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
_ => {
|
||||
error!("Unknown event for vhost-user");
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
false
|
||||
}
|
||||
}
|
||||
|
@ -34,14 +34,6 @@ pub enum Error {
|
||||
CloneKillEventFd(io::Error),
|
||||
/// Invalid descriptor table address.
|
||||
DescriptorTableAddress,
|
||||
/// Create Epoll eventfd failed
|
||||
EpollCreateFd(io::Error),
|
||||
/// Epoll ctl error
|
||||
EpollCtl(io::Error),
|
||||
/// Epoll wait error
|
||||
EpollWait(io::Error),
|
||||
/// Read queue failed.
|
||||
FailedReadingQueue(io::Error),
|
||||
/// Signal used queue failed.
|
||||
FailedSignalingUsedQueue(io::Error),
|
||||
/// Failed to read vhost eventfd.
|
||||
@ -88,8 +80,6 @@ pub enum Error {
|
||||
VhostIrqRead(io::Error),
|
||||
/// Failed to read vhost eventfd.
|
||||
VhostUserMemoryRegion(MmapError),
|
||||
/// Failed to handle vhost-user slave request.
|
||||
VhostUserSlaveRequest(vhost_rs::vhost_user::Error),
|
||||
/// Failed to create the master request handler from slave.
|
||||
MasterReqHandlerCreation(vhost_rs::vhost_user::Error),
|
||||
/// Set slave request fd failed.
|
||||
|
@ -5,10 +5,11 @@ use super::super::net_util::{
|
||||
build_net_config_space, CtrlVirtio, NetCtrlEpollHandler, VirtioNetConfig,
|
||||
};
|
||||
use super::super::Error as CtrlError;
|
||||
use super::super::{ActivateError, ActivateResult, Queue, VirtioDevice, VirtioDeviceType};
|
||||
use super::super::{
|
||||
ActivateError, ActivateResult, EpollHelperError, Queue, VirtioDevice, VirtioDeviceType,
|
||||
};
|
||||
use super::handler::*;
|
||||
use super::vu_common_ctrl::*;
|
||||
use super::Error as DeviceError;
|
||||
use super::{Error, Result};
|
||||
use crate::VirtioInterrupt;
|
||||
use libc::EFD_NONBLOCK;
|
||||
@ -45,7 +46,7 @@ pub struct Net {
|
||||
queue_sizes: Vec<u16>,
|
||||
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>>>>,
|
||||
ctrl_queue_epoll_thread: Option<thread::JoinHandle<result::Result<(), CtrlError>>>,
|
||||
paused: Arc<AtomicBool>,
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user