mirror of
https://chromium.googlesource.com/crosvm/crosvm
synced 2025-02-05 10:10:41 +00:00
crosvm: virtio-pmem device
Adds support for virtio-pmem device as an alternative for virtio-blk. Exposing disk image to guest as virtio-blk device results in both guest and host independently caching the disk I/O. Using virtio-pmem device allows to mount disk image as direct access (DAX) in the guest and thus bypass the guest cache. This will reduce memory foodprint of the VMs. BUG=None TEST=cargo test TEST=Boot patched termina kernel in crosvm; mount virtio-pmem device as DAX and run xfstests. Change-Id: I935fc8fc7527f79e5169f07ec7927e4ea4fa6027 Reviewed-on: https://chromium-review.googlesource.com/c/chromiumos/platform/crosvm/+/1605517 Tested-by: kokoro <noreply+kokoro@google.com> Reviewed-by: Zach Reizner <zachr@chromium.org> Commit-Queue: Jakub Staroń <jstaron@google.com>
This commit is contained in:
parent
127453d7ec
commit
a3411eaac1
10 changed files with 576 additions and 20 deletions
|
@ -200,6 +200,8 @@ impl arch::LinuxArch for AArch64 {
|
|||
where
|
||||
F: FnOnce(
|
||||
&GuestMemory,
|
||||
&mut Vm,
|
||||
&mut SystemAllocator,
|
||||
&EventFd,
|
||||
) -> std::result::Result<Vec<(Box<dyn PciDevice>, Option<Minijail>)>, E>,
|
||||
E: StdError + 'static,
|
||||
|
@ -233,8 +235,8 @@ impl arch::LinuxArch for AArch64 {
|
|||
|
||||
let exit_evt = EventFd::new().map_err(Error::CreateEventFd)?;
|
||||
|
||||
let pci_devices =
|
||||
create_devices(&mem, &exit_evt).map_err(|e| Error::CreateDevices(Box::new(e)))?;
|
||||
let pci_devices = create_devices(&mem, &mut vm, &mut resources, &exit_evt)
|
||||
.map_err(|e| Error::CreateDevices(Box::new(e)))?;
|
||||
let (pci, pci_irqs, pid_debug_label_map) =
|
||||
arch::generate_pci_root(pci_devices, &mut mmio_bus, &mut resources, &mut vm)
|
||||
.map_err(Error::CreatePciRoot)?;
|
||||
|
|
|
@ -83,7 +83,12 @@ pub trait LinuxArch {
|
|||
create_devices: F,
|
||||
) -> Result<RunnableLinuxVm, Self::Error>
|
||||
where
|
||||
F: FnOnce(&GuestMemory, &EventFd) -> Result<Vec<(Box<dyn PciDevice>, Option<Minijail>)>, E>,
|
||||
F: FnOnce(
|
||||
&GuestMemory,
|
||||
&mut Vm,
|
||||
&mut SystemAllocator,
|
||||
&EventFd,
|
||||
) -> Result<Vec<(Box<dyn PciDevice>, Option<Minijail>)>, E>,
|
||||
E: StdError + 'static;
|
||||
}
|
||||
|
||||
|
|
|
@ -11,6 +11,7 @@ mod gpu;
|
|||
mod input;
|
||||
mod net;
|
||||
mod p9;
|
||||
mod pmem;
|
||||
mod queue;
|
||||
mod rng;
|
||||
#[cfg(feature = "tpm")]
|
||||
|
@ -30,6 +31,7 @@ pub use self::gpu::*;
|
|||
pub use self::input::*;
|
||||
pub use self::net::*;
|
||||
pub use self::p9::*;
|
||||
pub use self::pmem::*;
|
||||
pub use self::queue::*;
|
||||
pub use self::rng::*;
|
||||
#[cfg(feature = "tpm")]
|
||||
|
@ -54,6 +56,7 @@ const TYPE_GPU: u32 = 16;
|
|||
const TYPE_9P: u32 = 9;
|
||||
const TYPE_INPUT: u32 = 18;
|
||||
const TYPE_VSOCK: u32 = 19;
|
||||
const TYPE_PMEM: u32 = 27;
|
||||
// Additional types invented by crosvm
|
||||
const TYPE_WL: u32 = 30;
|
||||
#[cfg(feature = "tpm")]
|
||||
|
|
367
devices/src/virtio/pmem.rs
Normal file
367
devices/src/virtio/pmem.rs
Normal file
|
@ -0,0 +1,367 @@
|
|||
// Copyright 2019 The Chromium OS Authors. All rights reserved.
|
||||
// Use of this source code is governed by a BSD-style license that can be
|
||||
// found in the LICENSE file.
|
||||
|
||||
use std::cmp;
|
||||
use std::fmt::{self, Display};
|
||||
use std::fs::File;
|
||||
use std::io::Write;
|
||||
use std::mem::{size_of, size_of_val};
|
||||
use std::os::unix::io::{AsRawFd, RawFd};
|
||||
use std::result;
|
||||
use std::sync::atomic::{AtomicUsize, Ordering};
|
||||
use std::sync::Arc;
|
||||
use std::thread;
|
||||
|
||||
use sys_util::Result as SysResult;
|
||||
use sys_util::{
|
||||
error, EventFd, GuestAddress, GuestMemory, GuestMemoryError, PollContext, PollToken,
|
||||
};
|
||||
|
||||
use data_model::{DataInit, Le32, Le64};
|
||||
|
||||
use super::{
|
||||
DescriptorChain, Queue, VirtioDevice, INTERRUPT_STATUS_USED_RING, TYPE_PMEM, VIRTIO_F_VERSION_1,
|
||||
};
|
||||
|
||||
const QUEUE_SIZE: u16 = 256;
|
||||
const QUEUE_SIZES: &[u16] = &[QUEUE_SIZE];
|
||||
|
||||
const VIRTIO_PMEM_REQ_TYPE_FLUSH: u32 = 0;
|
||||
const VIRTIO_PMEM_RESP_TYPE_OK: u32 = 0;
|
||||
const VIRTIO_PMEM_RESP_TYPE_EIO: u32 = 1;
|
||||
|
||||
#[derive(Copy, Clone, Debug, Default)]
|
||||
#[repr(C)]
|
||||
struct virtio_pmem_config {
|
||||
start_address: Le64,
|
||||
size: Le64,
|
||||
}
|
||||
|
||||
// Safe because it only has data and has no implicit padding.
|
||||
unsafe impl DataInit for virtio_pmem_config {}
|
||||
|
||||
#[derive(Copy, Clone, Debug, Default)]
|
||||
#[repr(C)]
|
||||
struct virtio_pmem_resp {
|
||||
status_code: Le32,
|
||||
}
|
||||
|
||||
// Safe because it only has data and has no implicit padding.
|
||||
unsafe impl DataInit for virtio_pmem_resp {}
|
||||
|
||||
#[derive(Copy, Clone, Debug, Default)]
|
||||
#[repr(C)]
|
||||
struct virtio_pmem_req {
|
||||
type_: Le32,
|
||||
}
|
||||
|
||||
// Safe because it only has data and has no implicit padding.
|
||||
unsafe impl DataInit for virtio_pmem_req {}
|
||||
|
||||
#[derive(Debug)]
|
||||
enum ParseError {
|
||||
/// Guest gave us bad memory addresses.
|
||||
GuestMemory(GuestMemoryError),
|
||||
/// Guest gave us a write only descriptor that protocol says to read from.
|
||||
UnexpectedWriteOnlyDescriptor,
|
||||
/// Guest gave us a read only descriptor that protocol says to write to.
|
||||
UnexpectedReadOnlyDescriptor,
|
||||
/// Guest gave us too few descriptors in a descriptor chain.
|
||||
DescriptorChainTooShort,
|
||||
/// Guest gave us a buffer that was too short to use.
|
||||
BufferLengthTooSmall,
|
||||
/// Guest sent us invalid request.
|
||||
InvalidRequest,
|
||||
}
|
||||
|
||||
impl Display for ParseError {
|
||||
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
|
||||
use self::ParseError::*;
|
||||
|
||||
match self {
|
||||
BufferLengthTooSmall => write!(f, "buffer length too small"),
|
||||
DescriptorChainTooShort => write!(f, "descriptor chain too short"),
|
||||
GuestMemory(e) => write!(f, "bad guest memory address: {}", e),
|
||||
InvalidRequest => write!(f, "invalid request"),
|
||||
UnexpectedReadOnlyDescriptor => write!(f, "unexpected read-only descriptor"),
|
||||
UnexpectedWriteOnlyDescriptor => write!(f, "unexpected write-only descriptor"),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
enum Request {
|
||||
Flush { status_address: GuestAddress },
|
||||
}
|
||||
|
||||
impl Request {
|
||||
fn parse(
|
||||
avail_desc: &DescriptorChain,
|
||||
memory: &GuestMemory,
|
||||
) -> result::Result<Request, ParseError> {
|
||||
// The head contains the request type which MUST be readable.
|
||||
if avail_desc.is_write_only() {
|
||||
return Err(ParseError::UnexpectedWriteOnlyDescriptor);
|
||||
}
|
||||
|
||||
if avail_desc.len as usize != size_of::<virtio_pmem_req>() {
|
||||
return Err(ParseError::InvalidRequest);
|
||||
}
|
||||
|
||||
let request: virtio_pmem_req = memory
|
||||
.read_obj_from_addr(avail_desc.addr)
|
||||
.map_err(ParseError::GuestMemory)?;
|
||||
|
||||
// Currently, there is only one virtio-pmem request, FLUSH.
|
||||
if request.type_ != VIRTIO_PMEM_REQ_TYPE_FLUSH {
|
||||
error!("unknown request type: {}", request.type_.to_native());
|
||||
return Err(ParseError::InvalidRequest);
|
||||
}
|
||||
|
||||
let status_desc = avail_desc
|
||||
.next_descriptor()
|
||||
.ok_or(ParseError::DescriptorChainTooShort)?;
|
||||
|
||||
// The status MUST always be writable
|
||||
if status_desc.is_read_only() {
|
||||
return Err(ParseError::UnexpectedReadOnlyDescriptor);
|
||||
}
|
||||
|
||||
if (status_desc.len as usize) < size_of::<virtio_pmem_resp>() {
|
||||
return Err(ParseError::BufferLengthTooSmall);
|
||||
}
|
||||
|
||||
Ok(Request::Flush {
|
||||
status_address: status_desc.addr,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
struct Worker {
|
||||
queue: Queue,
|
||||
memory: GuestMemory,
|
||||
disk_image: File,
|
||||
interrupt_status: Arc<AtomicUsize>,
|
||||
interrupt_event: EventFd,
|
||||
interrupt_resample_event: EventFd,
|
||||
}
|
||||
|
||||
impl Worker {
|
||||
fn process_queue(&mut self) -> bool {
|
||||
let mut needs_interrupt = false;
|
||||
while let Some(avail_desc) = self.queue.pop(&self.memory) {
|
||||
let len;
|
||||
match Request::parse(&avail_desc, &self.memory) {
|
||||
Ok(Request::Flush { status_address }) => {
|
||||
let status_code = match self.disk_image.sync_all() {
|
||||
Ok(()) => VIRTIO_PMEM_RESP_TYPE_OK,
|
||||
Err(e) => {
|
||||
error!("failed flushing disk image: {}", e);
|
||||
VIRTIO_PMEM_RESP_TYPE_EIO
|
||||
}
|
||||
};
|
||||
|
||||
let response = virtio_pmem_resp {
|
||||
status_code: status_code.into(),
|
||||
};
|
||||
len = match self.memory.write_obj_at_addr(response, status_address) {
|
||||
Ok(_) => size_of::<virtio_pmem_resp>() as u32,
|
||||
Err(e) => {
|
||||
error!("bad guest memory address: {}", e);
|
||||
0
|
||||
}
|
||||
}
|
||||
}
|
||||
Err(e) => {
|
||||
error!("failed processing available descriptor chain: {}", e);
|
||||
len = 0;
|
||||
}
|
||||
}
|
||||
self.queue.add_used(&self.memory, avail_desc.index, len);
|
||||
needs_interrupt = true;
|
||||
}
|
||||
|
||||
needs_interrupt
|
||||
}
|
||||
|
||||
fn signal_used_queue(&self) {
|
||||
self.interrupt_status
|
||||
.fetch_or(INTERRUPT_STATUS_USED_RING as usize, Ordering::SeqCst);
|
||||
self.interrupt_event.write(1).unwrap();
|
||||
}
|
||||
|
||||
fn run(&mut self, queue_evt: EventFd, kill_evt: EventFd) {
|
||||
#[derive(PollToken)]
|
||||
enum Token {
|
||||
QueueAvailable,
|
||||
InterruptResample,
|
||||
Kill,
|
||||
}
|
||||
|
||||
let poll_ctx: PollContext<Token> = match PollContext::new()
|
||||
.and_then(|pc| pc.add(&queue_evt, Token::QueueAvailable).and(Ok(pc)))
|
||||
.and_then(|pc| {
|
||||
pc.add(&self.interrupt_resample_event, Token::InterruptResample)
|
||||
.and(Ok(pc))
|
||||
})
|
||||
.and_then(|pc| pc.add(&kill_evt, Token::Kill).and(Ok(pc)))
|
||||
{
|
||||
Ok(pc) => pc,
|
||||
Err(e) => {
|
||||
error!("failed creating PollContext: {}", e);
|
||||
return;
|
||||
}
|
||||
};
|
||||
|
||||
'poll: loop {
|
||||
let events = match poll_ctx.wait() {
|
||||
Ok(v) => v,
|
||||
Err(e) => {
|
||||
error!("failed polling for events: {}", e);
|
||||
break;
|
||||
}
|
||||
};
|
||||
|
||||
let mut needs_interrupt = false;
|
||||
for event in events.iter_readable() {
|
||||
match event.token() {
|
||||
Token::QueueAvailable => {
|
||||
if let Err(e) = queue_evt.read() {
|
||||
error!("failed reading queue EventFd: {}", e);
|
||||
break 'poll;
|
||||
}
|
||||
needs_interrupt |= self.process_queue();
|
||||
}
|
||||
Token::InterruptResample => {
|
||||
let _ = self.interrupt_resample_event.read();
|
||||
if self.interrupt_status.load(Ordering::SeqCst) != 0 {
|
||||
self.interrupt_event.write(1).unwrap();
|
||||
}
|
||||
}
|
||||
Token::Kill => break 'poll,
|
||||
}
|
||||
}
|
||||
if needs_interrupt {
|
||||
self.signal_used_queue();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub struct Pmem {
|
||||
kill_event: Option<EventFd>,
|
||||
disk_image: Option<File>,
|
||||
mapping_address: GuestAddress,
|
||||
mapping_size: u64,
|
||||
}
|
||||
|
||||
impl Pmem {
|
||||
pub fn new(
|
||||
disk_image: File,
|
||||
mapping_address: GuestAddress,
|
||||
mapping_size: u64,
|
||||
) -> SysResult<Pmem> {
|
||||
Ok(Pmem {
|
||||
kill_event: None,
|
||||
disk_image: Some(disk_image),
|
||||
mapping_address,
|
||||
mapping_size,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
impl Drop for Pmem {
|
||||
fn drop(&mut self) {
|
||||
if let Some(kill_evt) = self.kill_event.take() {
|
||||
// Ignore the result because there is nothing we can do about it.
|
||||
let _ = kill_evt.write(1);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl VirtioDevice for Pmem {
|
||||
fn keep_fds(&self) -> Vec<RawFd> {
|
||||
if let Some(disk_image) = &self.disk_image {
|
||||
vec![disk_image.as_raw_fd()]
|
||||
} else {
|
||||
vec![]
|
||||
}
|
||||
}
|
||||
|
||||
fn device_type(&self) -> u32 {
|
||||
TYPE_PMEM
|
||||
}
|
||||
|
||||
fn queue_max_sizes(&self) -> &[u16] {
|
||||
QUEUE_SIZES
|
||||
}
|
||||
|
||||
fn features(&self) -> u64 {
|
||||
1 << VIRTIO_F_VERSION_1
|
||||
}
|
||||
|
||||
fn read_config(&self, offset: u64, mut data: &mut [u8]) {
|
||||
let config = virtio_pmem_config {
|
||||
start_address: Le64::from(self.mapping_address.offset()),
|
||||
size: Le64::from(self.mapping_size as u64),
|
||||
};
|
||||
let config_len = size_of_val(&config) as u64;
|
||||
if offset >= config_len {
|
||||
return;
|
||||
}
|
||||
|
||||
if let Some(end) = offset.checked_add(data.len() as u64) {
|
||||
let offset = offset as usize;
|
||||
let end = cmp::min(end, config_len) as usize;
|
||||
// This write can't fail, offset and end are checked against config_len.
|
||||
data.write_all(&config.as_slice()[offset..end]).unwrap();
|
||||
}
|
||||
}
|
||||
|
||||
fn activate(
|
||||
&mut self,
|
||||
memory: GuestMemory,
|
||||
interrupt_event: EventFd,
|
||||
interrupt_resample_event: EventFd,
|
||||
status: Arc<AtomicUsize>,
|
||||
mut queues: Vec<Queue>,
|
||||
mut queue_events: Vec<EventFd>,
|
||||
) {
|
||||
if queues.len() != 1 || queue_events.len() != 1 {
|
||||
return;
|
||||
}
|
||||
|
||||
let queue = queues.remove(0);
|
||||
let queue_event = queue_events.remove(0);
|
||||
|
||||
if let Some(disk_image) = self.disk_image.take() {
|
||||
let (self_kill_event, kill_event) =
|
||||
match EventFd::new().and_then(|e| Ok((e.try_clone()?, e))) {
|
||||
Ok(v) => v,
|
||||
Err(e) => {
|
||||
error!("failed creating kill EventFd pair: {}", e);
|
||||
return;
|
||||
}
|
||||
};
|
||||
self.kill_event = Some(self_kill_event);
|
||||
|
||||
let worker_result = thread::Builder::new()
|
||||
.name("virtio_pmem".to_string())
|
||||
.spawn(move || {
|
||||
let mut worker = Worker {
|
||||
memory,
|
||||
disk_image,
|
||||
queue,
|
||||
interrupt_status: status,
|
||||
interrupt_event,
|
||||
interrupt_resample_event,
|
||||
};
|
||||
worker.run(queue_event, kill_event);
|
||||
});
|
||||
if let Err(e) = worker_result {
|
||||
error!("failed to spawn virtio_pmem worker: {}", e);
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -2,6 +2,7 @@
|
|||
// Use of this source code is governed by a BSD-style license that can be
|
||||
// found in the LICENSE file.
|
||||
|
||||
use std::cmp;
|
||||
use std::collections::HashMap;
|
||||
|
||||
use crate::{Alloc, Error, Result};
|
||||
|
@ -63,18 +64,29 @@ impl AddressAllocator {
|
|||
})
|
||||
}
|
||||
|
||||
/// Allocates a range of addresses from the managed region with an optional tag.
|
||||
/// Returns allocated_address. (allocated_address, size, tag) can be retrieved
|
||||
/// through the `get` method.
|
||||
pub fn allocate(&mut self, size: u64, alloc: Alloc, tag: String) -> Result<u64> {
|
||||
/// Allocates a range of addresses from the managed region with an optional tag
|
||||
/// and minimal alignment. Returns allocated_address. (allocated_address, size, tag)
|
||||
/// can be retrieved through the `get` method.
|
||||
pub fn allocate_with_align(
|
||||
&mut self,
|
||||
size: u64,
|
||||
alloc: Alloc,
|
||||
tag: String,
|
||||
alignment: u64,
|
||||
) -> Result<u64> {
|
||||
let alignment = cmp::max(self.alignment, alignment);
|
||||
|
||||
if self.allocs.contains_key(&alloc) {
|
||||
return Err(Error::ExistingAlloc(alloc));
|
||||
}
|
||||
if size == 0 {
|
||||
return Err(Error::AllocSizeZero);
|
||||
}
|
||||
let align_adjust = if self.next_addr % self.alignment != 0 {
|
||||
self.alignment - (self.next_addr % self.alignment)
|
||||
if !alignment.is_power_of_two() {
|
||||
return Err(Error::BadAlignment);
|
||||
}
|
||||
let align_adjust = if self.next_addr % alignment != 0 {
|
||||
alignment - (self.next_addr % alignment)
|
||||
} else {
|
||||
0
|
||||
};
|
||||
|
@ -95,6 +107,10 @@ impl AddressAllocator {
|
|||
Ok(addr)
|
||||
}
|
||||
|
||||
pub fn allocate(&mut self, size: u64, alloc: Alloc, tag: String) -> Result<u64> {
|
||||
self.allocate_with_align(size, alloc, tag, self.alignment)
|
||||
}
|
||||
|
||||
/// Returns allocation associated with `alloc`, or None if no such allocation exists.
|
||||
pub fn get(&self, alloc: &Alloc) -> Option<&(u64, u64, String)> {
|
||||
self.allocs.get(alloc)
|
||||
|
@ -180,4 +196,51 @@ mod tests {
|
|||
Some(&(0x1000, 0x110, String::from("bar0")))
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn allocate_with_alignment_allocator_alignment() {
|
||||
let mut pool = AddressAllocator::new(0x1000, 0x10000, Some(0x100)).unwrap();
|
||||
assert_eq!(
|
||||
pool.allocate_with_align(0x110, Alloc::Anon(0), String::from("bar0"), 0x1),
|
||||
Ok(0x1000)
|
||||
);
|
||||
assert_eq!(
|
||||
pool.allocate_with_align(0x100, Alloc::Anon(1), String::from("bar1"), 0x1),
|
||||
Ok(0x1200)
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn allocate_with_alignment_custom_alignment() {
|
||||
let mut pool = AddressAllocator::new(0x1000, 0x10000, Some(0x4)).unwrap();
|
||||
assert_eq!(
|
||||
pool.allocate_with_align(0x110, Alloc::Anon(0), String::from("bar0"), 0x100),
|
||||
Ok(0x1000)
|
||||
);
|
||||
assert_eq!(
|
||||
pool.allocate_with_align(0x100, Alloc::Anon(1), String::from("bar1"), 0x100),
|
||||
Ok(0x1200)
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn allocate_with_alignment_no_allocator_alignment() {
|
||||
let mut pool = AddressAllocator::new(0x1000, 0x10000, None).unwrap();
|
||||
assert_eq!(
|
||||
pool.allocate_with_align(0x110, Alloc::Anon(0), String::from("bar0"), 0x100),
|
||||
Ok(0x1000)
|
||||
);
|
||||
assert_eq!(
|
||||
pool.allocate_with_align(0x100, Alloc::Anon(1), String::from("bar1"), 0x100),
|
||||
Ok(0x1200)
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn allocate_with_alignment_alignment_non_power_of_two() {
|
||||
let mut pool = AddressAllocator::new(0x1000, 0x10000, None).unwrap();
|
||||
assert!(pool
|
||||
.allocate_with_align(0x110, Alloc::Anon(0), String::from("bar0"), 200)
|
||||
.is_err());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,6 +33,8 @@ pub enum Alloc {
|
|||
PciBar { bus: u8, dev: u8, bar: u8 },
|
||||
/// GPU render node region.
|
||||
GpuRenderNode,
|
||||
/// Pmem device region with associated device index.
|
||||
PmemDevice(usize),
|
||||
}
|
||||
|
||||
#[derive(Debug, Eq, PartialEq)]
|
||||
|
|
106
src/linux.rs
106
src/linux.rs
|
@ -4,6 +4,7 @@
|
|||
|
||||
use std;
|
||||
use std::cmp::min;
|
||||
use std::convert::TryFrom;
|
||||
use std::error::Error as StdError;
|
||||
use std::ffi::CStr;
|
||||
use std::fmt::{self, Display};
|
||||
|
@ -32,18 +33,16 @@ use net_util::{Error as NetError, MacAddress, Tap};
|
|||
use qcow::{self, ImageType, QcowFile};
|
||||
use rand_ish::SimpleRng;
|
||||
use remain::sorted;
|
||||
#[cfg(feature = "gpu-forward")]
|
||||
use resources::Alloc;
|
||||
use resources::{Alloc, SystemAllocator};
|
||||
use sync::{Condvar, Mutex};
|
||||
use sys_util::net::{UnixSeqpacket, UnixSeqpacketListener, UnlinkUnixSeqpacketListener};
|
||||
|
||||
use sys_util::{
|
||||
self, block_signal, clear_signal, drop_capabilities, error, flock, get_blocked_signals,
|
||||
get_group_id, get_user_id, getegid, geteuid, info, register_signal_handler, set_cpu_affinity,
|
||||
validate_raw_fd, warn, EventFd, FlockOperation, GuestMemory, Killable, PollContext, PollToken,
|
||||
SignalFd, Terminal, TimerFd, SIGRTMIN,
|
||||
validate_raw_fd, warn, EventFd, FlockOperation, GuestAddress, GuestMemory, Killable,
|
||||
MemoryMapping, PollContext, PollToken, Protection, SignalFd, Terminal, TimerFd, SIGRTMIN,
|
||||
};
|
||||
#[cfg(feature = "gpu-forward")]
|
||||
use sys_util::{GuestAddress, MemoryMapping, Protection};
|
||||
use vhost;
|
||||
use vm_control::{
|
||||
BalloonControlCommand, BalloonControlRequestSocket, BalloonControlResponseSocket,
|
||||
|
@ -70,7 +69,9 @@ type RenderNodeHost = ();
|
|||
#[derive(Debug)]
|
||||
pub enum Error {
|
||||
AddGpuDeviceMemory(sys_util::Error),
|
||||
AddPmemDeviceMemory(sys_util::Error),
|
||||
AllocateGpuDeviceAddress,
|
||||
AllocatePmemDeviceAddress(resources::Error),
|
||||
BalloonDeviceNew(virtio::BalloonError),
|
||||
BlockDeviceNew(sys_util::Error),
|
||||
BlockSignal(sys_util::signal::Error),
|
||||
|
@ -106,6 +107,8 @@ pub enum Error {
|
|||
OpenVinput(PathBuf, io::Error),
|
||||
P9DeviceNew(virtio::P9Error),
|
||||
PivotRootDoesntExist(&'static str),
|
||||
PmemDeviceImageTooBig,
|
||||
PmemDeviceNew(sys_util::Error),
|
||||
PollContextAdd(sys_util::Error),
|
||||
PollContextDelete(sys_util::Error),
|
||||
QcowDeviceCreate(qcow::Error),
|
||||
|
@ -121,6 +124,7 @@ pub enum Error {
|
|||
RegisterWayland(arch::DeviceRegistrationError),
|
||||
ReserveGpuMemory(sys_util::MmapError),
|
||||
ReserveMemory(sys_util::Error),
|
||||
ReservePmemMemory(sys_util::MmapError),
|
||||
ResetTimerFd(sys_util::Error),
|
||||
RngDeviceNew(virtio::RngError),
|
||||
SettingGidMap(io_jail::Error),
|
||||
|
@ -143,7 +147,11 @@ impl Display for Error {
|
|||
#[sorted]
|
||||
match self {
|
||||
AddGpuDeviceMemory(e) => write!(f, "failed to add gpu device memory: {}", e),
|
||||
AddPmemDeviceMemory(e) => write!(f, "failed to add pmem device memory: {}", e),
|
||||
AllocateGpuDeviceAddress => write!(f, "failed to allocate gpu device guest address"),
|
||||
AllocatePmemDeviceAddress(e) => {
|
||||
write!(f, "failed to allocate memory for pmem device: {}", e)
|
||||
}
|
||||
BalloonDeviceNew(e) => write!(f, "failed to create balloon: {}", e),
|
||||
BlockDeviceNew(e) => write!(f, "failed to create block device: {}", e),
|
||||
BlockSignal(e) => write!(f, "failed to block signal: {}", e),
|
||||
|
@ -186,6 +194,10 @@ impl Display for Error {
|
|||
OpenVinput(p, e) => write!(f, "failed to open vinput device {}: {}", p.display(), e),
|
||||
P9DeviceNew(e) => write!(f, "failed to create 9p device: {}", e),
|
||||
PivotRootDoesntExist(p) => write!(f, "{} doesn't exist, can't jail devices.", p),
|
||||
PmemDeviceImageTooBig => {
|
||||
write!(f, "failed to create pmem device: pmem device image too big")
|
||||
}
|
||||
PmemDeviceNew(e) => write!(f, "failed to create pmem device: {}", e),
|
||||
PollContextAdd(e) => write!(f, "failed to add fd to poll context: {}", e),
|
||||
PollContextDelete(e) => write!(f, "failed to remove fd from poll context: {}", e),
|
||||
QcowDeviceCreate(e) => write!(f, "failed to read qcow formatted file {}", e),
|
||||
|
@ -209,6 +221,7 @@ impl Display for Error {
|
|||
RegisterWayland(e) => write!(f, "error registering wayland device: {}", e),
|
||||
ReserveGpuMemory(e) => write!(f, "failed to reserve gpu memory: {}", e),
|
||||
ReserveMemory(e) => write!(f, "failed to reserve memory: {}", e),
|
||||
ReservePmemMemory(e) => write!(f, "failed to reserve pmem memory: {}", e),
|
||||
ResetTimerFd(e) => write!(f, "failed to reset timerfd: {}", e),
|
||||
RngDeviceNew(e) => write!(f, "failed to set up rng: {}", e),
|
||||
SettingGidMap(e) => write!(f, "error setting GID map: {}", e),
|
||||
|
@ -692,11 +705,76 @@ fn create_9p_device(cfg: &Config, chronos: Ids, src: &Path, tag: &str) -> Device
|
|||
})
|
||||
}
|
||||
|
||||
fn create_pmem_device(
|
||||
cfg: &Config,
|
||||
vm: &mut Vm,
|
||||
resources: &mut SystemAllocator,
|
||||
disk: &DiskOption,
|
||||
index: usize,
|
||||
) -> DeviceResult {
|
||||
let fd = OpenOptions::new()
|
||||
.read(true)
|
||||
.write(!disk.read_only)
|
||||
.open(&disk.path)
|
||||
.map_err(Error::Disk)?;
|
||||
|
||||
let image_size = {
|
||||
let metadata = std::fs::metadata(&disk.path).map_err(Error::Disk)?;
|
||||
metadata.len()
|
||||
};
|
||||
|
||||
let protection = {
|
||||
if disk.read_only {
|
||||
Protection::read()
|
||||
} else {
|
||||
Protection::read_write()
|
||||
}
|
||||
};
|
||||
|
||||
let memory_mapping = {
|
||||
// Conversion from u64 to usize may fail on 32bit system.
|
||||
let image_size = usize::try_from(image_size).map_err(|_| Error::PmemDeviceImageTooBig)?;
|
||||
|
||||
MemoryMapping::from_fd_offset_protection(&fd, image_size, 0, protection)
|
||||
.map_err(Error::ReservePmemMemory)?
|
||||
};
|
||||
|
||||
let mapping_address = resources
|
||||
.device_allocator()
|
||||
.allocate_with_align(
|
||||
image_size,
|
||||
Alloc::PmemDevice(index),
|
||||
format!("pmem_disk_image_{}", index),
|
||||
// Linux kernel requires pmem namespaces to be 128 MiB aligned.
|
||||
128 * 1024 * 1024, /* 128 MiB */
|
||||
)
|
||||
.map_err(Error::AllocatePmemDeviceAddress)?;
|
||||
|
||||
vm.add_device_memory(
|
||||
GuestAddress(mapping_address),
|
||||
memory_mapping,
|
||||
/* read_only = */ disk.read_only,
|
||||
/* log_dirty_pages = */ false,
|
||||
)
|
||||
.map_err(Error::AddPmemDeviceMemory)?;
|
||||
|
||||
let dev = virtio::Pmem::new(fd, GuestAddress(mapping_address), image_size)
|
||||
.map_err(Error::PmemDeviceNew)?;
|
||||
|
||||
Ok(VirtioDeviceStub {
|
||||
dev: Box::new(dev) as Box<dyn VirtioDevice>,
|
||||
/// TODO(jstaron) Create separate device policy for pmem_device.
|
||||
jail: simple_jail(&cfg, "block_device.policy")?,
|
||||
})
|
||||
}
|
||||
|
||||
// gpu_device_socket is not used when GPU support is disabled.
|
||||
#[cfg_attr(not(feature = "gpu"), allow(unused_variables))]
|
||||
fn create_virtio_devices(
|
||||
cfg: &Config,
|
||||
mem: &GuestMemory,
|
||||
vm: &mut Vm,
|
||||
resources: &mut SystemAllocator,
|
||||
_exit_evt: &EventFd,
|
||||
wayland_device_socket: VmMemoryControlRequestSocket,
|
||||
gpu_device_socket: VmMemoryControlRequestSocket,
|
||||
|
@ -710,6 +788,10 @@ fn create_virtio_devices(
|
|||
devs.push(create_block_device(cfg, disk, disk_device_socket)?);
|
||||
}
|
||||
|
||||
for (index, pmem_disk) in cfg.pmem_devices.iter().enumerate() {
|
||||
devs.push(create_pmem_device(cfg, vm, resources, pmem_disk, index)?);
|
||||
}
|
||||
|
||||
devs.push(create_rng_device(cfg)?);
|
||||
|
||||
#[cfg(feature = "tpm")]
|
||||
|
@ -799,6 +881,8 @@ fn create_virtio_devices(
|
|||
fn create_devices(
|
||||
cfg: &Config,
|
||||
mem: &GuestMemory,
|
||||
vm: &mut Vm,
|
||||
resources: &mut SystemAllocator,
|
||||
exit_evt: &EventFd,
|
||||
wayland_device_socket: VmMemoryControlRequestSocket,
|
||||
gpu_device_socket: VmMemoryControlRequestSocket,
|
||||
|
@ -809,6 +893,8 @@ fn create_devices(
|
|||
let stubs = create_virtio_devices(
|
||||
&cfg,
|
||||
mem,
|
||||
vm,
|
||||
resources,
|
||||
exit_evt,
|
||||
wayland_device_socket,
|
||||
gpu_device_socket,
|
||||
|
@ -1210,11 +1296,13 @@ pub fn run_config(cfg: Config) -> Result<()> {
|
|||
components,
|
||||
cfg.split_irqchip,
|
||||
&cfg.serial_parameters,
|
||||
|m, e| {
|
||||
|mem, vm, sys_allocator, exit_evt| {
|
||||
create_devices(
|
||||
&cfg,
|
||||
m,
|
||||
e,
|
||||
mem,
|
||||
vm,
|
||||
sys_allocator,
|
||||
exit_evt,
|
||||
wayland_device_socket,
|
||||
gpu_device_socket,
|
||||
balloon_device_socket,
|
||||
|
|
18
src/main.rs
18
src/main.rs
|
@ -102,6 +102,7 @@ pub struct Config {
|
|||
plugin_mounts: Vec<BindMount>,
|
||||
plugin_gid_maps: Vec<GidMap>,
|
||||
disks: Vec<DiskOption>,
|
||||
pmem_devices: Vec<DiskOption>,
|
||||
host_ip: Option<net::Ipv4Addr>,
|
||||
netmask: Option<net::Ipv4Addr>,
|
||||
mac_address: Option<net_util::MacAddress>,
|
||||
|
@ -143,6 +144,7 @@ impl Default for Config {
|
|||
plugin_mounts: Vec::new(),
|
||||
plugin_gid_maps: Vec::new(),
|
||||
disks: Vec::new(),
|
||||
pmem_devices: Vec::new(),
|
||||
host_ip: None,
|
||||
netmask: None,
|
||||
mac_address: None,
|
||||
|
@ -440,6 +442,20 @@ fn set_argument(cfg: &mut Config, name: &str, value: Option<&str>) -> argument::
|
|||
read_only: !name.starts_with("rw"),
|
||||
});
|
||||
}
|
||||
"pmem-device" | "rw-pmem-device" => {
|
||||
let disk_path = PathBuf::from(value.unwrap());
|
||||
if !disk_path.exists() {
|
||||
return Err(argument::Error::InvalidValue {
|
||||
value: value.unwrap().to_owned(),
|
||||
expected: "this disk path does not exist",
|
||||
});
|
||||
}
|
||||
|
||||
cfg.pmem_devices.push(DiskOption {
|
||||
path: disk_path,
|
||||
read_only: !name.starts_with("rw"),
|
||||
});
|
||||
}
|
||||
"host_ip" => {
|
||||
if cfg.host_ip.is_some() {
|
||||
return Err(argument::Error::TooManyArguments(
|
||||
|
@ -805,6 +821,8 @@ fn run_vm(args: std::env::Args) -> std::result::Result<(), ()> {
|
|||
Argument::value("qcow", "PATH", "Path to a qcow2 disk image. (Deprecated; use --disk instead.)"),
|
||||
Argument::value("rwdisk", "PATH", "Path to a writable disk image."),
|
||||
Argument::value("rwqcow", "PATH", "Path to a writable qcow2 disk image. (Deprecated; use --rwdisk instead.)"),
|
||||
Argument::value("rw-pmem-device", "PATH", "Path to a writable disk image."),
|
||||
Argument::value("pmem-device", "PATH", "Path to a disk image."),
|
||||
Argument::value("host_ip",
|
||||
"IP",
|
||||
"IP address to assign to host tap interface."),
|
||||
|
|
|
@ -82,6 +82,12 @@ impl Protection {
|
|||
Protection(libc::PROT_READ | libc::PROT_WRITE)
|
||||
}
|
||||
|
||||
/// Returns Protection allowing read access.
|
||||
#[inline(always)]
|
||||
pub fn read() -> Protection {
|
||||
Protection(libc::PROT_READ)
|
||||
}
|
||||
|
||||
/// Set read events.
|
||||
#[inline(always)]
|
||||
pub fn set_read(self) -> Protection {
|
||||
|
|
|
@ -309,6 +309,8 @@ impl arch::LinuxArch for X8664arch {
|
|||
where
|
||||
F: FnOnce(
|
||||
&GuestMemory,
|
||||
&mut Vm,
|
||||
&mut SystemAllocator,
|
||||
&EventFd,
|
||||
) -> std::result::Result<Vec<(Box<dyn PciDevice>, Option<Minijail>)>, E>,
|
||||
E: StdError + 'static,
|
||||
|
@ -348,8 +350,8 @@ impl arch::LinuxArch for X8664arch {
|
|||
|
||||
let exit_evt = EventFd::new().map_err(Error::CreateEventFd)?;
|
||||
|
||||
let pci_devices =
|
||||
create_devices(&mem, &exit_evt).map_err(|e| Error::CreateDevices(Box::new(e)))?;
|
||||
let pci_devices = create_devices(&mem, &mut vm, &mut resources, &exit_evt)
|
||||
.map_err(|e| Error::CreateDevices(Box::new(e)))?;
|
||||
let (pci, pci_irqs, pid_debug_label_map) =
|
||||
arch::generate_pci_root(pci_devices, &mut mmio_bus, &mut resources, &mut vm)
|
||||
.map_err(Error::CreatePciRoot)?;
|
||||
|
|
Loading…
Reference in a new issue