2019-07-22 18:50:56 +00:00
|
|
|
// Copyright 2019 Intel Corporation. All Rights Reserved.
|
|
|
|
// SPDX-License-Identifier: Apache-2.0
|
|
|
|
|
2019-12-31 10:49:11 +00:00
|
|
|
use super::Error as DeviceError;
|
|
|
|
use super::{
|
|
|
|
ActivateError, ActivateResult, DeviceEventT, Queue, VirtioDevice, VirtioDeviceType,
|
|
|
|
VirtioInterruptType, VIRTIO_F_IOMMU_PLATFORM, VIRTIO_F_VERSION_1,
|
|
|
|
};
|
|
|
|
use crate::VirtioInterrupt;
|
2019-07-22 18:50:56 +00:00
|
|
|
use epoll;
|
|
|
|
use libc::EFD_NONBLOCK;
|
|
|
|
use std;
|
|
|
|
use std::cmp;
|
|
|
|
use std::collections::VecDeque;
|
|
|
|
use std::io;
|
|
|
|
use std::io::Write;
|
2019-10-04 16:38:43 +00:00
|
|
|
use std::ops::DerefMut;
|
2019-07-22 18:50:56 +00:00
|
|
|
use std::os::unix::io::AsRawFd;
|
|
|
|
use std::result;
|
2019-11-19 00:42:31 +00:00
|
|
|
use std::sync::atomic::{AtomicBool, AtomicU64, Ordering};
|
2019-12-31 10:49:11 +00:00
|
|
|
use std::sync::{Arc, Mutex};
|
|
|
|
use std::thread;
|
2019-11-19 00:42:31 +00:00
|
|
|
use vm_device::{Migratable, MigratableError, Pausable, Snapshotable};
|
2020-02-11 16:22:40 +00:00
|
|
|
use vm_memory::{ByteValued, Bytes, GuestAddressSpace, GuestMemoryAtomic, GuestMemoryMmap};
|
2019-08-02 14:23:52 +00:00
|
|
|
use vmm_sys_util::eventfd::EventFd;
|
2019-07-22 18:50:56 +00:00
|
|
|
|
|
|
|
const QUEUE_SIZE: u16 = 256;
|
|
|
|
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;
|
|
|
|
// 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;
|
vm-virtio: Implement console size config feature
One of the features of the virtio console device is its size can be
configured and updated. Our first iteration of the console device
implementation is lack of this feature. As a result, it had a
default fixed size which could not be changed. This commit implements
the console config feature and lets us change the console size from
the vmm side.
During the activation of the device, vmm reads the current terminal
size, sets the console configuration accordinly, and lets the driver
know about this configuration by sending an interrupt. Later, if
someone changes the terminal size, the vmm detects the corresponding
event, updates the configuration, and sends interrupt as before. As a
result, the console device driver, in the guest, updates the console
size.
Signed-off-by: A K M Fazla Mehrab <fazla.mehrab.akm@intel.com>
2019-07-23 19:18:20 +00:00
|
|
|
// Console configuration change event is triggered.
|
|
|
|
const CONFIG_EVENT: DeviceEventT = 4;
|
2019-11-19 00:42:31 +00:00
|
|
|
// The device should be paused.
|
|
|
|
const PAUSE_EVENT: DeviceEventT = 5;
|
vm-virtio: Implement console size config feature
One of the features of the virtio console device is its size can be
configured and updated. Our first iteration of the console device
implementation is lack of this feature. As a result, it had a
default fixed size which could not be changed. This commit implements
the console config feature and lets us change the console size from
the vmm side.
During the activation of the device, vmm reads the current terminal
size, sets the console configuration accordinly, and lets the driver
know about this configuration by sending an interrupt. Later, if
someone changes the terminal size, the vmm detects the corresponding
event, updates the configuration, and sends interrupt as before. As a
result, the console device driver, in the guest, updates the console
size.
Signed-off-by: A K M Fazla Mehrab <fazla.mehrab.akm@intel.com>
2019-07-23 19:18:20 +00:00
|
|
|
|
|
|
|
//Console size feature bit
|
|
|
|
const VIRTIO_CONSOLE_F_SIZE: u64 = 0;
|
|
|
|
|
|
|
|
#[derive(Copy, Clone, Debug, Default)]
|
|
|
|
#[repr(C)]
|
|
|
|
pub struct VirtioConsoleConfig {
|
|
|
|
cols: u16,
|
|
|
|
rows: u16,
|
|
|
|
max_nr_ports: u32,
|
|
|
|
emerg_wr: u32,
|
|
|
|
}
|
|
|
|
|
|
|
|
// Safe because it only has data and has no implicit padding.
|
|
|
|
unsafe impl ByteValued for VirtioConsoleConfig {}
|
2019-07-22 18:50:56 +00:00
|
|
|
|
|
|
|
struct ConsoleEpollHandler {
|
|
|
|
queues: Vec<Queue>,
|
2020-02-11 16:22:40 +00:00
|
|
|
mem: GuestMemoryAtomic<GuestMemoryMmap>,
|
2020-01-13 17:52:19 +00:00
|
|
|
interrupt_cb: Arc<dyn VirtioInterrupt>,
|
2019-07-22 18:50:56 +00:00
|
|
|
in_buffer: Arc<Mutex<VecDeque<u8>>>,
|
2019-10-04 16:38:43 +00:00
|
|
|
out: Arc<Mutex<Box<dyn io::Write + Send + Sync + 'static>>>,
|
2019-07-22 18:50:56 +00:00
|
|
|
input_queue_evt: EventFd,
|
|
|
|
output_queue_evt: EventFd,
|
|
|
|
input_evt: EventFd,
|
vm-virtio: Implement console size config feature
One of the features of the virtio console device is its size can be
configured and updated. Our first iteration of the console device
implementation is lack of this feature. As a result, it had a
default fixed size which could not be changed. This commit implements
the console config feature and lets us change the console size from
the vmm side.
During the activation of the device, vmm reads the current terminal
size, sets the console configuration accordinly, and lets the driver
know about this configuration by sending an interrupt. Later, if
someone changes the terminal size, the vmm detects the corresponding
event, updates the configuration, and sends interrupt as before. As a
result, the console device driver, in the guest, updates the console
size.
Signed-off-by: A K M Fazla Mehrab <fazla.mehrab.akm@intel.com>
2019-07-23 19:18:20 +00:00
|
|
|
config_evt: EventFd,
|
2019-07-22 18:50:56 +00:00
|
|
|
kill_evt: EventFd,
|
2019-11-19 00:42:31 +00:00
|
|
|
pause_evt: EventFd,
|
2019-07-22 18:50:56 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
impl ConsoleEpollHandler {
|
|
|
|
/*
|
|
|
|
* Each port of virtio console device has one receive
|
|
|
|
* queue. One or more empty buffers are placed by the
|
|
|
|
* dirver in the receive queue for incoming data. Here,
|
|
|
|
* we place the input data to these empty buffers.
|
|
|
|
*/
|
|
|
|
fn process_input_queue(&mut self) -> bool {
|
|
|
|
let mut in_buffer = self.in_buffer.lock().unwrap();
|
|
|
|
let recv_queue = &mut self.queues[0]; //receiveq
|
|
|
|
let mut used_desc_heads = [(0, 0); QUEUE_SIZE as usize];
|
|
|
|
let mut used_count = 0;
|
2020-01-08 10:55:30 +00:00
|
|
|
|
|
|
|
if in_buffer.is_empty() {
|
|
|
|
return false;
|
|
|
|
}
|
2019-07-22 18:50:56 +00:00
|
|
|
|
2020-02-11 16:22:40 +00:00
|
|
|
let mem = self.mem.memory();
|
2019-08-20 22:43:23 +00:00
|
|
|
for avail_desc in recv_queue.iter(&mem) {
|
2020-01-08 10:55:30 +00:00
|
|
|
let len = cmp::min(avail_desc.len as u32, in_buffer.len() as u32);
|
|
|
|
let source_slice = in_buffer.drain(..len as usize).collect::<Vec<u8>>();
|
|
|
|
if let Err(e) = mem.write_slice(&source_slice[..], avail_desc.addr) {
|
|
|
|
error!("Failed to write slice: {:?}", e);
|
2020-01-08 13:42:12 +00:00
|
|
|
recv_queue.go_to_previous_position();
|
2020-01-08 10:55:30 +00:00
|
|
|
break;
|
2019-07-22 18:50:56 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
used_desc_heads[used_count] = (avail_desc.index, len);
|
|
|
|
used_count += 1;
|
|
|
|
|
2020-01-08 10:55:30 +00:00
|
|
|
if in_buffer.is_empty() {
|
2019-07-22 18:50:56 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
for &(desc_index, len) in &used_desc_heads[..used_count] {
|
2019-08-20 22:43:23 +00:00
|
|
|
recv_queue.add_used(&mem, desc_index, len);
|
2019-07-22 18:50:56 +00:00
|
|
|
}
|
2020-01-08 10:55:30 +00:00
|
|
|
|
2019-07-22 18:50:56 +00:00
|
|
|
used_count > 0
|
|
|
|
}
|
|
|
|
|
|
|
|
/*
|
|
|
|
* Each port of virtio console device has one transmit
|
|
|
|
* queue. For outgoing data, characters are placed in
|
|
|
|
* the transmit queue by the driver. Therefore, here
|
|
|
|
* we read data from the transmit queue and flush them
|
|
|
|
* to the referenced address.
|
|
|
|
*/
|
|
|
|
fn process_output_queue(&mut self) -> bool {
|
|
|
|
let trans_queue = &mut self.queues[1]; //transmitq
|
|
|
|
let mut used_desc_heads = [(0, 0); QUEUE_SIZE as usize];
|
|
|
|
let mut used_count = 0;
|
|
|
|
|
2020-02-11 16:22:40 +00:00
|
|
|
let mem = self.mem.memory();
|
2019-08-20 22:43:23 +00:00
|
|
|
for avail_desc in trans_queue.iter(&mem) {
|
2019-07-22 18:50:56 +00:00
|
|
|
let len;
|
2019-10-04 16:38:43 +00:00
|
|
|
let mut out = self.out.lock().unwrap();
|
|
|
|
let _ = mem.write_to(
|
|
|
|
avail_desc.addr,
|
|
|
|
&mut out.deref_mut(),
|
|
|
|
avail_desc.len as usize,
|
|
|
|
);
|
|
|
|
let _ = out.flush();
|
2019-07-22 18:50:56 +00:00
|
|
|
|
|
|
|
len = avail_desc.len;
|
|
|
|
used_desc_heads[used_count] = (avail_desc.index, len);
|
|
|
|
used_count += 1;
|
|
|
|
}
|
|
|
|
|
|
|
|
for &(desc_index, len) in &used_desc_heads[..used_count] {
|
2019-08-20 22:43:23 +00:00
|
|
|
trans_queue.add_used(&mem, desc_index, len);
|
2019-07-22 18:50:56 +00:00
|
|
|
}
|
|
|
|
used_count > 0
|
|
|
|
}
|
|
|
|
|
|
|
|
fn signal_used_queue(&self) -> result::Result<(), DeviceError> {
|
2020-01-13 17:52:19 +00:00
|
|
|
self.interrupt_cb
|
|
|
|
.trigger(&VirtioInterruptType::Queue, Some(&self.queues[0]))
|
|
|
|
.map_err(|e| {
|
|
|
|
error!("Failed to signal used queue: {:?}", e);
|
|
|
|
DeviceError::FailedSignalingUsedQueue(e)
|
|
|
|
})
|
2019-07-22 18:50:56 +00:00
|
|
|
}
|
|
|
|
|
2019-11-19 00:42:31 +00:00
|
|
|
fn run(&mut self, paused: Arc<AtomicBool>) -> result::Result<(), DeviceError> {
|
2019-07-22 18:50:56 +00:00
|
|
|
// Create the epoll file descriptor
|
|
|
|
let epoll_fd = epoll::create(true).map_err(DeviceError::EpollCreateFd)?;
|
|
|
|
|
|
|
|
// Add events
|
|
|
|
epoll::ctl(
|
|
|
|
epoll_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_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_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)?;
|
vm-virtio: Implement console size config feature
One of the features of the virtio console device is its size can be
configured and updated. Our first iteration of the console device
implementation is lack of this feature. As a result, it had a
default fixed size which could not be changed. This commit implements
the console config feature and lets us change the console size from
the vmm side.
During the activation of the device, vmm reads the current terminal
size, sets the console configuration accordinly, and lets the driver
know about this configuration by sending an interrupt. Later, if
someone changes the terminal size, the vmm detects the corresponding
event, updates the configuration, and sends interrupt as before. As a
result, the console device driver, in the guest, updates the console
size.
Signed-off-by: A K M Fazla Mehrab <fazla.mehrab.akm@intel.com>
2019-07-23 19:18:20 +00:00
|
|
|
epoll::ctl(
|
|
|
|
epoll_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)?;
|
|
|
|
|
2019-07-22 18:50:56 +00:00
|
|
|
epoll::ctl(
|
|
|
|
epoll_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)?;
|
2019-11-19 00:42:31 +00:00
|
|
|
epoll::ctl(
|
|
|
|
epoll_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)?;
|
2019-07-22 18:50:56 +00:00
|
|
|
|
|
|
|
const EPOLL_EVENTS_LEN: usize = 100;
|
|
|
|
let mut events = vec![epoll::Event::new(epoll::Events::empty(), 0); EPOLL_EVENTS_LEN];
|
|
|
|
|
|
|
|
'epoll: loop {
|
2019-08-01 20:08:47 +00:00
|
|
|
let num_events = match epoll::wait(epoll_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));
|
|
|
|
}
|
|
|
|
};
|
2019-07-22 18:50:56 +00:00
|
|
|
|
|
|
|
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;
|
2020-01-08 13:53:00 +00:00
|
|
|
} else if self.process_input_queue() {
|
|
|
|
if let Err(e) = self.signal_used_queue() {
|
|
|
|
error!("Failed to signal used queue: {:?}", e);
|
|
|
|
break 'epoll;
|
|
|
|
}
|
2019-07-22 18:50:56 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
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;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
vm-virtio: Implement console size config feature
One of the features of the virtio console device is its size can be
configured and updated. Our first iteration of the console device
implementation is lack of this feature. As a result, it had a
default fixed size which could not be changed. This commit implements
the console config feature and lets us change the console size from
the vmm side.
During the activation of the device, vmm reads the current terminal
size, sets the console configuration accordinly, and lets the driver
know about this configuration by sending an interrupt. Later, if
someone changes the terminal size, the vmm detects the corresponding
event, updates the configuration, and sends interrupt as before. As a
result, the console device driver, in the guest, updates the console
size.
Signed-off-by: A K M Fazla Mehrab <fazla.mehrab.akm@intel.com>
2019-07-23 19:18:20 +00:00
|
|
|
CONFIG_EVENT => {
|
|
|
|
if let Err(e) = self.config_evt.read() {
|
|
|
|
error!("Failed to get config event: {:?}", e);
|
|
|
|
break 'epoll;
|
2020-01-13 17:52:19 +00:00
|
|
|
} else if let Err(e) = self
|
|
|
|
.interrupt_cb
|
|
|
|
.trigger(&VirtioInterruptType::Config, None)
|
vm-virtio: Implement console size config feature
One of the features of the virtio console device is its size can be
configured and updated. Our first iteration of the console device
implementation is lack of this feature. As a result, it had a
default fixed size which could not be changed. This commit implements
the console config feature and lets us change the console size from
the vmm side.
During the activation of the device, vmm reads the current terminal
size, sets the console configuration accordinly, and lets the driver
know about this configuration by sending an interrupt. Later, if
someone changes the terminal size, the vmm detects the corresponding
event, updates the configuration, and sends interrupt as before. As a
result, the console device driver, in the guest, updates the console
size.
Signed-off-by: A K M Fazla Mehrab <fazla.mehrab.akm@intel.com>
2019-07-23 19:18:20 +00:00
|
|
|
{
|
|
|
|
error!("Failed to signal console driver: {:?}", e);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-07-22 18:50:56 +00:00
|
|
|
KILL_EVENT => {
|
|
|
|
debug!("KILL_EVENT received, stopping epoll loop");
|
|
|
|
break 'epoll;
|
|
|
|
}
|
2019-11-19 00:42:31 +00:00
|
|
|
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();
|
|
|
|
}
|
|
|
|
}
|
2019-07-22 18:50:56 +00:00
|
|
|
_ => {
|
|
|
|
error!("Unknown event for virtio-console");
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
Ok(())
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Input device.
|
|
|
|
pub struct ConsoleInput {
|
|
|
|
input_evt: EventFd,
|
vm-virtio: Implement console size config feature
One of the features of the virtio console device is its size can be
configured and updated. Our first iteration of the console device
implementation is lack of this feature. As a result, it had a
default fixed size which could not be changed. This commit implements
the console config feature and lets us change the console size from
the vmm side.
During the activation of the device, vmm reads the current terminal
size, sets the console configuration accordinly, and lets the driver
know about this configuration by sending an interrupt. Later, if
someone changes the terminal size, the vmm detects the corresponding
event, updates the configuration, and sends interrupt as before. As a
result, the console device driver, in the guest, updates the console
size.
Signed-off-by: A K M Fazla Mehrab <fazla.mehrab.akm@intel.com>
2019-07-23 19:18:20 +00:00
|
|
|
config_evt: EventFd,
|
2019-07-22 18:50:56 +00:00
|
|
|
in_buffer: Arc<Mutex<VecDeque<u8>>>,
|
vm-virtio: Implement console size config feature
One of the features of the virtio console device is its size can be
configured and updated. Our first iteration of the console device
implementation is lack of this feature. As a result, it had a
default fixed size which could not be changed. This commit implements
the console config feature and lets us change the console size from
the vmm side.
During the activation of the device, vmm reads the current terminal
size, sets the console configuration accordinly, and lets the driver
know about this configuration by sending an interrupt. Later, if
someone changes the terminal size, the vmm detects the corresponding
event, updates the configuration, and sends interrupt as before. As a
result, the console device driver, in the guest, updates the console
size.
Signed-off-by: A K M Fazla Mehrab <fazla.mehrab.akm@intel.com>
2019-07-23 19:18:20 +00:00
|
|
|
config: Arc<Mutex<VirtioConsoleConfig>>,
|
|
|
|
acked_features: AtomicU64,
|
2019-07-22 18:50:56 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
impl ConsoleInput {
|
|
|
|
pub fn queue_input_bytes(&self, input: &[u8]) {
|
|
|
|
let mut in_buffer = self.in_buffer.lock().unwrap();
|
|
|
|
in_buffer.extend(input);
|
|
|
|
let _ = self.input_evt.write(1);
|
|
|
|
}
|
vm-virtio: Implement console size config feature
One of the features of the virtio console device is its size can be
configured and updated. Our first iteration of the console device
implementation is lack of this feature. As a result, it had a
default fixed size which could not be changed. This commit implements
the console config feature and lets us change the console size from
the vmm side.
During the activation of the device, vmm reads the current terminal
size, sets the console configuration accordinly, and lets the driver
know about this configuration by sending an interrupt. Later, if
someone changes the terminal size, the vmm detects the corresponding
event, updates the configuration, and sends interrupt as before. As a
result, the console device driver, in the guest, updates the console
size.
Signed-off-by: A K M Fazla Mehrab <fazla.mehrab.akm@intel.com>
2019-07-23 19:18:20 +00:00
|
|
|
|
|
|
|
pub fn update_console_size(&self, cols: u16, rows: u16) {
|
|
|
|
if self
|
|
|
|
.acked_features
|
|
|
|
.fetch_and(1u64 << VIRTIO_CONSOLE_F_SIZE, Ordering::SeqCst)
|
|
|
|
!= 0
|
|
|
|
{
|
|
|
|
self.config.lock().unwrap().update_console_size(cols, rows);
|
|
|
|
//Send the interrupt to the driver
|
|
|
|
let _ = self.config_evt.write(1);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
impl VirtioConsoleConfig {
|
|
|
|
pub fn new(cols: u16, rows: u16) -> Self {
|
|
|
|
VirtioConsoleConfig {
|
|
|
|
cols,
|
|
|
|
rows,
|
|
|
|
max_nr_ports: 1u32,
|
|
|
|
emerg_wr: 0u32,
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
pub fn update_console_size(&mut self, cols: u16, rows: u16) {
|
|
|
|
self.cols = cols;
|
|
|
|
self.rows = rows;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Virtio device for exposing console to the guest OS through virtio.
|
|
|
|
pub struct Console {
|
|
|
|
kill_evt: Option<EventFd>,
|
2019-11-19 00:42:31 +00:00
|
|
|
pause_evt: Option<EventFd>,
|
vm-virtio: Implement console size config feature
One of the features of the virtio console device is its size can be
configured and updated. Our first iteration of the console device
implementation is lack of this feature. As a result, it had a
default fixed size which could not be changed. This commit implements
the console config feature and lets us change the console size from
the vmm side.
During the activation of the device, vmm reads the current terminal
size, sets the console configuration accordinly, and lets the driver
know about this configuration by sending an interrupt. Later, if
someone changes the terminal size, the vmm detects the corresponding
event, updates the configuration, and sends interrupt as before. As a
result, the console device driver, in the guest, updates the console
size.
Signed-off-by: A K M Fazla Mehrab <fazla.mehrab.akm@intel.com>
2019-07-23 19:18:20 +00:00
|
|
|
avail_features: u64,
|
|
|
|
acked_features: u64,
|
|
|
|
config: Arc<Mutex<VirtioConsoleConfig>>,
|
|
|
|
input: Arc<ConsoleInput>,
|
2019-10-04 16:38:43 +00:00
|
|
|
out: Arc<Mutex<Box<dyn io::Write + Send + Sync + 'static>>>,
|
|
|
|
queue_evts: Option<Vec<EventFd>>,
|
2020-01-13 17:52:19 +00:00
|
|
|
interrupt_cb: Option<Arc<dyn VirtioInterrupt>>,
|
2020-01-27 13:14:56 +00:00
|
|
|
epoll_threads: Option<Vec<thread::JoinHandle<result::Result<(), DeviceError>>>>,
|
2019-11-19 00:42:31 +00:00
|
|
|
paused: Arc<AtomicBool>,
|
2019-07-22 18:50:56 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
impl Console {
|
|
|
|
/// Create a new virtio console device that gets random data from /dev/urandom.
|
vm-virtio: Implement console size config feature
One of the features of the virtio console device is its size can be
configured and updated. Our first iteration of the console device
implementation is lack of this feature. As a result, it had a
default fixed size which could not be changed. This commit implements
the console config feature and lets us change the console size from
the vmm side.
During the activation of the device, vmm reads the current terminal
size, sets the console configuration accordinly, and lets the driver
know about this configuration by sending an interrupt. Later, if
someone changes the terminal size, the vmm detects the corresponding
event, updates the configuration, and sends interrupt as before. As a
result, the console device driver, in the guest, updates the console
size.
Signed-off-by: A K M Fazla Mehrab <fazla.mehrab.akm@intel.com>
2019-07-23 19:18:20 +00:00
|
|
|
pub fn new(
|
2019-10-04 16:38:43 +00:00
|
|
|
out: Box<dyn io::Write + Send + Sync + 'static>,
|
vm-virtio: Implement console size config feature
One of the features of the virtio console device is its size can be
configured and updated. Our first iteration of the console device
implementation is lack of this feature. As a result, it had a
default fixed size which could not be changed. This commit implements
the console config feature and lets us change the console size from
the vmm side.
During the activation of the device, vmm reads the current terminal
size, sets the console configuration accordinly, and lets the driver
know about this configuration by sending an interrupt. Later, if
someone changes the terminal size, the vmm detects the corresponding
event, updates the configuration, and sends interrupt as before. As a
result, the console device driver, in the guest, updates the console
size.
Signed-off-by: A K M Fazla Mehrab <fazla.mehrab.akm@intel.com>
2019-07-23 19:18:20 +00:00
|
|
|
cols: u16,
|
|
|
|
rows: u16,
|
2019-10-04 17:32:26 +00:00
|
|
|
iommu: bool,
|
vm-virtio: Implement console size config feature
One of the features of the virtio console device is its size can be
configured and updated. Our first iteration of the console device
implementation is lack of this feature. As a result, it had a
default fixed size which could not be changed. This commit implements
the console config feature and lets us change the console size from
the vmm side.
During the activation of the device, vmm reads the current terminal
size, sets the console configuration accordinly, and lets the driver
know about this configuration by sending an interrupt. Later, if
someone changes the terminal size, the vmm detects the corresponding
event, updates the configuration, and sends interrupt as before. As a
result, the console device driver, in the guest, updates the console
size.
Signed-off-by: A K M Fazla Mehrab <fazla.mehrab.akm@intel.com>
2019-07-23 19:18:20 +00:00
|
|
|
) -> io::Result<(Console, Arc<ConsoleInput>)> {
|
2019-10-04 17:32:26 +00:00
|
|
|
let mut avail_features = 1u64 << VIRTIO_F_VERSION_1 | 1u64 << VIRTIO_CONSOLE_F_SIZE;
|
|
|
|
|
|
|
|
if iommu {
|
|
|
|
avail_features |= 1u64 << VIRTIO_F_IOMMU_PLATFORM;
|
|
|
|
}
|
2019-07-22 18:50:56 +00:00
|
|
|
|
|
|
|
let input_evt = EventFd::new(EFD_NONBLOCK).unwrap();
|
vm-virtio: Implement console size config feature
One of the features of the virtio console device is its size can be
configured and updated. Our first iteration of the console device
implementation is lack of this feature. As a result, it had a
default fixed size which could not be changed. This commit implements
the console config feature and lets us change the console size from
the vmm side.
During the activation of the device, vmm reads the current terminal
size, sets the console configuration accordinly, and lets the driver
know about this configuration by sending an interrupt. Later, if
someone changes the terminal size, the vmm detects the corresponding
event, updates the configuration, and sends interrupt as before. As a
result, the console device driver, in the guest, updates the console
size.
Signed-off-by: A K M Fazla Mehrab <fazla.mehrab.akm@intel.com>
2019-07-23 19:18:20 +00:00
|
|
|
let config_evt = EventFd::new(EFD_NONBLOCK).unwrap();
|
|
|
|
let console_config = Arc::new(Mutex::new(VirtioConsoleConfig::new(cols, rows)));
|
2019-07-22 18:50:56 +00:00
|
|
|
let console_input = Arc::new(ConsoleInput {
|
|
|
|
input_evt,
|
vm-virtio: Implement console size config feature
One of the features of the virtio console device is its size can be
configured and updated. Our first iteration of the console device
implementation is lack of this feature. As a result, it had a
default fixed size which could not be changed. This commit implements
the console config feature and lets us change the console size from
the vmm side.
During the activation of the device, vmm reads the current terminal
size, sets the console configuration accordinly, and lets the driver
know about this configuration by sending an interrupt. Later, if
someone changes the terminal size, the vmm detects the corresponding
event, updates the configuration, and sends interrupt as before. As a
result, the console device driver, in the guest, updates the console
size.
Signed-off-by: A K M Fazla Mehrab <fazla.mehrab.akm@intel.com>
2019-07-23 19:18:20 +00:00
|
|
|
config_evt,
|
2019-07-22 18:50:56 +00:00
|
|
|
in_buffer: Arc::new(Mutex::new(VecDeque::new())),
|
vm-virtio: Implement console size config feature
One of the features of the virtio console device is its size can be
configured and updated. Our first iteration of the console device
implementation is lack of this feature. As a result, it had a
default fixed size which could not be changed. This commit implements
the console config feature and lets us change the console size from
the vmm side.
During the activation of the device, vmm reads the current terminal
size, sets the console configuration accordinly, and lets the driver
know about this configuration by sending an interrupt. Later, if
someone changes the terminal size, the vmm detects the corresponding
event, updates the configuration, and sends interrupt as before. As a
result, the console device driver, in the guest, updates the console
size.
Signed-off-by: A K M Fazla Mehrab <fazla.mehrab.akm@intel.com>
2019-07-23 19:18:20 +00:00
|
|
|
config: console_config.clone(),
|
|
|
|
acked_features: AtomicU64::new(0),
|
2019-07-22 18:50:56 +00:00
|
|
|
});
|
|
|
|
|
|
|
|
Ok((
|
|
|
|
Console {
|
|
|
|
kill_evt: None,
|
2019-11-19 00:42:31 +00:00
|
|
|
pause_evt: None,
|
2019-07-22 18:50:56 +00:00
|
|
|
avail_features,
|
|
|
|
acked_features: 0u64,
|
vm-virtio: Implement console size config feature
One of the features of the virtio console device is its size can be
configured and updated. Our first iteration of the console device
implementation is lack of this feature. As a result, it had a
default fixed size which could not be changed. This commit implements
the console config feature and lets us change the console size from
the vmm side.
During the activation of the device, vmm reads the current terminal
size, sets the console configuration accordinly, and lets the driver
know about this configuration by sending an interrupt. Later, if
someone changes the terminal size, the vmm detects the corresponding
event, updates the configuration, and sends interrupt as before. As a
result, the console device driver, in the guest, updates the console
size.
Signed-off-by: A K M Fazla Mehrab <fazla.mehrab.akm@intel.com>
2019-07-23 19:18:20 +00:00
|
|
|
config: console_config,
|
2019-07-22 18:50:56 +00:00
|
|
|
input: console_input.clone(),
|
2019-10-04 16:38:43 +00:00
|
|
|
out: Arc::new(Mutex::new(out)),
|
|
|
|
queue_evts: None,
|
|
|
|
interrupt_cb: None,
|
2020-01-27 13:14:56 +00:00
|
|
|
epoll_threads: None,
|
2019-11-19 00:42:31 +00:00
|
|
|
paused: Arc::new(AtomicBool::new(false)),
|
2019-07-22 18:50:56 +00:00
|
|
|
},
|
|
|
|
console_input,
|
|
|
|
))
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
impl Drop for Console {
|
|
|
|
fn drop(&mut self) {
|
|
|
|
if let Some(kill_evt) = self.kill_evt.take() {
|
|
|
|
// Ignore the result because there is nothing we can do about it.
|
|
|
|
let _ = kill_evt.write(1);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
impl VirtioDevice for Console {
|
|
|
|
fn device_type(&self) -> u32 {
|
|
|
|
VirtioDeviceType::TYPE_CONSOLE as u32
|
|
|
|
}
|
|
|
|
|
|
|
|
fn queue_max_sizes(&self) -> &[u16] {
|
|
|
|
QUEUE_SIZES
|
|
|
|
}
|
|
|
|
|
2020-01-23 10:14:38 +00:00
|
|
|
fn features(&self) -> u64 {
|
|
|
|
self.avail_features
|
2019-07-22 18:50:56 +00:00
|
|
|
}
|
|
|
|
|
2020-01-23 10:14:38 +00:00
|
|
|
fn ack_features(&mut self, value: u64) {
|
|
|
|
let mut v = value;
|
2019-07-22 18:50:56 +00:00
|
|
|
// Check if the guest is ACK'ing a feature that we didn't claim to have.
|
|
|
|
let unrequested_features = v & !self.avail_features;
|
|
|
|
if unrequested_features != 0 {
|
|
|
|
warn!("Received acknowledge request for unknown feature.");
|
|
|
|
|
|
|
|
// Don't count these features as acked.
|
|
|
|
v &= !unrequested_features;
|
|
|
|
}
|
|
|
|
self.acked_features |= v;
|
|
|
|
}
|
|
|
|
|
vm-virtio: Implement console size config feature
One of the features of the virtio console device is its size can be
configured and updated. Our first iteration of the console device
implementation is lack of this feature. As a result, it had a
default fixed size which could not be changed. This commit implements
the console config feature and lets us change the console size from
the vmm side.
During the activation of the device, vmm reads the current terminal
size, sets the console configuration accordinly, and lets the driver
know about this configuration by sending an interrupt. Later, if
someone changes the terminal size, the vmm detects the corresponding
event, updates the configuration, and sends interrupt as before. As a
result, the console device driver, in the guest, updates the console
size.
Signed-off-by: A K M Fazla Mehrab <fazla.mehrab.akm@intel.com>
2019-07-23 19:18:20 +00:00
|
|
|
fn read_config(&self, offset: u64, mut data: &mut [u8]) {
|
|
|
|
let config = self.config.lock().unwrap();
|
|
|
|
let config_slice = config.as_slice();
|
|
|
|
let config_len = config_slice.len() as u64;
|
|
|
|
if offset >= config_len {
|
|
|
|
error!("Failed to read config space");
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
|
|
|
if let Some(end) = offset.checked_add(data.len() as u64) {
|
|
|
|
// This write can't fail, offset and end are checked against config_len.
|
|
|
|
data.write_all(&config_slice[offset as usize..cmp::min(end, config_len) as usize])
|
|
|
|
.unwrap();
|
|
|
|
}
|
2019-07-22 18:50:56 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
fn write_config(&mut self, _offset: u64, _data: &[u8]) {
|
vm-virtio: Implement console size config feature
One of the features of the virtio console device is its size can be
configured and updated. Our first iteration of the console device
implementation is lack of this feature. As a result, it had a
default fixed size which could not be changed. This commit implements
the console config feature and lets us change the console size from
the vmm side.
During the activation of the device, vmm reads the current terminal
size, sets the console configuration accordinly, and lets the driver
know about this configuration by sending an interrupt. Later, if
someone changes the terminal size, the vmm detects the corresponding
event, updates the configuration, and sends interrupt as before. As a
result, the console device driver, in the guest, updates the console
size.
Signed-off-by: A K M Fazla Mehrab <fazla.mehrab.akm@intel.com>
2019-07-23 19:18:20 +00:00
|
|
|
warn!("No device specific configration requires write");
|
2019-07-22 18:50:56 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
fn activate(
|
|
|
|
&mut self,
|
2020-02-11 16:22:40 +00:00
|
|
|
mem: GuestMemoryAtomic<GuestMemoryMmap>,
|
2020-01-13 17:52:19 +00:00
|
|
|
interrupt_cb: Arc<dyn VirtioInterrupt>,
|
2019-07-22 18:50:56 +00:00
|
|
|
queues: Vec<Queue>,
|
|
|
|
mut queue_evts: Vec<EventFd>,
|
|
|
|
) -> ActivateResult {
|
|
|
|
if queues.len() != NUM_QUEUES || queue_evts.len() != NUM_QUEUES {
|
|
|
|
error!(
|
|
|
|
"Cannot perform activate. Expected {} queue(s), got {}",
|
|
|
|
NUM_QUEUES,
|
|
|
|
queues.len()
|
|
|
|
);
|
|
|
|
return Err(ActivateError::BadActivate);
|
|
|
|
}
|
|
|
|
|
2019-11-19 00:42:31 +00:00
|
|
|
let (self_kill_evt, kill_evt) = EventFd::new(EFD_NONBLOCK)
|
|
|
|
.and_then(|e| Ok((e.try_clone()?, e)))
|
|
|
|
.map_err(|e| {
|
|
|
|
error!("failed creating kill EventFd pair: {}", e);
|
|
|
|
ActivateError::BadActivate
|
|
|
|
})?;
|
|
|
|
|
2019-07-22 18:50:56 +00:00
|
|
|
self.kill_evt = Some(self_kill_evt);
|
|
|
|
|
2019-11-19 00:42:31 +00:00
|
|
|
let (self_pause_evt, pause_evt) = EventFd::new(EFD_NONBLOCK)
|
|
|
|
.and_then(|e| Ok((e.try_clone()?, e)))
|
|
|
|
.map_err(|e| {
|
|
|
|
error!("failed creating pause EventFd pair: {}", e);
|
|
|
|
ActivateError::BadActivate
|
|
|
|
})?;
|
|
|
|
self.pause_evt = Some(self_pause_evt);
|
|
|
|
|
2019-10-04 16:38:43 +00:00
|
|
|
// Save the interrupt EventFD as we need to return it on reset
|
|
|
|
// but clone it to pass into the thread.
|
|
|
|
self.interrupt_cb = Some(interrupt_cb.clone());
|
|
|
|
|
|
|
|
let mut tmp_queue_evts: Vec<EventFd> = Vec::new();
|
|
|
|
for queue_evt in queue_evts.iter() {
|
|
|
|
// Save the queue EventFD as we need to return it on reset
|
|
|
|
// but clone it to pass into the thread.
|
|
|
|
tmp_queue_evts.push(queue_evt.try_clone().map_err(|e| {
|
|
|
|
error!("failed to clone queue EventFd: {}", e);
|
|
|
|
ActivateError::BadActivate
|
|
|
|
})?);
|
|
|
|
}
|
|
|
|
self.queue_evts = Some(tmp_queue_evts);
|
|
|
|
|
vm-virtio: Implement console size config feature
One of the features of the virtio console device is its size can be
configured and updated. Our first iteration of the console device
implementation is lack of this feature. As a result, it had a
default fixed size which could not be changed. This commit implements
the console config feature and lets us change the console size from
the vmm side.
During the activation of the device, vmm reads the current terminal
size, sets the console configuration accordinly, and lets the driver
know about this configuration by sending an interrupt. Later, if
someone changes the terminal size, the vmm detects the corresponding
event, updates the configuration, and sends interrupt as before. As a
result, the console device driver, in the guest, updates the console
size.
Signed-off-by: A K M Fazla Mehrab <fazla.mehrab.akm@intel.com>
2019-07-23 19:18:20 +00:00
|
|
|
self.input
|
|
|
|
.acked_features
|
|
|
|
.store(self.acked_features, Ordering::Relaxed);
|
|
|
|
|
|
|
|
if (self.acked_features & (1u64 << VIRTIO_CONSOLE_F_SIZE)) != 0 {
|
2020-01-13 17:52:19 +00:00
|
|
|
if let Err(e) = interrupt_cb.trigger(&VirtioInterruptType::Config, None) {
|
vm-virtio: Implement console size config feature
One of the features of the virtio console device is its size can be
configured and updated. Our first iteration of the console device
implementation is lack of this feature. As a result, it had a
default fixed size which could not be changed. This commit implements
the console config feature and lets us change the console size from
the vmm side.
During the activation of the device, vmm reads the current terminal
size, sets the console configuration accordinly, and lets the driver
know about this configuration by sending an interrupt. Later, if
someone changes the terminal size, the vmm detects the corresponding
event, updates the configuration, and sends interrupt as before. As a
result, the console device driver, in the guest, updates the console
size.
Signed-off-by: A K M Fazla Mehrab <fazla.mehrab.akm@intel.com>
2019-07-23 19:18:20 +00:00
|
|
|
error!("Failed to signal console driver: {:?}", e);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-10-04 16:38:43 +00:00
|
|
|
let mut handler = ConsoleEpollHandler {
|
|
|
|
queues,
|
|
|
|
mem,
|
|
|
|
interrupt_cb,
|
|
|
|
in_buffer: self.input.in_buffer.clone(),
|
|
|
|
out: self.out.clone(),
|
|
|
|
input_queue_evt: queue_evts.remove(0),
|
|
|
|
output_queue_evt: queue_evts.remove(0),
|
|
|
|
input_evt: self.input.input_evt.try_clone().unwrap(),
|
|
|
|
config_evt: self.input.config_evt.try_clone().unwrap(),
|
|
|
|
kill_evt,
|
2019-11-19 00:42:31 +00:00
|
|
|
pause_evt,
|
2019-10-04 16:38:43 +00:00
|
|
|
};
|
2019-07-22 18:50:56 +00:00
|
|
|
|
2019-11-19 00:42:31 +00:00
|
|
|
let paused = self.paused.clone();
|
2020-01-27 12:56:05 +00:00
|
|
|
let mut epoll_threads = Vec::new();
|
2019-11-19 00:42:31 +00:00
|
|
|
thread::Builder::new()
|
2019-10-04 16:38:43 +00:00
|
|
|
.name("virtio_console".to_string())
|
2019-11-19 00:42:31 +00:00
|
|
|
.spawn(move || handler.run(paused))
|
2020-01-27 12:56:05 +00:00
|
|
|
.map(|thread| epoll_threads.push(thread))
|
2019-11-19 00:42:31 +00:00
|
|
|
.map_err(|e| {
|
|
|
|
error!("failed to clone the virtio-console epoll thread: {}", e);
|
|
|
|
ActivateError::BadActivate
|
|
|
|
})?;
|
2019-07-22 18:50:56 +00:00
|
|
|
|
2020-01-27 13:14:56 +00:00
|
|
|
self.epoll_threads = Some(epoll_threads);
|
2020-01-27 12:56:05 +00:00
|
|
|
|
2019-10-04 16:38:43 +00:00
|
|
|
Ok(())
|
|
|
|
}
|
|
|
|
|
2020-01-13 17:52:19 +00:00
|
|
|
fn reset(&mut self) -> Option<(Arc<dyn VirtioInterrupt>, Vec<EventFd>)> {
|
2019-11-19 00:42:31 +00:00
|
|
|
// We first must resume the virtio thread if it was paused.
|
|
|
|
if self.pause_evt.take().is_some() {
|
|
|
|
self.resume().ok()?;
|
|
|
|
}
|
|
|
|
|
2019-10-04 16:38:43 +00:00
|
|
|
if let Some(kill_evt) = self.kill_evt.take() {
|
|
|
|
// Ignore the result because there is nothing we can do about it.
|
|
|
|
let _ = kill_evt.write(1);
|
2019-07-22 18:50:56 +00:00
|
|
|
}
|
2019-10-04 16:38:43 +00:00
|
|
|
|
|
|
|
// Return the interrupt and queue EventFDs
|
|
|
|
Some((
|
|
|
|
self.interrupt_cb.take().unwrap(),
|
|
|
|
self.queue_evts.take().unwrap(),
|
|
|
|
))
|
2019-07-22 18:50:56 +00:00
|
|
|
}
|
|
|
|
}
|
2019-11-19 00:42:31 +00:00
|
|
|
|
|
|
|
virtio_pausable!(Console);
|
|
|
|
impl Snapshotable for Console {}
|
|
|
|
impl Migratable for Console {}
|