mirror of
https://github.com/cloud-hypervisor/cloud-hypervisor.git
synced 2025-02-21 19:02:30 +00:00
virtio-devices: console: Port to EpollHelper
Migrate 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
bdef54ead6
commit
d9992b5b5f
@ -3,8 +3,9 @@
|
||||
|
||||
use super::Error as DeviceError;
|
||||
use super::{
|
||||
ActivateError, ActivateResult, DeviceEventT, Queue, VirtioDevice, VirtioDeviceType,
|
||||
VirtioInterruptType, VIRTIO_F_IOMMU_PLATFORM, VIRTIO_F_VERSION_1,
|
||||
ActivateError, ActivateResult, EpollHelper, EpollHelperError, EpollHelperHandler, Queue,
|
||||
VirtioDevice, VirtioDeviceType, VirtioInterruptType, EPOLL_HELPER_EVENT_LAST,
|
||||
VIRTIO_F_IOMMU_PLATFORM, VIRTIO_F_VERSION_1,
|
||||
};
|
||||
use crate::seccomp_filters::{get_seccomp_filter, Thread};
|
||||
use crate::VirtioInterrupt;
|
||||
@ -14,11 +15,10 @@ use seccomp::{SeccompAction, SeccompFilter};
|
||||
use serde::ser::{Serialize, SerializeStruct, Serializer};
|
||||
use std::cmp;
|
||||
use std::collections::VecDeque;
|
||||
use std::fs::File;
|
||||
use std::io;
|
||||
use std::io::Write;
|
||||
use std::ops::DerefMut;
|
||||
use std::os::unix::io::{AsRawFd, FromRawFd};
|
||||
use std::os::unix::io::AsRawFd;
|
||||
use std::result;
|
||||
use std::sync::atomic::{AtomicBool, AtomicU64, Ordering};
|
||||
use std::sync::{Arc, Mutex};
|
||||
@ -35,16 +35,12 @@ const NUM_QUEUES: usize = 2;
|
||||
const QUEUE_SIZES: &[u16] = &[QUEUE_SIZE; NUM_QUEUES];
|
||||
|
||||
// New descriptors are pending on the virtio queue.
|
||||
const INPUT_QUEUE_EVENT: DeviceEventT = 0;
|
||||
const OUTPUT_QUEUE_EVENT: DeviceEventT = 1;
|
||||
const INPUT_QUEUE_EVENT: u16 = EPOLL_HELPER_EVENT_LAST + 1;
|
||||
const OUTPUT_QUEUE_EVENT: u16 = EPOLL_HELPER_EVENT_LAST + 2;
|
||||
// Some input from the VMM is ready to be injected into the VM.
|
||||
const INPUT_EVENT: DeviceEventT = 2;
|
||||
// The device has been dropped.
|
||||
const KILL_EVENT: DeviceEventT = 3;
|
||||
const INPUT_EVENT: u16 = EPOLL_HELPER_EVENT_LAST + 3;
|
||||
// Console configuration change event is triggered.
|
||||
const CONFIG_EVENT: DeviceEventT = 4;
|
||||
// The device should be paused.
|
||||
const PAUSE_EVENT: DeviceEventT = 5;
|
||||
const CONFIG_EVENT: u16 = EPOLL_HELPER_EVENT_LAST + 4;
|
||||
|
||||
//Console size feature bit
|
||||
const VIRTIO_CONSOLE_F_SIZE: u64 = 0;
|
||||
@ -184,158 +180,69 @@ impl ConsoleEpollHandler {
|
||||
})
|
||||
}
|
||||
|
||||
fn run(&mut self, 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.input_queue_evt.as_raw_fd(), INPUT_QUEUE_EVENT)?;
|
||||
helper.add_event(self.output_queue_evt.as_raw_fd(), OUTPUT_QUEUE_EVENT)?;
|
||||
helper.add_event(self.input_evt.as_raw_fd(), INPUT_EVENT)?;
|
||||
helper.add_event(self.config_evt.as_raw_fd(), CONFIG_EVENT)?;
|
||||
helper.run(paused, self)?;
|
||||
|
||||
// Add events
|
||||
epoll::ctl(
|
||||
epoll_file.as_raw_fd(),
|
||||
epoll::ControlOptions::EPOLL_CTL_ADD,
|
||||
self.input_queue_evt.as_raw_fd(),
|
||||
epoll::Event::new(epoll::Events::EPOLLIN, u64::from(INPUT_QUEUE_EVENT)),
|
||||
)
|
||||
.map_err(DeviceError::EpollCtl)?;
|
||||
epoll::ctl(
|
||||
epoll_file.as_raw_fd(),
|
||||
epoll::ControlOptions::EPOLL_CTL_ADD,
|
||||
self.output_queue_evt.as_raw_fd(),
|
||||
epoll::Event::new(epoll::Events::EPOLLIN, u64::from(OUTPUT_QUEUE_EVENT)),
|
||||
)
|
||||
.map_err(DeviceError::EpollCtl)?;
|
||||
epoll::ctl(
|
||||
epoll_file.as_raw_fd(),
|
||||
epoll::ControlOptions::EPOLL_CTL_ADD,
|
||||
self.input_evt.as_raw_fd(),
|
||||
epoll::Event::new(epoll::Events::EPOLLIN, u64::from(INPUT_EVENT)),
|
||||
)
|
||||
.map_err(DeviceError::EpollCtl)?;
|
||||
epoll::ctl(
|
||||
epoll_file.as_raw_fd(),
|
||||
epoll::ControlOptions::EPOLL_CTL_ADD,
|
||||
self.config_evt.as_raw_fd(),
|
||||
epoll::Event::new(epoll::Events::EPOLLIN, u64::from(CONFIG_EVENT)),
|
||||
)
|
||||
.map_err(DeviceError::EpollCtl)?;
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
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;
|
||||
let mut events = vec![epoll::Event::new(epoll::Events::empty(), 0); EPOLL_EVENTS_LEN];
|
||||
|
||||
// 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 {
|
||||
INPUT_QUEUE_EVENT => {
|
||||
if let Err(e) = self.input_queue_evt.read() {
|
||||
error!("Failed to get queue event: {:?}", e);
|
||||
break 'epoll;
|
||||
} else if self.process_input_queue() {
|
||||
if let Err(e) = self.signal_used_queue() {
|
||||
error!("Failed to signal used queue: {:?}", e);
|
||||
break 'epoll;
|
||||
}
|
||||
}
|
||||
}
|
||||
OUTPUT_QUEUE_EVENT => {
|
||||
if let Err(e) = self.output_queue_evt.read() {
|
||||
error!("Failed to get queue event: {:?}", e);
|
||||
break 'epoll;
|
||||
} else {
|
||||
self.process_output_queue();
|
||||
}
|
||||
}
|
||||
INPUT_EVENT => {
|
||||
if let Err(e) = self.input_evt.read() {
|
||||
error!("Failed to get input event: {:?}", e);
|
||||
break 'epoll;
|
||||
} else if self.process_input_queue() {
|
||||
if let Err(e) = self.signal_used_queue() {
|
||||
error!("Failed to signal used queue: {:?}", e);
|
||||
break 'epoll;
|
||||
}
|
||||
}
|
||||
}
|
||||
CONFIG_EVENT => {
|
||||
if let Err(e) = self.config_evt.read() {
|
||||
error!("Failed to get config event: {:?}", e);
|
||||
break 'epoll;
|
||||
} else if let Err(e) = self
|
||||
.interrupt_cb
|
||||
.trigger(&VirtioInterruptType::Config, None)
|
||||
{
|
||||
error!("Failed to signal console driver: {:?}", e);
|
||||
}
|
||||
}
|
||||
|
||||
KILL_EVENT => {
|
||||
debug!("KILL_EVENT received, stopping epoll loop");
|
||||
break 'epoll;
|
||||
}
|
||||
PAUSE_EVENT => {
|
||||
debug!("PAUSE_EVENT received, pausing virtio-console 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-console");
|
||||
impl EpollHelperHandler for ConsoleEpollHandler {
|
||||
fn handle_event(&mut self, _helper: &mut EpollHelper, event: u16) -> bool {
|
||||
match event {
|
||||
INPUT_QUEUE_EVENT => {
|
||||
if let Err(e) = self.input_queue_evt.read() {
|
||||
error!("Failed to get queue event: {:?}", e);
|
||||
return true;
|
||||
} else if self.process_input_queue() {
|
||||
if let Err(e) = self.signal_used_queue() {
|
||||
error!("Failed to signal used queue: {:?}", e);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
OUTPUT_QUEUE_EVENT => {
|
||||
if let Err(e) = self.output_queue_evt.read() {
|
||||
error!("Failed to get queue event: {:?}", e);
|
||||
return true;
|
||||
} else {
|
||||
self.process_output_queue();
|
||||
}
|
||||
}
|
||||
INPUT_EVENT => {
|
||||
if let Err(e) = self.input_evt.read() {
|
||||
error!("Failed to get input event: {:?}", e);
|
||||
return true;
|
||||
} else if self.process_input_queue() {
|
||||
if let Err(e) = self.signal_used_queue() {
|
||||
error!("Failed to signal used queue: {:?}", e);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
CONFIG_EVENT => {
|
||||
if let Err(e) = self.config_evt.read() {
|
||||
error!("Failed to get config event: {:?}", e);
|
||||
return true;
|
||||
} else if let Err(e) = self
|
||||
.interrupt_cb
|
||||
.trigger(&VirtioInterruptType::Config, None)
|
||||
{
|
||||
error!("Failed to signal console driver: {:?}", e);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
_ => {
|
||||
error!("Unknown event for virtio-console");
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
false
|
||||
}
|
||||
}
|
||||
|
||||
@ -396,7 +303,7 @@ pub struct Console {
|
||||
out: Arc<Mutex<Box<dyn io::Write + Send + Sync + 'static>>>,
|
||||
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<()>>>,
|
||||
paused: Arc<AtomicBool>,
|
||||
seccomp_action: SeccompAction,
|
||||
}
|
||||
@ -595,10 +502,11 @@ impl VirtioDevice for Console {
|
||||
thread::Builder::new()
|
||||
.name("virtio_console".to_string())
|
||||
.spawn(move || {
|
||||
SeccompFilter::apply(virtio_console_seccomp_filter)
|
||||
.map_err(DeviceError::ApplySeccompFilter)?;
|
||||
|
||||
handler.run(paused)
|
||||
if let Err(e) = SeccompFilter::apply(virtio_console_seccomp_filter) {
|
||||
error!("Error applying seccomp filter: {:?}", e);
|
||||
} else if let Err(e) = handler.run(paused) {
|
||||
error!("Error running worker: {:?}", e);
|
||||
}
|
||||
})
|
||||
.map(|thread| epoll_threads.push(thread))
|
||||
.map_err(|e| {
|
||||
|
Loading…
x
Reference in New Issue
Block a user