Merge pull request #4515 from openanolis/anolis/dragonball-3

runtime-rs: built-in Dragonball sandbox part III - virtio-blk, virtio-fs, virtio-net and VMM API support
This commit is contained in:
Peng Tao
2022-07-08 23:14:01 +08:00
committed by GitHub
28 changed files with 4620 additions and 83 deletions

View File

@@ -49,6 +49,10 @@ acpi = []
atomic-guest-memory = []
hotplug = ["virtio-vsock"]
virtio-vsock = ["dbs-virtio-devices/virtio-vsock", "virtio-queue"]
virtio-blk = ["dbs-virtio-devices/virtio-blk", "virtio-queue"]
virtio-net = ["dbs-virtio-devices/virtio-net", "virtio-queue"]
# virtio-fs only work on atomic-guest-memory
virtio-fs = ["dbs-virtio-devices/virtio-fs", "virtio-queue", "atomic-guest-memory"]
[patch.'crates-io']
dbs-device = { git = "https://github.com/openanolis/dragonball-sandbox.git", rev = "7a8e832b53d66994d6a16f0513d69f540583dcd0" }

View File

@@ -3,5 +3,25 @@
We provide plenty API for Kata runtime to interact with `Dragonball` virtual machine manager.
This document provides the introduction for each of them.
TODO: Details will be added in the Part III PR for `Dragonball`
## `ConfigureBootSource`
Configure the boot source of the VM using `BootSourceConfig`. This action can only be called before the VM has booted.
### Boot Source Config
1. `kernel_path`: Path of the kernel image. `Dragonball` only supports compressed kernel image for now.
2. `initrd_path`: Path of the initrd (could be None)
3. `boot_args`: Boot arguments passed to the kernel (could be None)
## `SetVmConfiguration`
Set virtual machine configuration using `VmConfigInfo` to initialize VM.
### VM Config Info
1. `vcpu_count`: Number of vCPU to start. Currently we only support up to 255 vCPUs.
2. `max_vcpu_count`: Max number of vCPU can be added through CPU hotplug.
3. `cpu_pm`: CPU power management.
4. `cpu_topology`: CPU topology information (including `threads_per_core`, `cores_per_die`, `dies_per_socket` and `sockets`).
5. `vpmu_feature`: `vPMU` feature level.
6. `mem_type`: Memory type that can be either `hugetlbfs` or `shmem`, default is `shmem`.
7. `mem_file_path` : Memory file path.
8. `mem_size_mib`: The memory size in MiB. The maximum memory size is 1TB.
9. `serial_path`: Optional sock path.

View File

@@ -27,7 +27,7 @@ use dbs_address_space::{
AddressSpaceRegionType, NumaNode, NumaNodeInfo, MPOL_MF_MOVE, MPOL_PREFERRED,
};
use dbs_allocator::Constraint;
use kvm_bindings::{kvm_userspace_memory_region, KVM_MEM_LOG_DIRTY_PAGES};
use kvm_bindings::kvm_userspace_memory_region;
use kvm_ioctls::VmFd;
use log::{debug, error, info, warn};
use nix::sys::mman;
@@ -245,6 +245,11 @@ impl AddressSpaceMgr {
self.address_space.is_some()
}
/// Gets address space.
pub fn address_space(&self) -> Option<&AddressSpace> {
self.address_space.as_ref()
}
/// Create the address space for a virtual machine.
///
/// This method is designed to be called when starting up a virtual machine instead of at
@@ -393,11 +398,8 @@ impl AddressSpaceMgr {
let host_addr = mmap_reg
.get_host_address(MemoryRegionAddress(0))
.map_err(|_e| AddressManagerError::InvalidOperation)?;
let flags = if param.dirty_page_logging {
KVM_MEM_LOG_DIRTY_PAGES
} else {
0
};
let flags = 0u32;
let mem_region = kvm_userspace_memory_region {
slot: slot as u32,
guest_phys_addr: reg.start_addr().raw_value(),
@@ -640,7 +642,7 @@ impl AddressSpaceMgr {
let node = self
.numa_nodes
.entry(guest_numa_node_id)
.or_insert(NumaNode::new());
.or_insert_with(NumaNode::new);
node.add_info(&NumaNodeInfo {
base: region.start_addr(),
size: region.len(),
@@ -734,7 +736,7 @@ mod tests {
.unwrap();
val = gmem.read_obj(GuestAddress(GUEST_MEM_START + 0x1)).unwrap();
assert_eq!(val, 0xa5);
val = gmem.read_obj(GuestAddress(GUEST_MEM_START + 0x0)).unwrap();
val = gmem.read_obj(GuestAddress(GUEST_MEM_START)).unwrap();
assert_eq!(val, 1);
val = gmem.read_obj(GuestAddress(GUEST_MEM_START + 0x2)).unwrap();
assert_eq!(val, 3);
@@ -835,7 +837,7 @@ mod tests {
size: mem_size >> 20,
host_numa_node_id: None,
guest_numa_node_id: Some(0),
vcpu_ids: cpu_vec.clone(),
vcpu_ids: cpu_vec,
}];
let mut builder = AddressSpaceMgrBuilder::new("hugeshmem", "").unwrap();
builder.toggle_prealloc(true);
@@ -850,9 +852,9 @@ mod tests {
assert_eq!(builder.mem_type, "shmem");
assert_eq!(builder.mem_file, "/tmp/shmem");
assert_eq!(builder.mem_index, 0);
assert_eq!(builder.mem_suffix, true);
assert_eq!(builder.mem_prealloc, false);
assert_eq!(builder.dirty_page_logging, false);
assert!(builder.mem_suffix);
assert!(!builder.mem_prealloc);
assert!(!builder.dirty_page_logging);
assert!(builder.vmfd.is_none());
assert_eq!(&builder.get_next_mem_file(), "/tmp/shmem0");
@@ -867,8 +869,8 @@ mod tests {
builder.toggle_prealloc(true);
builder.toggle_dirty_page_logging(true);
assert_eq!(builder.mem_prealloc, true);
assert_eq!(builder.dirty_page_logging, true);
assert!(builder.mem_prealloc);
assert!(builder.dirty_page_logging);
}
#[test]

View File

@@ -0,0 +1,55 @@
// Copyright (C) 2022 Alibaba Cloud. All rights reserved.
// Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
// SPDX-License-Identifier: Apache-2.0
use serde_derive::{Deserialize, Serialize};
/// Default guest kernel command line:
/// - `reboot=k` shut down the guest on reboot, instead of well... rebooting;
/// - `panic=1` on panic, reboot after 1 second;
/// - `pci=off` do not scan for PCI devices (ser boot time);
/// - `nomodules` disable loadable kernel module support;
/// - `8250.nr_uarts=0` disable 8250 serial interface;
/// - `i8042.noaux` do not probe the i8042 controller for an attached mouse (ser boot time);
/// - `i8042.nomux` do not probe i8042 for a multiplexing controller (ser boot time);
/// - `i8042.nopnp` do not use ACPIPnP to discover KBD/AUX controllers (ser boot time);
/// - `i8042.dumbkbd` do not attempt to control kbd state via the i8042 (ser boot time).
pub const DEFAULT_KERNEL_CMDLINE: &str = "reboot=k panic=1 pci=off nomodules 8250.nr_uarts=0 \
i8042.noaux i8042.nomux i8042.nopnp i8042.dumbkbd";
/// Strongly typed data structure used to configure the boot source of the microvm.
#[derive(Clone, Debug, Deserialize, PartialEq, Serialize, Default)]
#[serde(deny_unknown_fields)]
pub struct BootSourceConfig {
/// Path of the kernel image.
/// We only support uncompressed kernel for Dragonball.
pub kernel_path: String,
/// Path of the initrd, if there is one.
/// ps. rootfs is set in BlockDeviceConfigInfo
pub initrd_path: Option<String>,
/// The boot arguments to pass to the kernel.
#[serde(skip_serializing_if = "Option::is_none")]
pub boot_args: Option<String>,
}
/// Errors associated with actions on `BootSourceConfig`.
#[derive(Debug, thiserror::Error)]
pub enum BootSourceConfigError {
/// The kernel file cannot be opened.
#[error(
"the kernel file cannot be opened due to invalid kernel path or invalid permissions: {0}"
)]
InvalidKernelPath(#[source] std::io::Error),
/// The initrd file cannot be opened.
#[error("the initrd file cannot be opened due to invalid path or invalid permissions: {0}")]
InvalidInitrdPath(#[source] std::io::Error),
/// The kernel command line is invalid.
#[error("the kernel command line is invalid: {0}")]
InvalidKernelCommandLine(#[source] linux_loader::cmdline::Error),
/// The boot source cannot be update post boot.
#[error("the update operation is not allowed after boot")]
UpdateNotAllowedPostBoot,
}

View File

@@ -54,6 +54,8 @@ pub struct InstanceInfo {
pub async_state: AsyncState,
/// List of tids of vcpu threads (vcpu index, tid)
pub tids: Vec<(u8, u32)>,
/// Last instance downtime
pub last_instance_downtime: u64,
}
impl InstanceInfo {
@@ -66,6 +68,7 @@ impl InstanceInfo {
pid: std::process::id(),
async_state: AsyncState::Uninitialized,
tids: Vec::new(),
last_instance_downtime: 0,
}
}
}
@@ -79,6 +82,7 @@ impl Default for InstanceInfo {
pid: std::process::id(),
async_state: AsyncState::Uninitialized,
tids: Vec::new(),
last_instance_downtime: 0,
}
}
}

View File

@@ -0,0 +1,86 @@
// Copyright (C) 2022 Alibaba Cloud. All rights reserved.
// Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
// SPDX-License-Identifier: Apache-2.0
/// We only support this number of vcpus for now. Mostly because we have set all vcpu related metrics as u8
/// and breaking u8 will take extra efforts.
pub const MAX_SUPPORTED_VCPUS: u8 = 254;
/// Memory hotplug value should have alignment in this size (unit: MiB)
pub const MEMORY_HOTPLUG_ALIGHMENT: u8 = 64;
/// Errors associated with configuring the microVM.
#[derive(Debug, PartialEq, thiserror::Error)]
pub enum VmConfigError {
/// Cannot update the configuration of the microvm post boot.
#[error("update operation is not allowed after boot")]
UpdateNotAllowedPostBoot,
/// The max vcpu count is invalid.
#[error("the vCPU number shouldn't large than {}", MAX_SUPPORTED_VCPUS)]
VcpuCountExceedsMaximum,
/// The vcpu count is invalid. When hyperthreading is enabled, the `cpu_count` must be either
/// 1 or an even number.
#[error(
"the vCPU number '{0}' can only be 1 or an even number when hyperthreading is enabled"
)]
InvalidVcpuCount(u8),
/// The threads_per_core is invalid. It should be either 1 or 2.
#[error("the threads_per_core number '{0}' can only be 1 or 2")]
InvalidThreadsPerCore(u8),
/// The cores_per_die is invalid. It should be larger than 0.
#[error("the cores_per_die number '{0}' can only be larger than 0")]
InvalidCoresPerDie(u8),
/// The dies_per_socket is invalid. It should be larger than 0.
#[error("the dies_per_socket number '{0}' can only be larger than 0")]
InvalidDiesPerSocket(u8),
/// The socket number is invalid. It should be either 1 or 2.
#[error("the socket number '{0}' can only be 1 or 2")]
InvalidSocket(u8),
/// max vcpu count inferred from cpu topology(threads_per_core * cores_per_die * dies_per_socket * sockets) should be larger or equal to vcpu_count
#[error("the max vcpu count inferred from cpu topology '{0}' (threads_per_core * cores_per_die * dies_per_socket * sockets) should be larger or equal to vcpu_count")]
InvalidCpuTopology(u8),
/// The max vcpu count is invalid.
#[error(
"the max vCPU number '{0}' shouldn't less than vCPU count and can only be 1 or an even number when hyperthreading is enabled"
)]
InvalidMaxVcpuCount(u8),
/// The memory size is invalid. The memory can only be an unsigned integer.
#[error("the memory size 0x{0:x}MiB is invalid")]
InvalidMemorySize(usize),
/// The hotplug memory size is invalid. The memory can only be an unsigned integer.
#[error(
"the hotplug memory size '{0}' (MiB) is invalid, must be multiple of {}",
MEMORY_HOTPLUG_ALIGHMENT
)]
InvalidHotplugMemorySize(usize),
/// The memory type is invalid.
#[error("the memory type '{0}' is invalid")]
InvalidMemType(String),
/// The memory file path is invalid.
#[error("the memory file path is invalid")]
InvalidMemFilePath(String),
/// NUMA region memory size is invalid
#[error("Total size of memory in NUMA regions: {0}, should matches memory size in config")]
InvalidNumaRegionMemorySize(usize),
/// NUMA region vCPU count is invalid
#[error("Total counts of vCPUs in NUMA regions: {0}, should matches max vcpu count in config")]
InvalidNumaRegionCpuCount(u16),
/// NUMA region vCPU count is invalid
#[error("Max id of vCPUs in NUMA regions: {0}, should matches max vcpu count in config")]
InvalidNumaRegionCpuMaxId(u16),
}

View File

@@ -3,5 +3,17 @@
//! API Version 1 related data structures to configure the vmm.
mod vmm_action;
pub use self::vmm_action::*;
/// Wrapper for configuring the microVM boot source.
mod boot_source;
pub use self::boot_source::{BootSourceConfig, BootSourceConfigError, DEFAULT_KERNEL_CMDLINE};
/// Wrapper over the microVM general information.
mod instance_info;
pub use self::instance_info::{InstanceInfo, InstanceState};
/// Wrapper for configuring the memory and CPU of the microVM.
mod machine_config;
pub use self::machine_config::{VmConfigError, MAX_SUPPORTED_VCPUS};

View File

@@ -0,0 +1,637 @@
// Copyright (C) 2020-2022 Alibaba Cloud. All rights reserved.
// Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
// SPDX-License-Identifier: Apache-2.0
//
// Portions Copyright 2017 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 THIRD-PARTY file.
use std::fs::File;
use std::sync::mpsc::{Receiver, Sender, TryRecvError};
use log::{debug, error, info, warn};
use crate::error::{Result, StartMicroVmError, StopMicrovmError};
use crate::event_manager::EventManager;
use crate::vm::{CpuTopology, KernelConfigInfo, VmConfigInfo};
use crate::vmm::Vmm;
use self::VmConfigError::*;
use self::VmmActionError::MachineConfig;
#[cfg(feature = "virtio-blk")]
pub use crate::device_manager::blk_dev_mgr::{
BlockDeviceConfigInfo, BlockDeviceConfigUpdateInfo, BlockDeviceError, BlockDeviceMgr,
};
#[cfg(feature = "virtio-fs")]
pub use crate::device_manager::fs_dev_mgr::{
FsDeviceConfigInfo, FsDeviceConfigUpdateInfo, FsDeviceError, FsDeviceMgr, FsMountConfigInfo,
};
#[cfg(feature = "virtio-net")]
pub use crate::device_manager::virtio_net_dev_mgr::{
VirtioNetDeviceConfigInfo, VirtioNetDeviceConfigUpdateInfo, VirtioNetDeviceError,
VirtioNetDeviceMgr,
};
#[cfg(feature = "virtio-vsock")]
pub use crate::device_manager::vsock_dev_mgr::{VsockDeviceConfigInfo, VsockDeviceError};
use super::*;
/// Wrapper for all errors associated with VMM actions.
#[derive(Debug, thiserror::Error)]
pub enum VmmActionError {
/// Invalid virtual machine instance ID.
#[error("the virtual machine instance ID is invalid")]
InvalidVMID,
/// Failed to hotplug, due to Upcall not ready.
#[error("Upcall not ready, can't hotplug device.")]
UpcallNotReady,
/// The action `ConfigureBootSource` failed either because of bad user input or an internal
/// error.
#[error("failed to configure boot source for VM: {0}")]
BootSource(#[source] BootSourceConfigError),
/// The action `StartMicroVm` failed either because of bad user input or an internal error.
#[error("failed to boot the VM: {0}")]
StartMicroVm(#[source] StartMicroVmError),
/// The action `StopMicroVm` failed either because of bad user input or an internal error.
#[error("failed to shutdown the VM: {0}")]
StopMicrovm(#[source] StopMicrovmError),
/// One of the actions `GetVmConfiguration` or `SetVmConfiguration` failed either because of bad
/// input or an internal error.
#[error("failed to set configuration for the VM: {0}")]
MachineConfig(#[source] VmConfigError),
#[cfg(feature = "virtio-vsock")]
/// The action `InsertVsockDevice` failed either because of bad user input or an internal error.
#[error("failed to add virtio-vsock device: {0}")]
Vsock(#[source] VsockDeviceError),
#[cfg(feature = "virtio-blk")]
/// Block device related errors.
#[error("virtio-blk device error: {0}")]
Block(#[source] BlockDeviceError),
#[cfg(feature = "virtio-net")]
/// Net device related errors.
#[error("virtio-net device error: {0}")]
VirtioNet(#[source] VirtioNetDeviceError),
#[cfg(feature = "virtio-fs")]
/// The action `InsertFsDevice` failed either because of bad user input or an internal error.
#[error("virtio-fs device: {0}")]
FsDevice(#[source] FsDeviceError),
}
/// This enum represents the public interface of the VMM. Each action contains various
/// bits of information (ids, paths, etc.).
#[derive(Clone, Debug, PartialEq)]
pub enum VmmAction {
/// Configure the boot source of the microVM using `BootSourceConfig`.
/// This action can only be called before the microVM has booted.
ConfigureBootSource(BootSourceConfig),
/// Launch the microVM. This action can only be called before the microVM has booted.
StartMicroVm,
/// Shutdown the vmicroVM. This action can only be called after the microVM has booted.
/// When vmm is used as the crate by the other process, which is need to
/// shutdown the vcpu threads and destory all of the object.
ShutdownMicroVm,
/// Get the configuration of the microVM.
GetVmConfiguration,
/// Set the microVM configuration (memory & vcpu) using `VmConfig` as input. This
/// action can only be called before the microVM has booted.
SetVmConfiguration(VmConfigInfo),
#[cfg(feature = "virtio-vsock")]
/// Add a new vsock device or update one that already exists using the
/// `VsockDeviceConfig` as input. This action can only be called before the microVM has
/// booted. The response is sent using the `OutcomeSender`.
InsertVsockDevice(VsockDeviceConfigInfo),
#[cfg(feature = "virtio-blk")]
/// Add a new block device or update one that already exists using the `BlockDeviceConfig` as
/// input. This action can only be called before the microVM has booted.
InsertBlockDevice(BlockDeviceConfigInfo),
#[cfg(feature = "virtio-blk")]
/// Remove a new block device for according to given drive_id
RemoveBlockDevice(String),
#[cfg(feature = "virtio-blk")]
/// Update a block device, after microVM start. Currently, the only updatable properties
/// are the RX and TX rate limiters.
UpdateBlockDevice(BlockDeviceConfigUpdateInfo),
#[cfg(feature = "virtio-net")]
/// Add a new network interface config or update one that already exists using the
/// `NetworkInterfaceConfig` as input. This action can only be called before the microVM has
/// booted. The response is sent using the `OutcomeSender`.
InsertNetworkDevice(VirtioNetDeviceConfigInfo),
#[cfg(feature = "virtio-net")]
/// Update a network interface, after microVM start. Currently, the only updatable properties
/// are the RX and TX rate limiters.
UpdateNetworkInterface(VirtioNetDeviceConfigUpdateInfo),
#[cfg(feature = "virtio-fs")]
/// Add a new shared fs device or update one that already exists using the
/// `FsDeviceConfig` as input. This action can only be called before the microVM has
/// booted.
InsertFsDevice(FsDeviceConfigInfo),
#[cfg(feature = "virtio-fs")]
/// Attach a new virtiofs Backend fs or detach an existing virtiofs Backend fs using the
/// `FsMountConfig` as input. This action can only be called _after_ the microVM has
/// booted.
ManipulateFsBackendFs(FsMountConfigInfo),
#[cfg(feature = "virtio-fs")]
/// Update fs rate limiter, after microVM start.
UpdateFsDevice(FsDeviceConfigUpdateInfo),
}
/// The enum represents the response sent by the VMM in case of success. The response is either
/// empty, when no data needs to be sent, or an internal VMM structure.
#[derive(Debug)]
pub enum VmmData {
/// No data is sent on the channel.
Empty,
/// The microVM configuration represented by `VmConfigInfo`.
MachineConfiguration(Box<VmConfigInfo>),
}
/// Request data type used to communicate between the API and the VMM.
pub type VmmRequest = Box<VmmAction>;
/// Data type used to communicate between the API and the VMM.
pub type VmmRequestResult = std::result::Result<VmmData, VmmActionError>;
/// Response data type used to communicate between the API and the VMM.
pub type VmmResponse = Box<VmmRequestResult>;
/// VMM Service to handle requests from the API server.
///
/// There are two levels of API servers as below:
/// API client <--> VMM API Server <--> VMM Core
pub struct VmmService {
from_api: Receiver<VmmRequest>,
to_api: Sender<VmmResponse>,
machine_config: VmConfigInfo,
}
impl VmmService {
/// Create a new VMM API server instance.
pub fn new(from_api: Receiver<VmmRequest>, to_api: Sender<VmmResponse>) -> Self {
VmmService {
from_api,
to_api,
machine_config: VmConfigInfo::default(),
}
}
/// Handle requests from the HTTP API Server and send back replies.
pub fn run_vmm_action(&mut self, vmm: &mut Vmm, event_mgr: &mut EventManager) -> Result<()> {
let request = match self.from_api.try_recv() {
Ok(t) => *t,
Err(TryRecvError::Empty) => {
warn!("Got a spurious notification from api thread");
return Ok(());
}
Err(TryRecvError::Disconnected) => {
panic!("The channel's sending half was disconnected. Cannot receive data.");
}
};
debug!("receive vmm action: {:?}", request);
let response = match request {
VmmAction::ConfigureBootSource(boot_source_body) => {
self.configure_boot_source(vmm, boot_source_body)
}
VmmAction::StartMicroVm => self.start_microvm(vmm, event_mgr),
VmmAction::ShutdownMicroVm => self.shutdown_microvm(vmm),
VmmAction::GetVmConfiguration => Ok(VmmData::MachineConfiguration(Box::new(
self.machine_config.clone(),
))),
VmmAction::SetVmConfiguration(machine_config) => {
self.set_vm_configuration(vmm, machine_config)
}
#[cfg(feature = "virtio-vsock")]
VmmAction::InsertVsockDevice(vsock_cfg) => self.add_vsock_device(vmm, vsock_cfg),
#[cfg(feature = "virtio-blk")]
VmmAction::InsertBlockDevice(block_device_config) => {
self.add_block_device(vmm, event_mgr, block_device_config)
}
#[cfg(feature = "virtio-blk")]
VmmAction::UpdateBlockDevice(blk_update) => {
self.update_blk_rate_limiters(vmm, blk_update)
}
#[cfg(feature = "virtio-blk")]
VmmAction::RemoveBlockDevice(drive_id) => {
self.remove_block_device(vmm, event_mgr, &drive_id)
}
#[cfg(feature = "virtio-net")]
VmmAction::InsertNetworkDevice(virtio_net_cfg) => {
self.add_virtio_net_device(vmm, event_mgr, virtio_net_cfg)
}
#[cfg(feature = "virtio-net")]
VmmAction::UpdateNetworkInterface(netif_update) => {
self.update_net_rate_limiters(vmm, netif_update)
}
#[cfg(feature = "virtio-fs")]
VmmAction::InsertFsDevice(fs_cfg) => self.add_fs_device(vmm, fs_cfg),
#[cfg(feature = "virtio-fs")]
VmmAction::ManipulateFsBackendFs(fs_mount_cfg) => {
self.manipulate_fs_backend_fs(vmm, fs_mount_cfg)
}
#[cfg(feature = "virtio-fs")]
VmmAction::UpdateFsDevice(fs_update_cfg) => {
self.update_fs_rate_limiters(vmm, fs_update_cfg)
}
};
debug!("send vmm response: {:?}", response);
self.send_response(response)
}
fn send_response(&self, result: VmmRequestResult) -> Result<()> {
self.to_api
.send(Box::new(result))
.map_err(|_| ())
.expect("vmm: one-shot API result channel has been closed");
Ok(())
}
fn configure_boot_source(
&self,
vmm: &mut Vmm,
boot_source_config: BootSourceConfig,
) -> VmmRequestResult {
use super::BootSourceConfigError::{
InvalidInitrdPath, InvalidKernelCommandLine, InvalidKernelPath,
UpdateNotAllowedPostBoot,
};
use super::VmmActionError::BootSource;
let vm = vmm.get_vm_mut().ok_or(VmmActionError::InvalidVMID)?;
if vm.is_vm_initialized() {
return Err(BootSource(UpdateNotAllowedPostBoot));
}
let kernel_file = File::open(&boot_source_config.kernel_path)
.map_err(|e| BootSource(InvalidKernelPath(e)))?;
let initrd_file = match boot_source_config.initrd_path {
None => None,
Some(ref path) => Some(File::open(path).map_err(|e| BootSource(InvalidInitrdPath(e)))?),
};
let mut cmdline = linux_loader::cmdline::Cmdline::new(dbs_boot::layout::CMDLINE_MAX_SIZE);
let boot_args = boot_source_config
.boot_args
.clone()
.unwrap_or_else(|| String::from(DEFAULT_KERNEL_CMDLINE));
cmdline
.insert_str(boot_args)
.map_err(|e| BootSource(InvalidKernelCommandLine(e)))?;
let kernel_config = KernelConfigInfo::new(kernel_file, initrd_file, cmdline);
vm.set_kernel_config(kernel_config);
Ok(VmmData::Empty)
}
fn start_microvm(&mut self, vmm: &mut Vmm, event_mgr: &mut EventManager) -> VmmRequestResult {
use self::StartMicroVmError::MicroVMAlreadyRunning;
use self::VmmActionError::StartMicroVm;
let vmm_seccomp_filter = vmm.vmm_seccomp_filter();
let vcpu_seccomp_filter = vmm.vcpu_seccomp_filter();
let vm = vmm.get_vm_mut().ok_or(VmmActionError::InvalidVMID)?;
if vm.is_vm_initialized() {
return Err(StartMicroVm(MicroVMAlreadyRunning));
}
vm.start_microvm(event_mgr, vmm_seccomp_filter, vcpu_seccomp_filter)
.map(|_| VmmData::Empty)
.map_err(StartMicroVm)
}
fn shutdown_microvm(&mut self, vmm: &mut Vmm) -> VmmRequestResult {
vmm.event_ctx.exit_evt_triggered = true;
Ok(VmmData::Empty)
}
/// Set virtual machine configuration configurations.
pub fn set_vm_configuration(
&mut self,
vmm: &mut Vmm,
machine_config: VmConfigInfo,
) -> VmmRequestResult {
let vm = vmm.get_vm_mut().ok_or(VmmActionError::InvalidVMID)?;
if vm.is_vm_initialized() {
return Err(MachineConfig(UpdateNotAllowedPostBoot));
}
// If the check is successful, set it up together.
let mut config = vm.vm_config().clone();
if config.vcpu_count != machine_config.vcpu_count {
let vcpu_count = machine_config.vcpu_count;
// Check that the vcpu_count value is >=1.
if vcpu_count == 0 {
return Err(MachineConfig(InvalidVcpuCount(vcpu_count)));
}
config.vcpu_count = vcpu_count;
}
if config.cpu_topology != machine_config.cpu_topology {
let cpu_topology = &machine_config.cpu_topology;
config.cpu_topology = handle_cpu_topology(cpu_topology, config.vcpu_count)?.clone();
} else {
// the same default
let mut default_cpu_topology = CpuTopology {
threads_per_core: 1,
cores_per_die: config.vcpu_count,
dies_per_socket: 1,
sockets: 1,
};
if machine_config.max_vcpu_count > config.vcpu_count {
default_cpu_topology.cores_per_die = machine_config.max_vcpu_count;
}
config.cpu_topology = default_cpu_topology;
}
let cpu_topology = &config.cpu_topology;
let max_vcpu_from_topo = cpu_topology.threads_per_core
* cpu_topology.cores_per_die
* cpu_topology.dies_per_socket
* cpu_topology.sockets;
// If the max_vcpu_count inferred by cpu_topology is not equal to
// max_vcpu_count, max_vcpu_count will be changed. currently, max vcpu size
// is used when cpu_topology is not defined and help define the cores_per_die
// for the default cpu topology.
let mut max_vcpu_count = machine_config.max_vcpu_count;
if max_vcpu_count < config.vcpu_count {
return Err(MachineConfig(InvalidMaxVcpuCount(max_vcpu_count)));
}
if max_vcpu_from_topo != max_vcpu_count {
max_vcpu_count = max_vcpu_from_topo;
info!("Since max_vcpu_count is not equal to cpu topo information, we have changed the max vcpu count to {}", max_vcpu_from_topo);
}
config.max_vcpu_count = max_vcpu_count;
config.cpu_pm = machine_config.cpu_pm;
config.mem_type = machine_config.mem_type;
let mem_size_mib_value = machine_config.mem_size_mib;
// Support 1TB memory at most, 2MB aligned for huge page.
if mem_size_mib_value == 0 || mem_size_mib_value > 0x10_0000 || mem_size_mib_value % 2 != 0
{
return Err(MachineConfig(InvalidMemorySize(mem_size_mib_value)));
}
config.mem_size_mib = mem_size_mib_value;
config.mem_file_path = machine_config.mem_file_path.clone();
if config.mem_type == "hugetlbfs" && config.mem_file_path.is_empty() {
return Err(MachineConfig(InvalidMemFilePath("".to_owned())));
}
config.vpmu_feature = machine_config.vpmu_feature;
let vm_id = vm.shared_info().read().unwrap().id.clone();
let serial_path = match machine_config.serial_path {
Some(value) => value,
None => {
if config.serial_path.is_none() {
String::from("/run/dragonball/") + &vm_id + "_com1"
} else {
// Safe to unwrap() because we have checked it has a value.
config.serial_path.as_ref().unwrap().clone()
}
}
};
config.serial_path = Some(serial_path);
vm.set_vm_config(config.clone());
self.machine_config = config;
Ok(VmmData::Empty)
}
#[cfg(feature = "virtio-vsock")]
fn add_vsock_device(&self, vmm: &mut Vmm, config: VsockDeviceConfigInfo) -> VmmRequestResult {
let vm = vmm.get_vm_mut().ok_or(VmmActionError::InvalidVMID)?;
if vm.is_vm_initialized() {
return Err(VmmActionError::Vsock(
VsockDeviceError::UpdateNotAllowedPostBoot,
));
}
// VMADDR_CID_ANY (-1U) means any address for binding;
// VMADDR_CID_HYPERVISOR (0) is reserved for services built into the hypervisor;
// VMADDR_CID_RESERVED (1) must not be used;
// VMADDR_CID_HOST (2) is the well-known address of the host.
if config.guest_cid <= 2 {
return Err(VmmActionError::Vsock(VsockDeviceError::GuestCIDInvalid(
config.guest_cid,
)));
}
info!("add_vsock_device: {:?}", config);
let ctx = vm.create_device_op_context(None).map_err(|e| {
info!("create device op context error: {:?}", e);
VmmActionError::Vsock(VsockDeviceError::UpdateNotAllowedPostBoot)
})?;
vm.device_manager_mut()
.vsock_manager
.insert_device(ctx, config)
.map(|_| VmmData::Empty)
.map_err(VmmActionError::Vsock)
}
#[cfg(feature = "virtio-blk")]
// Only call this function as part of the API.
// If the drive_id does not exist, a new Block Device Config is added to the list.
fn add_block_device(
&mut self,
vmm: &mut Vmm,
event_mgr: &mut EventManager,
config: BlockDeviceConfigInfo,
) -> VmmRequestResult {
let vm = vmm.get_vm_mut().ok_or(VmmActionError::InvalidVMID)?;
let ctx = vm
.create_device_op_context(Some(event_mgr.epoll_manager()))
.map_err(|e| {
if let StartMicroVmError::UpcallNotReady = e {
return VmmActionError::UpcallNotReady;
}
VmmActionError::Block(BlockDeviceError::UpdateNotAllowedPostBoot)
})?;
BlockDeviceMgr::insert_device(vm.device_manager_mut(), ctx, config)
.map(|_| VmmData::Empty)
.map_err(VmmActionError::Block)
}
#[cfg(feature = "virtio-blk")]
/// Updates configuration for an emulated net device as described in `config`.
fn update_blk_rate_limiters(
&mut self,
vmm: &mut Vmm,
config: BlockDeviceConfigUpdateInfo,
) -> VmmRequestResult {
let vm = vmm.get_vm_mut().ok_or(VmmActionError::InvalidVMID)?;
BlockDeviceMgr::update_device_ratelimiters(vm.device_manager_mut(), config)
.map(|_| VmmData::Empty)
.map_err(VmmActionError::Block)
}
#[cfg(feature = "virtio-blk")]
// Only call this function as part of the API.
// If the drive_id does not exist, a new Block Device Config is added to the list.
fn remove_block_device(
&mut self,
vmm: &mut Vmm,
event_mgr: &mut EventManager,
drive_id: &str,
) -> VmmRequestResult {
let vm = vmm.get_vm_mut().ok_or(VmmActionError::InvalidVMID)?;
let ctx = vm
.create_device_op_context(Some(event_mgr.epoll_manager()))
.map_err(|_| VmmActionError::Block(BlockDeviceError::UpdateNotAllowedPostBoot))?;
BlockDeviceMgr::remove_device(vm.device_manager_mut(), ctx, drive_id)
.map(|_| VmmData::Empty)
.map_err(VmmActionError::Block)
}
#[cfg(feature = "virtio-net")]
fn add_virtio_net_device(
&mut self,
vmm: &mut Vmm,
event_mgr: &mut EventManager,
config: VirtioNetDeviceConfigInfo,
) -> VmmRequestResult {
let vm = vmm.get_vm_mut().ok_or(VmmActionError::InvalidVMID)?;
let ctx = vm
.create_device_op_context(Some(event_mgr.epoll_manager()))
.map_err(|e| {
if let StartMicroVmError::MicroVMAlreadyRunning = e {
VmmActionError::VirtioNet(VirtioNetDeviceError::UpdateNotAllowedPostBoot)
} else if let StartMicroVmError::UpcallNotReady = e {
VmmActionError::UpcallNotReady
} else {
VmmActionError::StartMicroVm(e)
}
})?;
VirtioNetDeviceMgr::insert_device(vm.device_manager_mut(), ctx, config)
.map(|_| VmmData::Empty)
.map_err(VmmActionError::VirtioNet)
}
#[cfg(feature = "virtio-net")]
fn update_net_rate_limiters(
&mut self,
vmm: &mut Vmm,
config: VirtioNetDeviceConfigUpdateInfo,
) -> VmmRequestResult {
let vm = vmm.get_vm_mut().ok_or(VmmActionError::InvalidVMID)?;
VirtioNetDeviceMgr::update_device_ratelimiters(vm.device_manager_mut(), config)
.map(|_| VmmData::Empty)
.map_err(VmmActionError::VirtioNet)
}
#[cfg(feature = "virtio-fs")]
fn add_fs_device(&mut self, vmm: &mut Vmm, config: FsDeviceConfigInfo) -> VmmRequestResult {
let vm = vmm.get_vm_mut().ok_or(VmmActionError::InvalidVMID)?;
let hotplug = vm.is_vm_initialized();
if !cfg!(feature = "hotplug") && hotplug {
return Err(VmmActionError::FsDevice(
FsDeviceError::UpdateNotAllowedPostBoot,
));
}
let ctx = vm.create_device_op_context(None).map_err(|e| {
info!("create device op context error: {:?}", e);
VmmActionError::FsDevice(FsDeviceError::UpdateNotAllowedPostBoot)
})?;
FsDeviceMgr::insert_device(vm.device_manager_mut(), ctx, config)
.map(|_| VmmData::Empty)
.map_err(VmmActionError::FsDevice)
}
#[cfg(feature = "virtio-fs")]
fn manipulate_fs_backend_fs(
&self,
vmm: &mut Vmm,
config: FsMountConfigInfo,
) -> VmmRequestResult {
let vm = vmm.get_vm_mut().ok_or(VmmActionError::InvalidVMID)?;
if !vm.is_vm_initialized() {
return Err(VmmActionError::FsDevice(FsDeviceError::MicroVMNotRunning));
}
FsDeviceMgr::manipulate_backend_fs(vm.device_manager_mut(), config)
.map(|_| VmmData::Empty)
.map_err(VmmActionError::FsDevice)
}
#[cfg(feature = "virtio-fs")]
fn update_fs_rate_limiters(
&self,
vmm: &mut Vmm,
config: FsDeviceConfigUpdateInfo,
) -> VmmRequestResult {
let vm = vmm.get_vm_mut().ok_or(VmmActionError::InvalidVMID)?;
if !vm.is_vm_initialized() {
return Err(VmmActionError::FsDevice(FsDeviceError::MicroVMNotRunning));
}
FsDeviceMgr::update_device_ratelimiters(vm.device_manager_mut(), config)
.map(|_| VmmData::Empty)
.map_err(VmmActionError::FsDevice)
}
}
fn handle_cpu_topology(
cpu_topology: &CpuTopology,
vcpu_count: u8,
) -> std::result::Result<&CpuTopology, VmmActionError> {
// Check if dies_per_socket, cores_per_die, threads_per_core and socket number is valid
if cpu_topology.threads_per_core < 1 || cpu_topology.threads_per_core > 2 {
return Err(MachineConfig(InvalidThreadsPerCore(
cpu_topology.threads_per_core,
)));
}
let vcpu_count_from_topo = cpu_topology
.sockets
.checked_mul(cpu_topology.dies_per_socket)
.ok_or(MachineConfig(VcpuCountExceedsMaximum))?
.checked_mul(cpu_topology.cores_per_die)
.ok_or(MachineConfig(VcpuCountExceedsMaximum))?
.checked_mul(cpu_topology.threads_per_core)
.ok_or(MachineConfig(VcpuCountExceedsMaximum))?;
if vcpu_count_from_topo > MAX_SUPPORTED_VCPUS {
return Err(MachineConfig(VcpuCountExceedsMaximum));
}
if vcpu_count_from_topo < vcpu_count {
return Err(MachineConfig(InvalidCpuTopology(vcpu_count_from_topo)));
}
Ok(cpu_topology)
}

View File

@@ -10,6 +10,29 @@ use dbs_device::DeviceIo;
use dbs_utils::rate_limiter::{RateLimiter, TokenBucket};
use serde_derive::{Deserialize, Serialize};
/// Get bucket update for rate limiter.
#[macro_export]
macro_rules! get_bucket_update {
($self:ident, $rate_limiter: ident, $metric: ident) => {{
match &$self.$rate_limiter {
Some(rl_cfg) => {
let tb_cfg = &rl_cfg.$metric;
dbs_utils::rate_limiter::RateLimiter::make_bucket(
tb_cfg.size,
tb_cfg.one_time_burst,
tb_cfg.refill_time,
)
// Updated active rate-limiter.
.map(dbs_utils::rate_limiter::BucketUpdate::Update)
// Updated/deactivated rate-limiter
.unwrap_or(dbs_utils::rate_limiter::BucketUpdate::Disabled)
}
// No update to the rate-limiter.
None => dbs_utils::rate_limiter::BucketUpdate::None,
}
}};
}
/// Trait for generic configuration information.
pub trait ConfigItem {
/// Related errors.
@@ -178,6 +201,15 @@ where
info_list: Vec<DeviceConfigInfo<T>>,
}
impl<T> Default for DeviceConfigInfos<T>
where
T: ConfigItem + Clone,
{
fn default() -> Self {
Self::new()
}
}
impl<T> DeviceConfigInfos<T>
where
T: ConfigItem + Clone,
@@ -221,12 +253,16 @@ where
}
}
#[allow(dead_code)]
/// Get number of device configuration information objects.
pub fn len(&self) -> usize {
self.info_list.len()
}
/// Returns true if the device configuration information objects is empty.
pub fn is_empty(&self) -> bool {
self.info_list.len() == 0
}
/// Add a device configuration information object at the tail.
pub fn push(&mut self, info: DeviceConfigInfo<T>) {
self.info_list.push(info);
@@ -379,7 +415,7 @@ mod tests {
Exist,
}
#[derive(Clone, Debug)]
#[derive(Clone, Debug, Default)]
pub struct DummyConfigInfo {
id: String,
content: String,

View File

@@ -0,0 +1,776 @@
// Copyright 2020-2022 Alibaba, Inc. or its affiliates. All Rights Reserved.
// Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
// SPDX-License-Identifier: Apache-2.0
//
// Portions Copyright 2017 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 THIRD-PARTY file.
//! Device manager for virtio-blk and vhost-user-blk devices.
use std::collections::{vec_deque, VecDeque};
use std::convert::TryInto;
use std::fs::OpenOptions;
use std::os::unix::fs::OpenOptionsExt;
use std::os::unix::io::AsRawFd;
use std::path::{Path, PathBuf};
use std::sync::Arc;
use dbs_virtio_devices as virtio;
use dbs_virtio_devices::block::{aio::Aio, io_uring::IoUring, Block, LocalFile, Ufile};
use serde_derive::{Deserialize, Serialize};
use crate::address_space_manager::GuestAddressSpaceImpl;
use crate::config_manager::{ConfigItem, DeviceConfigInfo, RateLimiterConfigInfo};
use crate::device_manager::blk_dev_mgr::BlockDeviceError::InvalidDeviceId;
use crate::device_manager::{DeviceManager, DeviceMgrError, DeviceOpContext};
use crate::get_bucket_update;
use crate::vm::KernelConfigInfo;
use super::DbsMmioV2Device;
// The flag of whether to use the shared irq.
const USE_SHARED_IRQ: bool = true;
// The flag of whether to use the generic irq.
const USE_GENERIC_IRQ: bool = true;
macro_rules! info(
($l:expr, $($args:tt)+) => {
slog::info!($l, $($args)+; slog::o!("subsystem" => "block_manager"))
};
);
macro_rules! error(
($l:expr, $($args:tt)+) => {
slog::error!($l, $($args)+; slog::o!("subsystem" => "block_manager"))
};
);
/// Default queue size for VirtIo block devices.
pub const QUEUE_SIZE: u16 = 128;
/// Errors associated with the operations allowed on a drive.
#[derive(Debug, thiserror::Error)]
pub enum BlockDeviceError {
/// Invalid VM instance ID.
#[error("invalid VM instance id")]
InvalidVMID,
/// The block device path is invalid.
#[error("invalid block device path '{0}'")]
InvalidBlockDevicePath(PathBuf),
/// The block device type is invalid.
#[error("invalid block device type")]
InvalidBlockDeviceType,
/// The block device path was already used for a different drive.
#[error("block device path '{0}' already exists")]
BlockDevicePathAlreadyExists(PathBuf),
/// The device id doesn't exist.
#[error("invalid block device id '{0}'")]
InvalidDeviceId(String),
/// Cannot perform the requested operation after booting the microVM.
#[error("block device does not support runtime update")]
UpdateNotAllowedPostBoot,
/// A root block device was already added.
#[error("could not add multiple virtual machine root devices")]
RootBlockDeviceAlreadyAdded,
/// Failed to send patch message to block epoll handler.
#[error("could not send patch message to the block epoll handler")]
BlockEpollHanderSendFail,
/// Failure from device manager,
#[error("device manager errors: {0}")]
DeviceManager(#[from] DeviceMgrError),
/// Failure from virtio subsystem.
#[error(transparent)]
Virtio(virtio::Error),
/// Unable to seek the block device backing file due to invalid permissions or
/// the file was deleted/corrupted.
#[error("cannot create block device: {0}")]
CreateBlockDevice(#[source] virtio::Error),
/// Cannot open the block device backing file.
#[error("cannot open the block device backing file: {0}")]
OpenBlockDevice(#[source] std::io::Error),
/// Cannot initialize a MMIO Block Device or add a device to the MMIO Bus.
#[error("failure while registering block device: {0}")]
RegisterBlockDevice(#[source] DeviceMgrError),
}
/// Type of low level storage device/protocol for virtio-blk devices.
#[derive(Clone, Copy, Debug, PartialEq, Serialize, Deserialize)]
pub enum BlockDeviceType {
/// Unknown low level device type.
Unknown,
/// Vhost-user-blk based low level device.
/// SPOOL is a reliable NVMe virtualization system for the cloud environment.
/// You could learn more SPOOL here: https://www.usenix.org/conference/atc20/presentation/xue
Spool,
/// Local disk/file based low level device.
RawBlock,
}
impl BlockDeviceType {
/// Get type of low level storage device/protocol by parsing `path`.
pub fn get_type(path: &str) -> BlockDeviceType {
// SPOOL path should be started with "spool", e.g. "spool:/device1"
if path.starts_with("spool:/") {
BlockDeviceType::Spool
} else {
BlockDeviceType::RawBlock
}
}
}
/// Configuration information for a block device.
#[derive(Clone, Debug, Deserialize, PartialEq, Serialize)]
pub struct BlockDeviceConfigUpdateInfo {
/// Unique identifier of the drive.
pub drive_id: String,
/// Rate Limiter for I/O operations.
pub rate_limiter: Option<RateLimiterConfigInfo>,
}
impl BlockDeviceConfigUpdateInfo {
/// Provides a `BucketUpdate` description for the bandwidth rate limiter.
pub fn bytes(&self) -> dbs_utils::rate_limiter::BucketUpdate {
get_bucket_update!(self, rate_limiter, bandwidth)
}
/// Provides a `BucketUpdate` description for the ops rate limiter.
pub fn ops(&self) -> dbs_utils::rate_limiter::BucketUpdate {
get_bucket_update!(self, rate_limiter, ops)
}
}
/// Configuration information for a block device.
#[derive(Clone, Debug, Deserialize, PartialEq, Serialize)]
pub struct BlockDeviceConfigInfo {
/// Unique identifier of the drive.
pub drive_id: String,
/// Type of low level storage/protocol.
pub device_type: BlockDeviceType,
/// Path of the drive.
pub path_on_host: PathBuf,
/// If set to true, it makes the current device the root block device.
/// Setting this flag to true will mount the block device in the
/// guest under /dev/vda unless the part_uuid is present.
pub is_root_device: bool,
/// Part-UUID. Represents the unique id of the boot partition of this device.
/// It is optional and it will be used only if the `is_root_device` field is true.
pub part_uuid: Option<String>,
/// If set to true, the drive is opened in read-only mode. Otherwise, the
/// drive is opened as read-write.
pub is_read_only: bool,
/// If set to false, the drive is opened with buffered I/O mode. Otherwise, the
/// drive is opened with direct I/O mode.
pub is_direct: bool,
/// Don't close `path_on_host` file when dropping the device.
pub no_drop: bool,
/// Block device multi-queue
pub num_queues: usize,
/// Virtio queue size. Size: byte
pub queue_size: u16,
/// Rate Limiter for I/O operations.
pub rate_limiter: Option<RateLimiterConfigInfo>,
/// Use shared irq
pub use_shared_irq: Option<bool>,
/// Use generic irq
pub use_generic_irq: Option<bool>,
}
impl std::default::Default for BlockDeviceConfigInfo {
fn default() -> Self {
Self {
drive_id: String::default(),
device_type: BlockDeviceType::RawBlock,
path_on_host: PathBuf::default(),
is_root_device: false,
part_uuid: None,
is_read_only: false,
is_direct: Self::default_direct(),
no_drop: Self::default_no_drop(),
num_queues: Self::default_num_queues(),
queue_size: 256,
rate_limiter: None,
use_shared_irq: None,
use_generic_irq: None,
}
}
}
impl BlockDeviceConfigInfo {
/// Get default queue numbers
pub fn default_num_queues() -> usize {
1
}
/// Get default value of is_direct switch
pub fn default_direct() -> bool {
true
}
/// Get default value of no_drop switch
pub fn default_no_drop() -> bool {
false
}
/// Get type of low level storage/protocol.
pub fn device_type(&self) -> BlockDeviceType {
self.device_type
}
/// Returns a reference to `path_on_host`.
pub fn path_on_host(&self) -> &PathBuf {
&self.path_on_host
}
/// Returns a reference to the part_uuid.
pub fn get_part_uuid(&self) -> Option<&String> {
self.part_uuid.as_ref()
}
/// Checks whether the drive had read only permissions.
pub fn is_read_only(&self) -> bool {
self.is_read_only
}
/// Checks whether the drive uses direct I/O
pub fn is_direct(&self) -> bool {
self.is_direct
}
/// Get number and size of queues supported.
pub fn queue_sizes(&self) -> Vec<u16> {
(0..self.num_queues)
.map(|_| self.queue_size)
.collect::<Vec<u16>>()
}
}
impl ConfigItem for BlockDeviceConfigInfo {
type Err = BlockDeviceError;
fn id(&self) -> &str {
&self.drive_id
}
fn check_conflicts(&self, other: &Self) -> Result<(), BlockDeviceError> {
if self.drive_id == other.drive_id {
Ok(())
} else if self.path_on_host == other.path_on_host {
Err(BlockDeviceError::BlockDevicePathAlreadyExists(
self.path_on_host.clone(),
))
} else {
Ok(())
}
}
}
impl std::fmt::Debug for BlockDeviceInfo {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
write!(f, "{:?}", self.config)
}
}
/// Block Device Info
pub type BlockDeviceInfo = DeviceConfigInfo<BlockDeviceConfigInfo>;
/// Wrapper for the collection that holds all the Block Devices Configs
//#[derive(Clone, Debug, Deserialize, PartialEq, Serialize)]
#[derive(Clone)]
pub struct BlockDeviceMgr {
/// A list of `BlockDeviceInfo` objects.
info_list: VecDeque<BlockDeviceInfo>,
has_root_block: bool,
has_part_uuid_root: bool,
read_only_root: bool,
part_uuid: Option<String>,
use_shared_irq: bool,
}
impl BlockDeviceMgr {
/// returns a front-to-back iterator.
pub fn iter(&self) -> vec_deque::Iter<BlockDeviceInfo> {
self.info_list.iter()
}
/// Checks whether any of the added BlockDevice is the root.
pub fn has_root_block_device(&self) -> bool {
self.has_root_block
}
/// Checks whether the root device is configured using a part UUID.
pub fn has_part_uuid_root(&self) -> bool {
self.has_part_uuid_root
}
/// Checks whether the root device has read-only permisssions.
pub fn is_read_only_root(&self) -> bool {
self.read_only_root
}
/// Gets the index of the device with the specified `drive_id` if it exists in the list.
pub fn get_index_of_drive_id(&self, id: &str) -> Option<usize> {
self.info_list
.iter()
.position(|info| info.config.id().eq(id))
}
/// Gets the 'BlockDeviceConfigInfo' of the device with the specified `drive_id` if it exists in the list.
pub fn get_config_of_drive_id(&self, drive_id: &str) -> Option<BlockDeviceConfigInfo> {
match self.get_index_of_drive_id(drive_id) {
Some(index) => {
let config = self.info_list.get(index).unwrap().config.clone();
Some(config)
}
None => None,
}
}
/// Inserts `block_device_config` in the block device configuration list.
/// If an entry with the same id already exists, it will attempt to update
/// the existing entry.
/// Inserting a secondary root block device will fail.
pub fn insert_device(
device_mgr: &mut DeviceManager,
mut ctx: DeviceOpContext,
config: BlockDeviceConfigInfo,
) -> std::result::Result<(), BlockDeviceError> {
if !cfg!(feature = "hotplug") && ctx.is_hotplug {
return Err(BlockDeviceError::UpdateNotAllowedPostBoot);
}
let mgr = &mut device_mgr.block_manager;
// If the id of the drive already exists in the list, the operation is update.
match mgr.get_index_of_drive_id(config.id()) {
Some(index) => {
// No support for runtime update yet.
if ctx.is_hotplug {
Err(BlockDeviceError::BlockDevicePathAlreadyExists(
config.path_on_host.clone(),
))
} else {
for (idx, info) in mgr.info_list.iter().enumerate() {
if idx != index {
info.config.check_conflicts(&config)?;
}
}
mgr.update(index, config)
}
}
None => {
for info in mgr.info_list.iter() {
info.config.check_conflicts(&config)?;
}
let config2 = config.clone();
let index = mgr.create(config2)?;
if !ctx.is_hotplug {
return Ok(());
}
match config.device_type {
BlockDeviceType::RawBlock => {
let device = Self::create_blk_device(&config, &mut ctx)
.map_err(BlockDeviceError::Virtio)?;
let dev = DeviceManager::create_mmio_virtio_device(
device,
&mut ctx,
config.use_shared_irq.unwrap_or(mgr.use_shared_irq),
config.use_generic_irq.unwrap_or(USE_GENERIC_IRQ),
)
.map_err(BlockDeviceError::DeviceManager)?;
mgr.update_device_by_index(index, Arc::clone(&dev))?;
// live-upgrade need save/restore device from info.device.
mgr.info_list[index].set_device(dev.clone());
ctx.insert_hotplug_mmio_device(&dev, None).map_err(|e| {
let logger = ctx.logger().new(slog::o!());
BlockDeviceMgr::remove_device(device_mgr, ctx, &config.drive_id)
.unwrap();
error!(
logger,
"failed to hot-add virtio block device {}, {:?}",
&config.drive_id,
e
);
BlockDeviceError::DeviceManager(e)
})
}
_ => Err(BlockDeviceError::InvalidBlockDeviceType),
}
}
}
}
/// Attaches all block devices from the BlockDevicesConfig.
pub fn attach_devices(
&mut self,
ctx: &mut DeviceOpContext,
) -> std::result::Result<(), BlockDeviceError> {
for info in self.info_list.iter_mut() {
match info.config.device_type {
BlockDeviceType::RawBlock => {
info!(
ctx.logger(),
"attach virtio-blk device, drive_id {}, path {}",
info.config.drive_id,
info.config.path_on_host.to_str().unwrap_or("<unknown>")
);
let device = Self::create_blk_device(&info.config, ctx)
.map_err(BlockDeviceError::Virtio)?;
let device = DeviceManager::create_mmio_virtio_device(
device,
ctx,
info.config.use_shared_irq.unwrap_or(self.use_shared_irq),
info.config.use_generic_irq.unwrap_or(USE_GENERIC_IRQ),
)
.map_err(BlockDeviceError::RegisterBlockDevice)?;
info.device = Some(device);
}
_ => {
return Err(BlockDeviceError::OpenBlockDevice(
std::io::Error::from_raw_os_error(libc::EINVAL),
));
}
}
}
Ok(())
}
/// Removes all virtio-blk devices
pub fn remove_devices(&mut self, ctx: &mut DeviceOpContext) -> Result<(), DeviceMgrError> {
while let Some(mut info) = self.info_list.pop_back() {
info!(ctx.logger(), "remove drive {}", info.config.drive_id);
if let Some(device) = info.device.take() {
DeviceManager::destroy_mmio_virtio_device(device, ctx)?;
}
}
Ok(())
}
fn remove(&mut self, drive_id: &str) -> Option<BlockDeviceInfo> {
match self.get_index_of_drive_id(drive_id) {
Some(index) => self.info_list.remove(index),
None => None,
}
}
/// remove a block device, it basically is the inverse operation of `insert_device``
pub fn remove_device(
dev_mgr: &mut DeviceManager,
mut ctx: DeviceOpContext,
drive_id: &str,
) -> std::result::Result<(), BlockDeviceError> {
if !cfg!(feature = "hotplug") {
return Err(BlockDeviceError::UpdateNotAllowedPostBoot);
}
let mgr = &mut dev_mgr.block_manager;
match mgr.remove(drive_id) {
Some(mut info) => {
info!(ctx.logger(), "remove drive {}", info.config.drive_id);
if let Some(device) = info.device.take() {
DeviceManager::destroy_mmio_virtio_device(device, &mut ctx)
.map_err(BlockDeviceError::DeviceManager)?;
}
}
None => return Err(BlockDeviceError::InvalidDeviceId(drive_id.to_owned())),
}
Ok(())
}
fn create_blk_device(
cfg: &BlockDeviceConfigInfo,
ctx: &mut DeviceOpContext,
) -> std::result::Result<Box<Block<GuestAddressSpaceImpl>>, virtio::Error> {
let epoll_mgr = ctx.epoll_mgr.clone().ok_or(virtio::Error::InvalidInput)?;
// Safe to unwrap() because we have verified it when parsing device type.
//let path = cfg.path_on_host.to_str().unwrap();
let mut block_files: Vec<Box<dyn Ufile>> = vec![];
match cfg.device_type {
BlockDeviceType::RawBlock => {
let custom_flags = if cfg.is_direct() {
info!(
ctx.logger(),
"Open block device \"{}\" in direct mode.",
cfg.path_on_host().display()
);
libc::O_DIRECT
} else {
info!(
ctx.logger(),
"Open block device \"{}\" in buffer mode.",
cfg.path_on_host().display(),
);
0
};
let io_uring_supported = IoUring::is_supported();
for i in 0..cfg.num_queues {
let queue_size = cfg.queue_sizes()[i] as u32;
let file = OpenOptions::new()
.read(true)
.custom_flags(custom_flags)
.write(!cfg.is_read_only())
.open(cfg.path_on_host())?;
info!(ctx.logger(), "Queue {}: block file opened", i);
if io_uring_supported {
info!(
ctx.logger(),
"Queue {}: Using io_uring Raw disk file, queue size {}.", i, queue_size
);
let io_engine = IoUring::new(file.as_raw_fd(), queue_size)?;
block_files.push(Box::new(LocalFile::new(file, cfg.no_drop, io_engine)?));
} else {
info!(
ctx.logger(),
"Queue {}: Since io_uring_supported is not enabled, change to default support of Aio Raw disk file, queue size {}", i, queue_size
);
let io_engine = Aio::new(file.as_raw_fd(), queue_size)?;
block_files.push(Box::new(LocalFile::new(file, cfg.no_drop, io_engine)?));
}
}
}
_ => {
error!(
ctx.logger(),
"invalid block device type: {:?}", cfg.device_type
);
return Err(virtio::Error::InvalidInput);
}
};
let mut limiters = vec![];
for _i in 0..cfg.num_queues {
if let Some(limiter) = cfg.rate_limiter.clone().map(|mut v| {
v.resize(cfg.num_queues as u64);
v.try_into().unwrap()
}) {
limiters.push(limiter);
}
}
Ok(Box::new(Block::new(
block_files,
cfg.is_read_only,
Arc::new(cfg.queue_sizes()),
epoll_mgr,
limiters,
)?))
}
/// Generated guest kernel commandline related to root block device.
pub fn generate_kernel_boot_args(
&self,
kernel_config: &mut KernelConfigInfo,
) -> std::result::Result<(), DeviceMgrError> {
// Respect user configuration if kernel_cmdline contains "root=",
// special attention for the case when kernel command line starting with "root=xxx"
let old_kernel_cmdline = format!(" {}", kernel_config.kernel_cmdline().as_str());
if !old_kernel_cmdline.contains(" root=") && self.has_root_block {
let cmdline = kernel_config.kernel_cmdline_mut();
if let Some(ref uuid) = self.part_uuid {
cmdline
.insert("root", &format!("PART_UUID={}", uuid))
.map_err(DeviceMgrError::Cmdline)?;
} else {
cmdline
.insert("root", "/dev/vda")
.map_err(DeviceMgrError::Cmdline)?;
}
if self.read_only_root {
if old_kernel_cmdline.contains(" rw") {
return Err(DeviceMgrError::InvalidOperation);
}
cmdline.insert_str("ro").map_err(DeviceMgrError::Cmdline)?;
}
}
Ok(())
}
/// insert a block device's config. return index on success.
fn create(
&mut self,
block_device_config: BlockDeviceConfigInfo,
) -> std::result::Result<usize, BlockDeviceError> {
self.check_data_file_present(&block_device_config)?;
if self
.get_index_of_drive_path(&block_device_config.path_on_host)
.is_some()
{
return Err(BlockDeviceError::BlockDevicePathAlreadyExists(
block_device_config.path_on_host,
));
}
// check whether the Device Config belongs to a root device
// we need to satisfy the condition by which a VMM can only have on root device
if block_device_config.is_root_device {
if self.has_root_block {
return Err(BlockDeviceError::RootBlockDeviceAlreadyAdded);
} else {
self.has_root_block = true;
self.read_only_root = block_device_config.is_read_only;
self.has_part_uuid_root = block_device_config.part_uuid.is_some();
self.part_uuid = block_device_config.part_uuid.clone();
// Root Device should be the first in the list whether or not PART_UUID is specified
// in order to avoid bugs in case of switching from part_uuid boot scenarios to
// /dev/vda boot type.
self.info_list
.push_front(BlockDeviceInfo::new(block_device_config));
Ok(0)
}
} else {
self.info_list
.push_back(BlockDeviceInfo::new(block_device_config));
Ok(self.info_list.len() - 1)
}
}
/// Updates a Block Device Config. The update fails if it would result in two
/// root block devices.
fn update(
&mut self,
mut index: usize,
new_config: BlockDeviceConfigInfo,
) -> std::result::Result<(), BlockDeviceError> {
// Check if the path exists
self.check_data_file_present(&new_config)?;
if let Some(idx) = self.get_index_of_drive_path(&new_config.path_on_host) {
if idx != index {
return Err(BlockDeviceError::BlockDevicePathAlreadyExists(
new_config.path_on_host.clone(),
));
}
}
if self.info_list.get(index).is_none() {
return Err(InvalidDeviceId(index.to_string()));
}
// Check if the root block device is being updated.
if self.info_list[index].config.is_root_device {
self.has_root_block = new_config.is_root_device;
self.read_only_root = new_config.is_root_device && new_config.is_read_only;
self.has_part_uuid_root = new_config.part_uuid.is_some();
self.part_uuid = new_config.part_uuid.clone();
} else if new_config.is_root_device {
// Check if a second root block device is being added.
if self.has_root_block {
return Err(BlockDeviceError::RootBlockDeviceAlreadyAdded);
} else {
// One of the non-root blocks is becoming root.
self.has_root_block = true;
self.read_only_root = new_config.is_read_only;
self.has_part_uuid_root = new_config.part_uuid.is_some();
self.part_uuid = new_config.part_uuid.clone();
// Make sure the root device is on the first position.
self.info_list.swap(0, index);
// Block config to be updated has moved to first position.
index = 0;
}
}
// Update the config.
self.info_list[index].config = new_config;
Ok(())
}
fn check_data_file_present(
&self,
block_device_config: &BlockDeviceConfigInfo,
) -> std::result::Result<(), BlockDeviceError> {
if block_device_config.device_type == BlockDeviceType::RawBlock
&& !block_device_config.path_on_host.exists()
{
Err(BlockDeviceError::InvalidBlockDevicePath(
block_device_config.path_on_host.clone(),
))
} else {
Ok(())
}
}
fn get_index_of_drive_path(&self, drive_path: &Path) -> Option<usize> {
self.info_list
.iter()
.position(|info| info.config.path_on_host.eq(drive_path))
}
/// update devce information in `info_list`. The caller of this method is
/// `insert_device` when hotplug is true.
pub fn update_device_by_index(
&mut self,
index: usize,
device: Arc<DbsMmioV2Device>,
) -> Result<(), BlockDeviceError> {
if let Some(info) = self.info_list.get_mut(index) {
info.device = Some(device);
return Ok(());
}
Err(BlockDeviceError::InvalidDeviceId("".to_owned()))
}
/// Update the ratelimiter settings of a virtio blk device.
pub fn update_device_ratelimiters(
device_mgr: &mut DeviceManager,
new_cfg: BlockDeviceConfigUpdateInfo,
) -> std::result::Result<(), BlockDeviceError> {
let mgr = &mut device_mgr.block_manager;
match mgr.get_index_of_drive_id(&new_cfg.drive_id) {
Some(index) => {
let config = &mut mgr.info_list[index].config;
config.rate_limiter = new_cfg.rate_limiter.clone();
let device = mgr.info_list[index]
.device
.as_mut()
.ok_or_else(|| BlockDeviceError::InvalidDeviceId("".to_owned()))?;
if let Some(mmio_dev) = device.as_any().downcast_ref::<DbsMmioV2Device>() {
let guard = mmio_dev.state();
let inner_dev = guard.get_inner_device();
if let Some(blk_dev) = inner_dev
.as_any()
.downcast_ref::<virtio::block::Block<GuestAddressSpaceImpl>>()
{
return blk_dev
.set_patch_rate_limiters(new_cfg.bytes(), new_cfg.ops())
.map(|_p| ())
.map_err(|_e| BlockDeviceError::BlockEpollHanderSendFail);
}
}
Ok(())
}
None => Err(BlockDeviceError::InvalidDeviceId(new_cfg.drive_id)),
}
}
}
impl Default for BlockDeviceMgr {
/// Constructor for the BlockDeviceConfigs. It initializes an empty LinkedList.
fn default() -> BlockDeviceMgr {
BlockDeviceMgr {
info_list: VecDeque::<BlockDeviceInfo>::new(),
has_root_block: false,
has_part_uuid_root: false,
read_only_root: false,
part_uuid: None,
use_shared_irq: USE_SHARED_IRQ,
}
}
}

View File

@@ -351,6 +351,16 @@ pub struct DmesgWriter {
logger: slog::Logger,
}
impl DmesgWriter {
/// Creates a new instance.
pub fn new(logger: &slog::Logger) -> Self {
Self {
buf: BytesMut::with_capacity(1024),
logger: logger.new(slog::o!("subsystem" => "dmesg")),
}
}
}
impl io::Write for DmesgWriter {
/// 0000000 [ 0 . 0 3 4 9 1 6 ] R
/// 5b 20 20 20 20 30 2e 30 33 34 39 31 36 5d 20 52
@@ -416,13 +426,13 @@ mod tests {
};
writer.flush().unwrap();
writer.write("".as_bytes()).unwrap();
writer.write("\n".as_bytes()).unwrap();
writer.write("\n\n".as_bytes()).unwrap();
writer.write("\n\n\n".as_bytes()).unwrap();
writer.write("12\n23\n34\n56".as_bytes()).unwrap();
writer.write("78".as_bytes()).unwrap();
writer.write("90\n".as_bytes()).unwrap();
writer.write_all("".as_bytes()).unwrap();
writer.write_all("\n".as_bytes()).unwrap();
writer.write_all("\n\n".as_bytes()).unwrap();
writer.write_all("\n\n\n".as_bytes()).unwrap();
writer.write_all("12\n23\n34\n56".as_bytes()).unwrap();
writer.write_all("78".as_bytes()).unwrap();
writer.write_all("90\n".as_bytes()).unwrap();
writer.flush().unwrap();
}

View File

@@ -0,0 +1,528 @@
// Copyright 2020-2022 Alibaba Cloud. All Rights Reserved.
// Copyright 2019 Intel Corporation. All Rights Reserved.
//
// SPDX-License-Identifier: Apache-2.0
use std::convert::TryInto;
use dbs_utils::epoll_manager::EpollManager;
use dbs_virtio_devices::{self as virtio, Error as VirtIoError};
use serde_derive::{Deserialize, Serialize};
use slog::{error, info};
use crate::address_space_manager::GuestAddressSpaceImpl;
use crate::config_manager::{
ConfigItem, DeviceConfigInfo, DeviceConfigInfos, RateLimiterConfigInfo,
};
use crate::device_manager::{
DbsMmioV2Device, DeviceManager, DeviceMgrError, DeviceOpContext, DeviceVirtioRegionHandler,
};
use crate::get_bucket_update;
use super::DbsVirtioDevice;
// The flag of whether to use the shared irq.
const USE_SHARED_IRQ: bool = true;
// The flag of whether to use the generic irq.
const USE_GENERIC_IRQ: bool = true;
// Default cache size is 2 Gi since this is a typical VM memory size.
const DEFAULT_CACHE_SIZE: u64 = 2 * 1024 * 1024 * 1024;
// We have 2 supported fs device mode, vhostuser and virtio
const VHOSTUSER_FS_MODE: &str = "vhostuser";
// We have 2 supported fs device mode, vhostuser and virtio
const VIRTIO_FS_MODE: &str = "virtio";
/// Errors associated with `FsDeviceConfig`.
#[derive(Debug, thiserror::Error)]
pub enum FsDeviceError {
/// Invalid fs, "virtio" or "vhostuser" is allowed.
#[error("the fs type is invalid, virtio or vhostuser is allowed")]
InvalidFs,
/// Cannot access address space.
#[error("Cannot access address space.")]
AddressSpaceNotInitialized,
/// Cannot convert RateLimterConfigInfo into RateLimiter.
#[error("failure while converting RateLimterConfigInfo into RateLimiter: {0}")]
RateLimterConfigInfoTryInto(#[source] std::io::Error),
/// The fs device tag was already used for a different fs.
#[error("VirtioFs device tag {0} already exists")]
FsDeviceTagAlreadyExists(String),
/// The fs device path was already used for a different fs.
#[error("VirtioFs device tag {0} already exists")]
FsDevicePathAlreadyExists(String),
/// The update is not allowed after booting the microvm.
#[error("update operation is not allowed after boot")]
UpdateNotAllowedPostBoot,
/// The attachbackendfs operation fails.
#[error("Fs device attach a backend fs failed")]
AttachBackendFailed(String),
/// attach backend fs must be done when vm is running.
#[error("vm is not running when attaching a backend fs")]
MicroVMNotRunning,
/// The mount tag doesn't exist.
#[error("fs tag'{0}' doesn't exist")]
TagNotExists(String),
/// Failed to send patch message to VirtioFs epoll handler.
#[error("could not send patch message to the VirtioFs epoll handler")]
VirtioFsEpollHanderSendFail,
/// Creating a shared-fs device fails (if the vhost-user socket cannot be open.)
#[error("cannot create shared-fs device: {0}")]
CreateFsDevice(#[source] VirtIoError),
/// Cannot initialize a shared-fs device or add a device to the MMIO Bus.
#[error("failure while registering shared-fs device: {0}")]
RegisterFsDevice(#[source] DeviceMgrError),
/// The device manager errors.
#[error("DeviceManager error: {0}")]
DeviceManager(#[source] DeviceMgrError),
}
/// Configuration information for a vhost-user-fs device.
#[derive(Clone, Debug, Deserialize, PartialEq, Serialize)]
pub struct FsDeviceConfigInfo {
/// vhost-user socket path.
pub sock_path: String,
/// virtiofs mount tag name used inside the guest.
/// used as the device name during mount.
pub tag: String,
/// Number of virtqueues to use.
pub num_queues: usize,
/// Size of each virtqueue. Unit: byte.
pub queue_size: u16,
/// DAX cache window size
pub cache_size: u64,
/// Number of thread pool workers.
pub thread_pool_size: u16,
/// The caching policy the file system should use (auto, always or never).
/// This cache policy is set for virtio-fs, visit https://gitlab.com/virtio-fs/virtiofsd to get further information.
pub cache_policy: String,
/// Writeback cache
pub writeback_cache: bool,
/// Enable no_open or not
pub no_open: bool,
/// Enable xattr or not
pub xattr: bool,
/// Drop CAP_SYS_RESOURCE or not
pub drop_sys_resource: bool,
/// virtio fs or vhostuser fs.
pub mode: String,
/// Enable kill_priv_v2 or not
pub fuse_killpriv_v2: bool,
/// Enable no_readdir or not
pub no_readdir: bool,
/// Rate Limiter for I/O operations.
pub rate_limiter: Option<RateLimiterConfigInfo>,
/// Use shared irq
pub use_shared_irq: Option<bool>,
/// Use generic irq
pub use_generic_irq: Option<bool>,
}
impl std::default::Default for FsDeviceConfigInfo {
fn default() -> Self {
Self {
sock_path: String::default(),
tag: String::default(),
num_queues: 1,
queue_size: 1024,
cache_size: DEFAULT_CACHE_SIZE,
thread_pool_size: 0,
cache_policy: Self::default_cache_policy(),
writeback_cache: Self::default_writeback_cache(),
no_open: Self::default_no_open(),
fuse_killpriv_v2: Self::default_fuse_killpriv_v2(),
no_readdir: Self::default_no_readdir(),
xattr: Self::default_xattr(),
drop_sys_resource: Self::default_drop_sys_resource(),
mode: Self::default_fs_mode(),
rate_limiter: Some(RateLimiterConfigInfo::default()),
use_shared_irq: None,
use_generic_irq: None,
}
}
}
impl FsDeviceConfigInfo {
/// The default mode is set to 'virtio' for 'virtio-fs' device.
pub fn default_fs_mode() -> String {
String::from(VIRTIO_FS_MODE)
}
/// The default cache policy
pub fn default_cache_policy() -> String {
"always".to_string()
}
/// The default setting of writeback cache
pub fn default_writeback_cache() -> bool {
true
}
/// The default setting of no_open
pub fn default_no_open() -> bool {
true
}
/// The default setting of killpriv_v2
pub fn default_fuse_killpriv_v2() -> bool {
false
}
/// The default setting of xattr
pub fn default_xattr() -> bool {
false
}
/// The default setting of drop_sys_resource
pub fn default_drop_sys_resource() -> bool {
false
}
/// The default setting of no_readdir
pub fn default_no_readdir() -> bool {
false
}
/// The default setting of rate limiter
pub fn default_fs_rate_limiter() -> Option<RateLimiterConfigInfo> {
None
}
}
/// Configuration information for virtio-fs.
#[derive(Clone, Debug, Deserialize, PartialEq, Serialize)]
pub struct FsDeviceConfigUpdateInfo {
/// virtiofs mount tag name used inside the guest.
/// used as the device name during mount.
pub tag: String,
/// Rate Limiter for I/O operations.
pub rate_limiter: Option<RateLimiterConfigInfo>,
}
impl FsDeviceConfigUpdateInfo {
/// Provides a `BucketUpdate` description for the bandwidth rate limiter.
pub fn bytes(&self) -> dbs_utils::rate_limiter::BucketUpdate {
get_bucket_update!(self, rate_limiter, bandwidth)
}
/// Provides a `BucketUpdate` description for the ops rate limiter.
pub fn ops(&self) -> dbs_utils::rate_limiter::BucketUpdate {
get_bucket_update!(self, rate_limiter, ops)
}
}
impl ConfigItem for FsDeviceConfigInfo {
type Err = FsDeviceError;
fn id(&self) -> &str {
&self.tag
}
fn check_conflicts(&self, other: &Self) -> Result<(), FsDeviceError> {
if self.tag == other.tag {
Err(FsDeviceError::FsDeviceTagAlreadyExists(self.tag.clone()))
} else if self.mode.as_str() == VHOSTUSER_FS_MODE && self.sock_path == other.sock_path {
Err(FsDeviceError::FsDevicePathAlreadyExists(
self.sock_path.clone(),
))
} else {
Ok(())
}
}
}
/// Configuration information of manipulating backend fs for a virtiofs device.
#[derive(Clone, Debug, Deserialize, PartialEq, Serialize)]
pub struct FsMountConfigInfo {
/// Mount operations, mount, update, umount
pub ops: String,
/// The backend fs type to mount.
pub fstype: Option<String>,
/// the source file/directory the backend fs points to
pub source: Option<String>,
/// where the backend fs gets mounted
pub mountpoint: String,
/// backend fs config content in json format
pub config: Option<String>,
/// virtiofs mount tag name used inside the guest.
/// used as the device name during mount.
pub tag: String,
/// Path to file that contains file lists that should be prefetched by rafs
pub prefetch_list_path: Option<String>,
/// What size file supports dax
pub dax_threshold_size_kb: Option<u64>,
}
pub(crate) type FsDeviceInfo = DeviceConfigInfo<FsDeviceConfigInfo>;
impl ConfigItem for FsDeviceInfo {
type Err = FsDeviceError;
fn id(&self) -> &str {
&self.config.tag
}
fn check_conflicts(&self, other: &Self) -> Result<(), FsDeviceError> {
if self.config.tag == other.config.tag {
Err(FsDeviceError::FsDeviceTagAlreadyExists(
self.config.tag.clone(),
))
} else if self.config.sock_path == other.config.sock_path {
Err(FsDeviceError::FsDevicePathAlreadyExists(
self.config.sock_path.clone(),
))
} else {
Ok(())
}
}
}
/// Wrapper for the collection that holds all the Fs Devices Configs
pub struct FsDeviceMgr {
/// A list of `FsDeviceConfig` objects.
pub(crate) info_list: DeviceConfigInfos<FsDeviceConfigInfo>,
pub(crate) use_shared_irq: bool,
}
impl FsDeviceMgr {
/// Inserts `fs_cfg` in the shared-fs device configuration list.
pub fn insert_device(
device_mgr: &mut DeviceManager,
ctx: DeviceOpContext,
fs_cfg: FsDeviceConfigInfo,
) -> std::result::Result<(), FsDeviceError> {
// It's too complicated to manage life cycle of shared-fs service process for hotplug.
if ctx.is_hotplug {
error!(
ctx.logger(),
"no support of shared-fs device hotplug";
"subsystem" => "shared-fs",
"tag" => &fs_cfg.tag,
);
return Err(FsDeviceError::UpdateNotAllowedPostBoot);
}
info!(
ctx.logger(),
"add shared-fs device configuration";
"subsystem" => "shared-fs",
"tag" => &fs_cfg.tag,
);
device_mgr
.fs_manager
.lock()
.unwrap()
.info_list
.insert_or_update(&fs_cfg)?;
Ok(())
}
/// Attaches all vhost-user-fs devices from the FsDevicesConfig.
pub fn attach_devices(
&mut self,
ctx: &mut DeviceOpContext,
) -> std::result::Result<(), FsDeviceError> {
let epoll_mgr = ctx
.epoll_mgr
.clone()
.ok_or(FsDeviceError::CreateFsDevice(virtio::Error::InvalidInput))?;
for info in self.info_list.iter_mut() {
let device = Self::create_fs_device(&info.config, ctx, epoll_mgr.clone())?;
let mmio_device = DeviceManager::create_mmio_virtio_device(
device,
ctx,
info.config.use_shared_irq.unwrap_or(self.use_shared_irq),
info.config.use_generic_irq.unwrap_or(USE_GENERIC_IRQ),
)
.map_err(FsDeviceError::RegisterFsDevice)?;
info.set_device(mmio_device);
}
Ok(())
}
fn create_fs_device(
config: &FsDeviceConfigInfo,
ctx: &mut DeviceOpContext,
epoll_mgr: EpollManager,
) -> std::result::Result<DbsVirtioDevice, FsDeviceError> {
match &config.mode as &str {
VIRTIO_FS_MODE => Self::attach_virtio_fs_devices(config, ctx, epoll_mgr),
_ => Err(FsDeviceError::CreateFsDevice(virtio::Error::InvalidInput)),
}
}
fn attach_virtio_fs_devices(
config: &FsDeviceConfigInfo,
ctx: &mut DeviceOpContext,
epoll_mgr: EpollManager,
) -> std::result::Result<DbsVirtioDevice, FsDeviceError> {
info!(
ctx.logger(),
"add virtio-fs device configuration";
"subsystem" => "virito-fs",
"tag" => &config.tag,
"dax_window_size" => &config.cache_size,
);
let limiter = if let Some(rlc) = config.rate_limiter.clone() {
Some(
rlc.try_into()
.map_err(FsDeviceError::RateLimterConfigInfoTryInto)?,
)
} else {
None
};
let vm_as = ctx.get_vm_as().map_err(|e| {
error!(ctx.logger(), "virtio-fs get vm_as error: {:?}", e;
"subsystem" => "virito-fs");
FsDeviceError::DeviceManager(e)
})?;
let address_space = match ctx.address_space.as_ref() {
Some(address_space) => address_space.clone(),
None => {
error!(ctx.logger(), "virtio-fs get address_space error"; "subsystem" => "virito-fs");
return Err(FsDeviceError::AddressSpaceNotInitialized);
}
};
let handler = DeviceVirtioRegionHandler {
vm_as,
address_space,
};
let device = Box::new(
virtio::fs::VirtioFs::new(
&config.tag,
config.num_queues,
config.queue_size,
config.cache_size,
&config.cache_policy,
config.thread_pool_size,
config.writeback_cache,
config.no_open,
config.fuse_killpriv_v2,
config.xattr,
config.drop_sys_resource,
config.no_readdir,
Box::new(handler),
epoll_mgr,
limiter,
)
.map_err(FsDeviceError::CreateFsDevice)?,
);
Ok(device)
}
/// Attach a backend fs to a VirtioFs device or detach a backend
/// fs from a Virtiofs device
pub fn manipulate_backend_fs(
device_mgr: &mut DeviceManager,
config: FsMountConfigInfo,
) -> std::result::Result<(), FsDeviceError> {
let mut found = false;
let mgr = &mut device_mgr.fs_manager.lock().unwrap();
for info in mgr
.info_list
.iter()
.filter(|info| info.config.tag.as_str() == config.tag.as_str())
{
found = true;
if let Some(device) = info.device.as_ref() {
if let Some(mmio_dev) = device.as_any().downcast_ref::<DbsMmioV2Device>() {
let mut guard = mmio_dev.state();
let inner_dev = guard.get_inner_device_mut();
if let Some(virtio_fs_dev) = inner_dev
.as_any_mut()
.downcast_mut::<virtio::fs::VirtioFs<GuestAddressSpaceImpl>>()
{
return virtio_fs_dev
.manipulate_backend_fs(
config.source,
config.fstype,
&config.mountpoint,
config.config,
&config.ops,
config.prefetch_list_path,
config.dax_threshold_size_kb,
)
.map(|_p| ())
.map_err(|e| FsDeviceError::AttachBackendFailed(e.to_string()));
}
}
}
}
if !found {
Err(FsDeviceError::AttachBackendFailed(
"fs tag not found".to_string(),
))
} else {
Ok(())
}
}
/// Gets the index of the device with the specified `tag` if it exists in the list.
pub fn get_index_of_tag(&self, tag: &str) -> Option<usize> {
self.info_list
.iter()
.position(|info| info.config.id().eq(tag))
}
/// Update the ratelimiter settings of a virtio fs device.
pub fn update_device_ratelimiters(
device_mgr: &mut DeviceManager,
new_cfg: FsDeviceConfigUpdateInfo,
) -> std::result::Result<(), FsDeviceError> {
let mgr = &mut device_mgr.fs_manager.lock().unwrap();
match mgr.get_index_of_tag(&new_cfg.tag) {
Some(index) => {
let config = &mut mgr.info_list[index].config;
config.rate_limiter = new_cfg.rate_limiter.clone();
let device = mgr.info_list[index]
.device
.as_mut()
.ok_or_else(|| FsDeviceError::TagNotExists("".to_owned()))?;
if let Some(mmio_dev) = device.as_any().downcast_ref::<DbsMmioV2Device>() {
let guard = mmio_dev.state();
let inner_dev = guard.get_inner_device();
if let Some(fs_dev) = inner_dev
.as_any()
.downcast_ref::<virtio::fs::VirtioFs<GuestAddressSpaceImpl>>()
{
return fs_dev
.set_patch_rate_limiters(new_cfg.bytes(), new_cfg.ops())
.map(|_p| ())
.map_err(|_e| FsDeviceError::VirtioFsEpollHanderSendFail);
}
}
Ok(())
}
None => Err(FsDeviceError::TagNotExists(new_cfg.tag)),
}
}
}
impl Default for FsDeviceMgr {
/// Create a new `FsDeviceMgr` object..
fn default() -> Self {
FsDeviceMgr {
info_list: DeviceConfigInfos::new(),
use_shared_irq: USE_SHARED_IRQ,
}
}
}

View File

@@ -0,0 +1,110 @@
// Copyright 2022 Alibaba, Inc. or its affiliates. All Rights Reserved.
//
// SPDX-License-Identifier: Apache-2.0
use std::io;
use std::sync::Arc;
use dbs_address_space::{AddressSpace, AddressSpaceRegion, AddressSpaceRegionType};
use dbs_virtio_devices::{Error as VirtIoError, VirtioRegionHandler};
use log::{debug, error};
use vm_memory::{FileOffset, GuestAddressSpace, GuestMemoryRegion, GuestRegionMmap};
use crate::address_space_manager::GuestAddressSpaceImpl;
/// This struct implements the VirtioRegionHandler trait, which inserts the memory
/// region of the virtio device into vm_as and address_space.
///
/// * After region is inserted into the vm_as, the virtio device can read guest memory
/// data using vm_as.get_slice with GuestAddress.
///
/// * Insert virtio memory into address_space so that the correct guest last address can
/// be found when initializing the e820 table. The e820 table is a table that describes
/// guest memory prepared before the guest startup. we need to config the correct guest
/// memory address and length in the table. The virtio device memory belongs to the MMIO
/// space and does not belong to the Guest Memory space. Therefore, it cannot be configured
/// into the e820 table. When creating AddressSpaceRegion we use
/// AddressSpaceRegionType::ReservedMemory type, in this way, address_space will know that
/// this region a special memory, it will don't put the this memory in e820 table.
///
/// This function relies on the atomic-guest-memory feature. Without this feature enabled, memory
/// regions cannot be inserted into vm_as. Because the insert_region interface of vm_as does
/// not insert regions in place, but returns an array of inserted regions. We need to manually
/// replace this array of regions with vm_as, and that's what atomic-guest-memory feature does.
/// So we rely on the atomic-guest-memory feature here
pub struct DeviceVirtioRegionHandler {
pub(crate) vm_as: GuestAddressSpaceImpl,
pub(crate) address_space: AddressSpace,
}
impl DeviceVirtioRegionHandler {
fn insert_address_space(
&mut self,
region: Arc<GuestRegionMmap>,
) -> std::result::Result<(), VirtIoError> {
let file_offset = match region.file_offset() {
// TODO: use from_arc
Some(f) => Some(FileOffset::new(f.file().try_clone()?, 0)),
None => None,
};
let as_region = Arc::new(AddressSpaceRegion::build(
AddressSpaceRegionType::DAXMemory,
region.start_addr(),
region.size() as u64,
None,
file_offset,
region.flags(),
false,
));
self.address_space.insert_region(as_region).map_err(|e| {
error!("inserting address apace error: {}", e);
// dbs-virtio-devices should not depend on dbs-address-space.
// So here io::Error is used instead of AddressSpaceError directly.
VirtIoError::IOError(io::Error::new(
io::ErrorKind::Other,
format!(
"invalid address space region ({0:#x}, {1:#x})",
region.start_addr().0,
region.len()
),
))
})?;
Ok(())
}
fn insert_vm_as(
&mut self,
region: Arc<GuestRegionMmap>,
) -> std::result::Result<(), VirtIoError> {
let vm_as_new = self.vm_as.memory().insert_region(region).map_err(|e| {
error!(
"DeviceVirtioRegionHandler failed to insert guest memory region: {:?}.",
e
);
VirtIoError::InsertMmap(e)
})?;
// Do not expect poisoned lock here, so safe to unwrap().
self.vm_as.lock().unwrap().replace(vm_as_new);
Ok(())
}
}
impl VirtioRegionHandler for DeviceVirtioRegionHandler {
fn insert_region(
&mut self,
region: Arc<GuestRegionMmap>,
) -> std::result::Result<(), VirtIoError> {
debug!(
"add geust memory region to address_space/vm_as, new region: {:?}",
region
);
self.insert_address_space(region.clone())?;
self.insert_vm_as(region)?;
Ok(())
}
}

View File

@@ -8,6 +8,8 @@ use std::sync::{Arc, Mutex, MutexGuard};
use arc_swap::ArcSwap;
use dbs_address_space::AddressSpace;
#[cfg(target_arch = "aarch64")]
use dbs_arch::{DeviceType, MMIODeviceInfo};
use dbs_device::device_manager::{Error as IoManagerError, IoManager, IoManagerContext};
use dbs_device::resources::Resource;
use dbs_device::DeviceIo;
@@ -34,11 +36,14 @@ use dbs_upcall::{
DevMgrRequest, DevMgrService, MmioDevRequest, UpcallClient, UpcallClientError,
UpcallClientRequest, UpcallClientResponse,
};
#[cfg(feature = "hotplug")]
use dbs_virtio_devices::vsock::backend::VsockInnerConnector;
use crate::address_space_manager::GuestAddressSpaceImpl;
use crate::error::StartMicrovmError;
use crate::error::StartMicroVmError;
use crate::resource_manager::ResourceManager;
use crate::vm::KernelConfigInfo;
use crate::vm::{KernelConfigInfo, Vm};
use crate::IoManagerCached;
/// Virtual machine console device manager.
pub mod console_manager;
@@ -54,6 +59,28 @@ pub mod vsock_dev_mgr;
#[cfg(feature = "virtio-vsock")]
use self::vsock_dev_mgr::VsockDeviceMgr;
#[cfg(feature = "virtio-blk")]
/// virtio-block device manager
pub mod blk_dev_mgr;
#[cfg(feature = "virtio-blk")]
use self::blk_dev_mgr::BlockDeviceMgr;
#[cfg(feature = "virtio-net")]
/// Device manager for virtio-net devices.
pub mod virtio_net_dev_mgr;
#[cfg(feature = "virtio-net")]
use self::virtio_net_dev_mgr::VirtioNetDeviceMgr;
#[cfg(feature = "virtio-fs")]
/// virtio-block device manager
pub mod fs_dev_mgr;
#[cfg(feature = "virtio-fs")]
use self::fs_dev_mgr::FsDeviceMgr;
#[cfg(feature = "virtio-fs")]
mod memory_region_handler;
#[cfg(feature = "virtio-fs")]
pub use self::memory_region_handler::*;
macro_rules! info(
($l:expr, $($args:tt)+) => {
slog::info!($l, $($args)+; slog::o!("subsystem" => "device_manager"))
@@ -66,21 +93,27 @@ pub enum DeviceMgrError {
/// Invalid operation.
#[error("invalid device manager operation")]
InvalidOperation,
/// Failed to get device resource.
#[error("failed to get device assigned resources")]
GetDeviceResource,
/// Appending to kernel command line failed.
#[error("failed to add kernel command line parameter for device: {0}")]
Cmdline(#[source] linux_loader::cmdline::Error),
/// Failed to manage console devices.
#[error(transparent)]
ConsoleManager(console_manager::ConsoleManagerError),
/// Failed to create the device.
#[error("failed to create virtual device: {0}")]
CreateDevice(#[source] io::Error),
/// Failed to perform an operation on the bus.
#[error(transparent)]
IoManager(IoManagerError),
/// Failure from legacy device manager.
#[error(transparent)]
LegacyManager(legacy::Error),
@@ -94,6 +127,10 @@ pub enum DeviceMgrError {
/// Failed to hotplug the device.
#[error("failed to hotplug virtual device")]
HotplugDevice(#[source] UpcallClientError),
/// Failed to free device resource.
#[error("failed to free device resources: {0}")]
ResourceError(#[source] crate::resource_manager::ResourceError),
}
/// Specialized version of `std::result::Result` for device manager operations.
@@ -240,6 +277,10 @@ impl DeviceOpContext {
}
}
pub(crate) fn create_boot_ctx(vm: &Vm, epoll_mgr: Option<EpollManager>) -> Self {
Self::new(epoll_mgr, vm.device_manager(), None, None, false)
}
pub(crate) fn get_vm_as(&self) -> Result<GuestAddressSpaceImpl> {
match self.vm_as.as_ref() {
Some(v) => Ok(v.clone()),
@@ -282,11 +323,11 @@ impl DeviceOpContext {
}
}
#[cfg(not(feature = "hotplug"))]
#[cfg(all(feature = "hotplug", not(feature = "dbs-upcall")))]
impl DeviceOpContext {
pub(crate) fn insert_hotplug_mmio_device(
&self,
_dev: &Arc<dyn DeviceIo>,
_dev: &Arc<DbsMmioV2Device>,
_callback: Option<()>,
) -> Result<()> {
Err(DeviceMgrError::InvalidOperation)
@@ -303,6 +344,20 @@ impl DeviceOpContext {
#[cfg(all(feature = "hotplug", feature = "dbs-upcall"))]
impl DeviceOpContext {
pub(crate) fn create_hotplug_ctx(vm: &Vm, epoll_mgr: Option<EpollManager>) -> Self {
let vm_as = vm.vm_as().expect("VM should have memory ready").clone();
let mut ctx = Self::new(
epoll_mgr,
vm.device_manager(),
Some(vm_as),
vm.vm_address_space().cloned(),
true,
);
ctx.upcall_client = vm.upcall_client().clone();
ctx
}
fn call_hotplug_device(
&self,
req: DevMgrRequest,
@@ -378,8 +433,21 @@ pub struct DeviceManager {
pub(crate) con_manager: ConsoleManager,
pub(crate) legacy_manager: Option<LegacyDeviceManager>,
#[cfg(target_arch = "aarch64")]
pub(crate) mmio_device_info: HashMap<(DeviceType, String), MMIODeviceInfo>,
#[cfg(feature = "virtio-vsock")]
pub(crate) vsock_manager: VsockDeviceMgr,
#[cfg(feature = "virtio-blk")]
// If there is a Root Block Device, this should be added as the first element of the list.
// This is necessary because we want the root to always be mounted on /dev/vda.
pub(crate) block_manager: BlockDeviceMgr,
#[cfg(feature = "virtio-net")]
pub(crate) virtio_net_manager: VirtioNetDeviceMgr,
#[cfg(feature = "virtio-fs")]
fs_manager: Arc<Mutex<FsDeviceMgr>>,
}
impl DeviceManager {
@@ -397,13 +465,27 @@ impl DeviceManager {
res_manager,
vm_fd,
logger: logger.new(slog::o!()),
con_manager: ConsoleManager::new(epoll_manager, logger),
legacy_manager: None,
#[cfg(target_arch = "aarch64")]
mmio_device_info: HashMap::new(),
#[cfg(feature = "virtio-vsock")]
vsock_manager: VsockDeviceMgr::default(),
#[cfg(feature = "virtio-blk")]
block_manager: BlockDeviceMgr::default(),
#[cfg(feature = "virtio-net")]
virtio_net_manager: VirtioNetDeviceMgr::default(),
#[cfg(feature = "virtio-fs")]
fs_manager: Arc::new(Mutex::new(FsDeviceMgr::default())),
}
}
/// Get the underlying IoManager to dispatch IO read/write requests.
pub fn io_manager(&self) -> IoManagerCached {
IoManagerCached::new(self.io_manager.clone())
}
/// Create the underline interrupt manager for the device manager.
pub fn create_interrupt_manager(&mut self) -> Result<()> {
self.irq_manager
@@ -417,10 +499,11 @@ impl DeviceManager {
}
/// Create legacy devices associted virtual machine
#[allow(unused_variables)]
pub fn create_legacy_devices(
&mut self,
ctx: &mut DeviceOpContext,
) -> std::result::Result<(), StartMicrovmError> {
) -> std::result::Result<(), StartMicroVmError> {
#[cfg(target_arch = "x86_64")]
{
let mut tx = ctx.io_context.begin_tx();
@@ -434,7 +517,7 @@ impl DeviceManager {
}
Err(e) => {
ctx.io_context.cancel_tx(tx);
return Err(StartMicrovmError::LegacyDevice(e));
return Err(StartMicroVmError::LegacyDevice(e));
}
}
}
@@ -448,10 +531,10 @@ impl DeviceManager {
dmesg_fifo: Option<Box<dyn io::Write + Send>>,
com1_sock_path: Option<String>,
_ctx: &mut DeviceOpContext,
) -> std::result::Result<(), StartMicrovmError> {
) -> std::result::Result<(), StartMicroVmError> {
// Connect serial ports to the console and dmesg_fifo.
self.set_guest_kernel_log_stream(dmesg_fifo)
.map_err(|_| StartMicrovmError::EventFd)?;
.map_err(|_| StartMicroVmError::EventFd)?;
info!(self.logger, "init console path: {:?}", com1_sock_path);
if let Some(path) = com1_sock_path {
@@ -459,13 +542,13 @@ impl DeviceManager {
let com1 = legacy_manager.get_com1_serial();
self.con_manager
.create_socket_console(com1, path)
.map_err(StartMicrovmError::DeviceManager)?;
.map_err(StartMicroVmError::DeviceManager)?;
}
} else if let Some(legacy_manager) = self.legacy_manager.as_ref() {
let com1 = legacy_manager.get_com1_serial();
self.con_manager
.create_stdio_console(com1)
.map_err(StartMicrovmError::DeviceManager)?;
.map_err(StartMicroVmError::DeviceManager)?;
}
Ok(())
@@ -503,7 +586,7 @@ impl DeviceManager {
com1_sock_path: Option<String>,
dmesg_fifo: Option<Box<dyn io::Write + Send>>,
address_space: Option<&AddressSpace>,
) -> std::result::Result<(), StartMicrovmError> {
) -> std::result::Result<(), StartMicroVmError> {
let mut ctx = DeviceOpContext::new(
Some(epoll_mgr),
self,
@@ -515,16 +598,67 @@ impl DeviceManager {
self.create_legacy_devices(&mut ctx)?;
self.init_legacy_devices(dmesg_fifo, com1_sock_path, &mut ctx)?;
#[cfg(feature = "virtio-blk")]
self.block_manager
.attach_devices(&mut ctx)
.map_err(StartMicroVmError::BlockDeviceError)?;
#[cfg(feature = "virtio-fs")]
{
let mut fs_manager = self.fs_manager.lock().unwrap();
fs_manager
.attach_devices(&mut ctx)
.map_err(StartMicroVmError::FsDeviceError)?;
}
#[cfg(feature = "virtio-net")]
self.virtio_net_manager
.attach_devices(&mut ctx)
.map_err(StartMicroVmError::VirtioNetDeviceError)?;
#[cfg(feature = "virtio-vsock")]
self.vsock_manager.attach_devices(&mut ctx)?;
#[cfg(feature = "virtio-blk")]
self.block_manager
.generate_kernel_boot_args(kernel_config)
.map_err(StartMicroVmError::DeviceManager)?;
ctx.generate_kernel_boot_args(kernel_config)
.map_err(StartMicrovmError::DeviceManager)?;
.map_err(StartMicroVmError::DeviceManager)?;
Ok(())
}
#[cfg(target_arch = "x86_64")]
/// Start all registered devices when booting the associated virtual machine.
pub fn start_devices(&mut self) -> std::result::Result<(), StartMicroVmError> {
// TODO: add vfio support here. issue #4589.
Ok(())
}
/// Remove all devices when shutdown the associated virtual machine
pub fn remove_devices(
&mut self,
vm_as: GuestAddressSpaceImpl,
epoll_mgr: EpollManager,
address_space: Option<&AddressSpace>,
) -> Result<()> {
// create context for removing devices
let mut ctx = DeviceOpContext::new(
Some(epoll_mgr),
self,
Some(vm_as),
address_space.cloned(),
true,
);
#[cfg(feature = "virtio-blk")]
self.block_manager.remove_devices(&mut ctx)?;
Ok(())
}
}
#[cfg(target_arch = "x86_64")]
impl DeviceManager {
/// Get the underlying eventfd for vm exit notification.
pub fn get_reset_eventfd(&self) -> Result<vmm_sys_util::eventfd::EventFd> {
if let Some(legacy) = self.legacy_manager.as_ref() {
@@ -539,6 +673,14 @@ impl DeviceManager {
}
}
#[cfg(target_arch = "aarch64")]
impl DeviceManager {
/// Return mmio device info for FDT build.
pub fn get_mmio_device_info(&self) -> Option<&HashMap<(DeviceType, String), MMIODeviceInfo>> {
Some(&self.mmio_device_info)
}
}
#[cfg(feature = "dbs-virtio-devices")]
impl DeviceManager {
fn get_virtio_device_info(device: &Arc<DbsMmioV2Device>) -> Result<(u64, u64, u32)> {
@@ -638,7 +780,9 @@ impl DeviceManager {
// unregister Resource manager
let resources = device.get_assigned_resources();
ctx.res_manager.free_device_resources(&resources);
ctx.res_manager
.free_device_resources(&resources)
.map_err(DeviceMgrError::ResourceError)?;
Ok(())
}
@@ -689,3 +833,21 @@ impl DeviceManager {
}
}
}
#[cfg(feature = "hotplug")]
impl DeviceManager {
/// Get Unix Domain Socket path for the vsock device.
pub fn get_vsock_inner_connector(&mut self) -> Option<VsockInnerConnector> {
#[cfg(feature = "virtio-vsock")]
{
self.vsock_manager
.get_default_connector()
.map(|d| Some(d))
.unwrap_or(None)
}
#[cfg(not(feature = "virtio-vsock"))]
{
return None;
}
}
}

View File

@@ -0,0 +1,387 @@
// Copyright 2020-2022 Alibaba, Inc. or its affiliates. All Rights Reserved.
// Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
// SPDX-License-Identifier: Apache-2.0
//
// Portions Copyright 2017 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 THIRD-PARTY file.
use std::convert::TryInto;
use std::sync::Arc;
use dbs_utils::net::{MacAddr, Tap, TapError};
use dbs_utils::rate_limiter::BucketUpdate;
use dbs_virtio_devices as virtio;
use dbs_virtio_devices::net::Net;
use dbs_virtio_devices::Error as VirtioError;
use serde_derive::{Deserialize, Serialize};
use crate::address_space_manager::GuestAddressSpaceImpl;
use crate::config_manager::{
ConfigItem, DeviceConfigInfo, DeviceConfigInfos, RateLimiterConfigInfo,
};
use crate::device_manager::{DeviceManager, DeviceMgrError, DeviceOpContext};
use crate::get_bucket_update;
use super::DbsMmioV2Device;
/// Default number of virtio queues, one rx/tx pair.
pub const NUM_QUEUES: usize = 2;
/// Default size of virtio queues.
pub const QUEUE_SIZE: u16 = 256;
// The flag of whether to use the shared irq.
const USE_SHARED_IRQ: bool = true;
// The flag of whether to use the generic irq.
const USE_GENERIC_IRQ: bool = true;
/// Errors associated with virtio net device operations.
#[derive(Debug, thiserror::Error)]
pub enum VirtioNetDeviceError {
/// The virtual machine instance ID is invalid.
#[error("the virtual machine instance ID is invalid")]
InvalidVMID,
/// The iface ID is invalid.
#[error("invalid virtio-net iface id '{0}'")]
InvalidIfaceId(String),
/// Invalid queue number configuration for virtio_net device.
#[error("invalid queue number {0} for virtio-net device")]
InvalidQueueNum(usize),
/// Failure from device manager,
#[error("failure in device manager operations, {0}")]
DeviceManager(#[source] DeviceMgrError),
/// The Context Identifier is already in use.
#[error("the device ID {0} already exists")]
DeviceIDAlreadyExist(String),
/// The MAC address is already in use.
#[error("the guest MAC address {0} is already in use")]
GuestMacAddressInUse(String),
/// The host device name is already in use.
#[error("the host device name {0} is already in use")]
HostDeviceNameInUse(String),
/// Cannot open/create tap device.
#[error("cannot open TAP device")]
OpenTap(#[source] TapError),
/// Failure from virtio subsystem.
#[error(transparent)]
Virtio(VirtioError),
/// Failed to send patch message to net epoll handler.
#[error("could not send patch message to the net epoll handler")]
NetEpollHanderSendFail,
/// The update is not allowed after booting the microvm.
#[error("update operation is not allowed after boot")]
UpdateNotAllowedPostBoot,
/// Split this at some point.
/// Internal errors are due to resource exhaustion.
/// Users errors are due to invalid permissions.
#[error("cannot create network device: {0}")]
CreateNetDevice(#[source] VirtioError),
/// Cannot initialize a MMIO Network Device or add a device to the MMIO Bus.
#[error("failure while registering network device: {0}")]
RegisterNetDevice(#[source] DeviceMgrError),
}
/// Configuration information for virtio net devices.
#[derive(Clone, Debug, Deserialize, PartialEq, Serialize)]
pub struct VirtioNetDeviceConfigUpdateInfo {
/// ID of the guest network interface.
pub iface_id: String,
/// Rate Limiter for received packages.
pub rx_rate_limiter: Option<RateLimiterConfigInfo>,
/// Rate Limiter for transmitted packages.
pub tx_rate_limiter: Option<RateLimiterConfigInfo>,
}
impl VirtioNetDeviceConfigUpdateInfo {
/// Provides a `BucketUpdate` description for the RX bandwidth rate limiter.
pub fn rx_bytes(&self) -> BucketUpdate {
get_bucket_update!(self, rx_rate_limiter, bandwidth)
}
/// Provides a `BucketUpdate` description for the RX ops rate limiter.
pub fn rx_ops(&self) -> BucketUpdate {
get_bucket_update!(self, rx_rate_limiter, ops)
}
/// Provides a `BucketUpdate` description for the TX bandwidth rate limiter.
pub fn tx_bytes(&self) -> BucketUpdate {
get_bucket_update!(self, tx_rate_limiter, bandwidth)
}
/// Provides a `BucketUpdate` description for the TX ops rate limiter.
pub fn tx_ops(&self) -> BucketUpdate {
get_bucket_update!(self, tx_rate_limiter, ops)
}
}
/// Configuration information for virtio net devices.
#[derive(Clone, Debug, Deserialize, PartialEq, Serialize, Default)]
pub struct VirtioNetDeviceConfigInfo {
/// ID of the guest network interface.
pub iface_id: String,
/// Host level path for the guest network interface.
pub host_dev_name: String,
/// Number of virtqueues to use.
pub num_queues: usize,
/// Size of each virtqueue. Unit: byte.
pub queue_size: u16,
/// Guest MAC address.
pub guest_mac: Option<MacAddr>,
/// Rate Limiter for received packages.
pub rx_rate_limiter: Option<RateLimiterConfigInfo>,
/// Rate Limiter for transmitted packages.
pub tx_rate_limiter: Option<RateLimiterConfigInfo>,
/// allow duplicate mac
pub allow_duplicate_mac: bool,
/// Use shared irq
pub use_shared_irq: Option<bool>,
/// Use generic irq
pub use_generic_irq: Option<bool>,
}
impl VirtioNetDeviceConfigInfo {
/// Returns the tap device that `host_dev_name` refers to.
pub fn open_tap(&self) -> std::result::Result<Tap, VirtioNetDeviceError> {
Tap::open_named(self.host_dev_name.as_str(), false).map_err(VirtioNetDeviceError::OpenTap)
}
/// Returns a reference to the mac address. It the mac address is not configured, it
/// return None.
pub fn guest_mac(&self) -> Option<&MacAddr> {
self.guest_mac.as_ref()
}
///Rx and Tx queue and max queue sizes
pub fn queue_sizes(&self) -> Vec<u16> {
let mut queue_size = self.queue_size;
if queue_size == 0 {
queue_size = QUEUE_SIZE;
}
let num_queues = if self.num_queues > 0 {
self.num_queues
} else {
NUM_QUEUES
};
(0..num_queues).map(|_| queue_size).collect::<Vec<u16>>()
}
}
impl ConfigItem for VirtioNetDeviceConfigInfo {
type Err = VirtioNetDeviceError;
fn id(&self) -> &str {
&self.iface_id
}
fn check_conflicts(&self, other: &Self) -> Result<(), VirtioNetDeviceError> {
if self.iface_id == other.iface_id {
Err(VirtioNetDeviceError::DeviceIDAlreadyExist(
self.iface_id.clone(),
))
} else if !other.allow_duplicate_mac
&& self.guest_mac.is_some()
&& self.guest_mac == other.guest_mac
{
Err(VirtioNetDeviceError::GuestMacAddressInUse(
self.guest_mac.as_ref().unwrap().to_string(),
))
} else if self.host_dev_name == other.host_dev_name {
Err(VirtioNetDeviceError::HostDeviceNameInUse(
self.host_dev_name.clone(),
))
} else {
Ok(())
}
}
}
/// Virtio Net Device Info
pub type VirtioNetDeviceInfo = DeviceConfigInfo<VirtioNetDeviceConfigInfo>;
/// Device manager to manage all virtio net devices.
pub struct VirtioNetDeviceMgr {
pub(crate) info_list: DeviceConfigInfos<VirtioNetDeviceConfigInfo>,
pub(crate) use_shared_irq: bool,
}
impl VirtioNetDeviceMgr {
/// Gets the index of the device with the specified `drive_id` if it exists in the list.
pub fn get_index_of_iface_id(&self, if_id: &str) -> Option<usize> {
self.info_list
.iter()
.position(|info| info.config.iface_id.eq(if_id))
}
/// Insert or update a virtio net device into the manager.
pub fn insert_device(
device_mgr: &mut DeviceManager,
mut ctx: DeviceOpContext,
config: VirtioNetDeviceConfigInfo,
) -> std::result::Result<(), VirtioNetDeviceError> {
if config.num_queues % 2 != 0 {
return Err(VirtioNetDeviceError::InvalidQueueNum(config.num_queues));
}
if !cfg!(feature = "hotplug") && ctx.is_hotplug {
return Err(VirtioNetDeviceError::UpdateNotAllowedPostBoot);
}
let mgr = &mut device_mgr.virtio_net_manager;
slog::info!(
ctx.logger(),
"add virtio-net device configuration";
"subsystem" => "net_dev_mgr",
"id" => &config.iface_id,
"host_dev_name" => &config.host_dev_name,
);
let device_index = mgr.info_list.insert_or_update(&config)?;
if ctx.is_hotplug {
slog::info!(
ctx.logger(),
"attach virtio-net device";
"subsystem" => "net_dev_mgr",
"id" => &config.iface_id,
"host_dev_name" => &config.host_dev_name,
);
match Self::create_device(&config, &mut ctx) {
Ok(device) => {
let dev = DeviceManager::create_mmio_virtio_device(
device,
&mut ctx,
config.use_shared_irq.unwrap_or(mgr.use_shared_irq),
config.use_generic_irq.unwrap_or(USE_GENERIC_IRQ),
)
.map_err(VirtioNetDeviceError::DeviceManager)?;
ctx.insert_hotplug_mmio_device(&dev.clone(), None)
.map_err(VirtioNetDeviceError::DeviceManager)?;
// live-upgrade need save/restore device from info.device.
mgr.info_list[device_index].set_device(dev);
}
Err(e) => {
mgr.info_list.remove(device_index);
return Err(VirtioNetDeviceError::Virtio(e));
}
}
}
Ok(())
}
/// Update the ratelimiter settings of a virtio net device.
pub fn update_device_ratelimiters(
device_mgr: &mut DeviceManager,
new_cfg: VirtioNetDeviceConfigUpdateInfo,
) -> std::result::Result<(), VirtioNetDeviceError> {
let mgr = &mut device_mgr.virtio_net_manager;
match mgr.get_index_of_iface_id(&new_cfg.iface_id) {
Some(index) => {
let config = &mut mgr.info_list[index].config;
config.rx_rate_limiter = new_cfg.rx_rate_limiter.clone();
config.tx_rate_limiter = new_cfg.tx_rate_limiter.clone();
let device = mgr.info_list[index].device.as_mut().ok_or_else(|| {
VirtioNetDeviceError::InvalidIfaceId(new_cfg.iface_id.clone())
})?;
if let Some(mmio_dev) = device.as_any().downcast_ref::<DbsMmioV2Device>() {
let guard = mmio_dev.state();
let inner_dev = guard.get_inner_device();
if let Some(net_dev) = inner_dev
.as_any()
.downcast_ref::<virtio::net::Net<GuestAddressSpaceImpl>>()
{
return net_dev
.set_patch_rate_limiters(
new_cfg.rx_bytes(),
new_cfg.rx_ops(),
new_cfg.tx_bytes(),
new_cfg.tx_ops(),
)
.map(|_p| ())
.map_err(|_e| VirtioNetDeviceError::NetEpollHanderSendFail);
}
}
Ok(())
}
None => Err(VirtioNetDeviceError::InvalidIfaceId(
new_cfg.iface_id.clone(),
)),
}
}
/// Attach all configured vsock device to the virtual machine instance.
pub fn attach_devices(
&mut self,
ctx: &mut DeviceOpContext,
) -> std::result::Result<(), VirtioNetDeviceError> {
for info in self.info_list.iter_mut() {
slog::info!(
ctx.logger(),
"attach virtio-net device";
"subsystem" => "net_dev_mgr",
"id" => &info.config.iface_id,
"host_dev_name" => &info.config.host_dev_name,
);
let device = Self::create_device(&info.config, ctx)
.map_err(VirtioNetDeviceError::CreateNetDevice)?;
let device = DeviceManager::create_mmio_virtio_device(
device,
ctx,
info.config.use_shared_irq.unwrap_or(self.use_shared_irq),
info.config.use_generic_irq.unwrap_or(USE_GENERIC_IRQ),
)
.map_err(VirtioNetDeviceError::RegisterNetDevice)?;
info.set_device(device);
}
Ok(())
}
fn create_device(
cfg: &VirtioNetDeviceConfigInfo,
ctx: &mut DeviceOpContext,
) -> std::result::Result<Box<Net<GuestAddressSpaceImpl>>, virtio::Error> {
let epoll_mgr = ctx.epoll_mgr.clone().ok_or(virtio::Error::InvalidInput)?;
let rx_rate_limiter = match cfg.rx_rate_limiter.as_ref() {
Some(rl) => Some(rl.try_into().map_err(virtio::Error::IOError)?),
None => None,
};
let tx_rate_limiter = match cfg.tx_rate_limiter.as_ref() {
Some(rl) => Some(rl.try_into().map_err(virtio::Error::IOError)?),
None => None,
};
let net_device = Net::new(
cfg.host_dev_name.clone(),
cfg.guest_mac(),
Arc::new(cfg.queue_sizes()),
epoll_mgr,
rx_rate_limiter,
tx_rate_limiter,
)?;
Ok(Box::new(net_device))
}
}
impl Default for VirtioNetDeviceMgr {
/// Create a new virtio net device manager.
fn default() -> Self {
VirtioNetDeviceMgr {
info_list: DeviceConfigInfos::new(),
use_shared_irq: USE_SHARED_IRQ,
}
}
}

View File

@@ -17,7 +17,7 @@ use dbs_virtio_devices::vsock::Vsock;
use dbs_virtio_devices::Error as VirtioError;
use serde_derive::{Deserialize, Serialize};
use super::StartMicrovmError;
use super::StartMicroVmError;
use crate::config_manager::{ConfigItem, DeviceConfigInfo, DeviceConfigInfos};
use crate::device_manager::{DeviceManager, DeviceOpContext};
@@ -88,6 +88,20 @@ pub struct VsockDeviceConfigInfo {
pub use_generic_irq: Option<bool>,
}
impl Default for VsockDeviceConfigInfo {
fn default() -> Self {
Self {
id: String::default(),
guest_cid: 0,
uds_path: None,
tcp_addr: None,
queue_size: Vec::from(QUEUE_SIZES),
use_shared_irq: None,
use_generic_irq: None,
}
}
}
impl VsockDeviceConfigInfo {
/// Get number and size of queues supported.
pub fn queue_sizes(&self) -> Vec<u16> {
@@ -185,11 +199,11 @@ impl VsockDeviceMgr {
pub fn attach_devices(
&mut self,
ctx: &mut DeviceOpContext,
) -> std::result::Result<(), StartMicrovmError> {
) -> std::result::Result<(), StartMicroVmError> {
let epoll_mgr = ctx
.epoll_mgr
.clone()
.ok_or(StartMicrovmError::CreateVsockDevice(
.ok_or(StartMicroVmError::CreateVsockDevice(
virtio::Error::InvalidInput,
))?;
@@ -209,32 +223,32 @@ impl VsockDeviceMgr {
epoll_mgr.clone(),
)
.map_err(VirtioError::VirtioVsockError)
.map_err(StartMicrovmError::CreateVsockDevice)?,
.map_err(StartMicroVmError::CreateVsockDevice)?,
);
if let Some(uds_path) = info.config.uds_path.as_ref() {
let unix_backend = VsockUnixStreamBackend::new(uds_path.clone())
.map_err(VirtioError::VirtioVsockError)
.map_err(StartMicrovmError::CreateVsockDevice)?;
.map_err(StartMicroVmError::CreateVsockDevice)?;
device
.add_backend(Box::new(unix_backend), true)
.map_err(VirtioError::VirtioVsockError)
.map_err(StartMicrovmError::CreateVsockDevice)?;
.map_err(StartMicroVmError::CreateVsockDevice)?;
}
if let Some(tcp_addr) = info.config.tcp_addr.as_ref() {
let tcp_backend = VsockTcpBackend::new(tcp_addr.clone())
.map_err(VirtioError::VirtioVsockError)
.map_err(StartMicrovmError::CreateVsockDevice)?;
.map_err(StartMicroVmError::CreateVsockDevice)?;
device
.add_backend(Box::new(tcp_backend), false)
.map_err(VirtioError::VirtioVsockError)
.map_err(StartMicrovmError::CreateVsockDevice)?;
.map_err(StartMicroVmError::CreateVsockDevice)?;
}
// add inner backend to the the first added vsock device
if let Some(inner_backend) = self.default_inner_backend.take() {
device
.add_backend(Box::new(inner_backend), false)
.map_err(VirtioError::VirtioVsockError)
.map_err(StartMicrovmError::CreateVsockDevice)?;
.map_err(StartMicroVmError::CreateVsockDevice)?;
}
let device = DeviceManager::create_mmio_virtio_device_with_features(
device,
@@ -243,7 +257,7 @@ impl VsockDeviceMgr {
info.config.use_shared_irq.unwrap_or(self.use_shared_irq),
info.config.use_generic_irq.unwrap_or(USE_GENERIC_IRQ),
)
.map_err(StartMicrovmError::RegisterVsockDevice)?;
.map_err(StartMicroVmError::RegisterVsockDevice)?;
info.device = Some(device);
}

View File

@@ -12,7 +12,7 @@
#[cfg(feature = "dbs-virtio-devices")]
use dbs_virtio_devices::Error as VirtIoError;
use crate::device_manager;
use crate::{address_space_manager, device_manager, vcpu, vm};
/// Shorthand result type for internal VMM commands.
pub type Result<T> = std::result::Result<T, Error>;
@@ -23,8 +23,20 @@ pub type Result<T> = std::result::Result<T, Error>;
/// of the host (for example if Dragonball doesn't have permissions to open the KVM fd).
#[derive(Debug, thiserror::Error)]
pub enum Error {
/// Empty AddressSpace from parameters.
#[error("Empty AddressSpace from parameters")]
AddressSpace,
/// The zero page extends past the end of guest_mem.
#[error("the guest zero page extends past the end of guest memory")]
ZeroPagePastRamEnd,
/// Error writing the zero page of guest memory.
#[error("failed to write to guest zero page")]
ZeroPageSetup,
/// Failure occurs in issuing KVM ioctls and errors will be returned from kvm_ioctls lib.
#[error("failure in issuing KVM ioctl command")]
#[error("failure in issuing KVM ioctl command: {0}")]
Kvm(#[source] kvm_ioctls::Error),
/// The host kernel reports an unsupported KVM API version.
@@ -32,26 +44,85 @@ pub enum Error {
KvmApiVersion(i32),
/// Cannot initialize the KVM context due to missing capabilities.
#[error("missing KVM capability")]
#[error("missing KVM capability: {0:?}")]
KvmCap(kvm_ioctls::Cap),
#[cfg(target_arch = "x86_64")]
#[error("failed to configure MSRs")]
#[error("failed to configure MSRs: {0:?}")]
/// Cannot configure MSRs
GuestMSRs(dbs_arch::msr::Error),
/// MSR inner error
#[error("MSR inner error")]
Msr(vmm_sys_util::fam::Error),
/// Error writing MP table to memory.
#[cfg(target_arch = "x86_64")]
#[error("failed to write MP table to guest memory: {0}")]
MpTableSetup(#[source] dbs_boot::mptable::Error),
/// Fail to boot system
#[error("failed to boot system: {0}")]
BootSystem(#[source] dbs_boot::Error),
/// Cannot open the VM file descriptor.
#[error(transparent)]
Vm(vm::VmError),
}
/// Errors associated with starting the instance.
#[derive(Debug, thiserror::Error)]
pub enum StartMicrovmError {
pub enum StartMicroVmError {
/// Cannot read from an Event file descriptor.
#[error("failure while reading from EventFd file descriptor")]
EventFd,
/// Cannot add event to Epoll.
#[error("failure while registering epoll event for file descriptor")]
RegisterEvent,
/// The start command was issued more than once.
#[error("the virtual machine is already running")]
MicroVMAlreadyRunning,
/// Cannot start the VM because the kernel was not configured.
#[error("cannot start the virtual machine without kernel configuration")]
MissingKernelConfig,
#[cfg(feature = "hotplug")]
/// Upcall initialize miss vsock device.
#[error("the upcall client needs a virtio-vsock device for communication")]
UpcallMissVsock,
/// Upcall is not ready
#[error("the upcall client is not ready")]
UpcallNotReady,
/// Configuration passed in is invalidate.
#[error("invalid virtual machine configuration: {0} ")]
ConfigureInvalid(String),
/// This error is thrown by the minimal boot loader implementation.
/// It is related to a faulty memory configuration.
#[error("failure while configuring boot information for the virtual machine: {0}")]
ConfigureSystem(#[source] Error),
/// Cannot configure the VM.
#[error("failure while configuring the virtual machine: {0}")]
ConfigureVm(#[source] vm::VmError),
/// Cannot load initrd.
#[error("cannot load Initrd into guest memory: {0}")]
InitrdLoader(#[from] LoadInitrdError),
/// Cannot load kernel due to invalid memory configuration or invalid kernel image.
#[error("cannot load guest kernel into guest memory: {0}")]
KernelLoader(#[source] linux_loader::loader::Error),
/// Cannot load command line string.
#[error("failure while configuring guest kernel commandline: {0}")]
LoadCommandline(#[source] linux_loader::loader::Error),
/// The device manager was not configured.
#[error("the device manager failed to manage devices: {0}")]
DeviceManager(#[source] device_manager::DeviceMgrError),
@@ -69,4 +140,81 @@ pub enum StartMicrovmError {
/// Cannot initialize a MMIO Vsock Device or add a device to the MMIO Bus.
#[error("failure while registering virtio-vsock device: {0}")]
RegisterVsockDevice(#[source] device_manager::DeviceMgrError),
/// Address space manager related error, e.g.cannot access guest address space manager.
#[error("address space manager related error: {0}")]
AddressManagerError(#[source] address_space_manager::AddressManagerError),
/// Cannot create a new vCPU file descriptor.
#[error("vCPU related error: {0}")]
Vcpu(#[source] vcpu::VcpuManagerError),
#[cfg(all(feature = "hotplug", feature = "dbs-upcall"))]
/// Upcall initialize Error.
#[error("failure while initializing the upcall client: {0}")]
UpcallInitError(#[source] dbs_upcall::UpcallClientError),
#[cfg(all(feature = "hotplug", feature = "dbs-upcall"))]
/// Upcall connect Error.
#[error("failure while connecting the upcall client: {0}")]
UpcallConnectError(#[source] dbs_upcall::UpcallClientError),
#[cfg(feature = "virtio-blk")]
/// Virtio-blk errors.
#[error("virtio-blk errors: {0}")]
BlockDeviceError(#[source] device_manager::blk_dev_mgr::BlockDeviceError),
#[cfg(feature = "virtio-net")]
/// Virtio-net errors.
#[error("virtio-net errors: {0}")]
VirtioNetDeviceError(#[source] device_manager::virtio_net_dev_mgr::VirtioNetDeviceError),
#[cfg(feature = "virtio-fs")]
/// Virtio-fs errors.
#[error("virtio-fs errors: {0}")]
FsDeviceError(#[source] device_manager::fs_dev_mgr::FsDeviceError),
}
/// Errors associated with starting the instance.
#[derive(Debug, thiserror::Error)]
pub enum StopMicrovmError {
/// Guest memory has not been initialized.
#[error("Guest memory has not been initialized")]
GuestMemoryNotInitialized,
/// Cannnot remove devices
#[error("Failed to remove devices in device_manager {0}")]
DeviceManager(#[source] device_manager::DeviceMgrError),
}
/// Errors associated with loading initrd
#[derive(Debug, thiserror::Error)]
pub enum LoadInitrdError {
/// Cannot load initrd due to an invalid memory configuration.
#[error("failed to load the initrd image to guest memory")]
LoadInitrd,
/// Cannot load initrd due to an invalid image.
#[error("failed to read the initrd image: {0}")]
ReadInitrd(#[source] std::io::Error),
}
/// A dedicated error type to glue with the vmm_epoll crate.
#[derive(Debug, thiserror::Error)]
pub enum EpollError {
/// Generic internal error.
#[error("unclassfied internal error")]
InternalError,
/// Errors from the epoll subsystem.
#[error("failed to issue epoll syscall: {0}")]
EpollMgr(#[from] dbs_utils::epoll_manager::Error),
/// Generic IO errors.
#[error(transparent)]
IOError(std::io::Error),
#[cfg(feature = "dbs-virtio-devices")]
/// Errors from virtio devices.
#[error("failed to manager Virtio device: {0}")]
VirtIoDevice(#[source] VirtIoError),
}

View File

@@ -0,0 +1,169 @@
// Copyright (C) 2020-2022 Alibaba Cloud. All rights reserved.
// Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
// SPDX-License-Identifier: Apache-2.0
//
// Portions Copyright 2017 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 THIRD-PARTY file.
//! Event manager to manage and handle IO events and requests from API server .
use std::sync::atomic::{AtomicUsize, Ordering};
use std::sync::{Arc, Mutex};
use dbs_utils::epoll_manager::{
EpollManager, EventOps, EventSet, Events, MutEventSubscriber, SubscriberId,
};
use log::{error, warn};
use vmm_sys_util::eventfd::EventFd;
use crate::error::{EpollError, Result};
use crate::vmm::Vmm;
// Statically assigned epoll slot for VMM events.
pub(crate) const EPOLL_EVENT_EXIT: u32 = 0;
pub(crate) const EPOLL_EVENT_API_REQUEST: u32 = 1;
/// Shared information between vmm::vmm_thread_event_loop() and VmmEpollHandler.
pub(crate) struct EventContext {
pub api_event_fd: EventFd,
pub api_event_triggered: bool,
pub exit_evt_triggered: bool,
}
impl EventContext {
/// Create a new instance of [`EventContext`].
pub fn new(api_event_fd: EventFd) -> Result<Self> {
Ok(EventContext {
api_event_fd,
api_event_triggered: false,
exit_evt_triggered: false,
})
}
}
/// Event manager for VMM to handle API requests and IO events.
pub struct EventManager {
epoll_mgr: EpollManager,
subscriber_id: SubscriberId,
vmm_event_count: Arc<AtomicUsize>,
}
impl Drop for EventManager {
fn drop(&mut self) {
// Vmm -> Vm -> EpollManager -> VmmEpollHandler -> Vmm
// We need to remove VmmEpollHandler to break the circular reference
// so that Vmm can drop.
self.epoll_mgr
.remove_subscriber(self.subscriber_id)
.map_err(|e| {
error!("event_manager: remove_subscriber err. {:?}", e);
e
})
.ok();
}
}
impl EventManager {
/// Create a new event manager associated with the VMM object.
pub fn new(vmm: &Arc<Mutex<Vmm>>, epoll_mgr: EpollManager) -> Result<Self> {
let vmm_event_count = Arc::new(AtomicUsize::new(0));
let handler: Box<dyn MutEventSubscriber + Send> = Box::new(VmmEpollHandler {
vmm: vmm.clone(),
vmm_event_count: vmm_event_count.clone(),
});
let subscriber_id = epoll_mgr.add_subscriber(handler);
Ok(EventManager {
epoll_mgr,
subscriber_id,
vmm_event_count,
})
}
/// Get the underlying epoll event manager.
pub fn epoll_manager(&self) -> EpollManager {
self.epoll_mgr.clone()
}
/// Registry the eventfd for exit notification.
pub fn register_exit_eventfd(
&mut self,
exit_evt: &EventFd,
) -> std::result::Result<(), EpollError> {
let events = Events::with_data(exit_evt, EPOLL_EVENT_EXIT, EventSet::IN);
self.epoll_mgr
.add_event(self.subscriber_id, events)
.map_err(EpollError::EpollMgr)
}
/// Poll pending events and invoke registered event handler.
///
/// # Arguments:
/// * max_events: maximum number of pending events to handle
/// * timeout: maximum time in milliseconds to wait
pub fn handle_events(&self, timeout: i32) -> std::result::Result<usize, EpollError> {
self.epoll_mgr
.handle_events(timeout)
.map_err(EpollError::EpollMgr)
}
/// Fetch the VMM event count and reset it to zero.
pub fn fetch_vmm_event_count(&self) -> usize {
self.vmm_event_count.swap(0, Ordering::AcqRel)
}
}
struct VmmEpollHandler {
vmm: Arc<Mutex<Vmm>>,
vmm_event_count: Arc<AtomicUsize>,
}
impl MutEventSubscriber for VmmEpollHandler {
fn process(&mut self, events: Events, _ops: &mut EventOps) {
// Do not try to recover when the lock has already been poisoned.
// And be careful to avoid deadlock between process() and vmm::vmm_thread_event_loop().
let mut vmm = self.vmm.lock().unwrap();
match events.data() {
EPOLL_EVENT_API_REQUEST => {
if let Err(e) = vmm.event_ctx.api_event_fd.read() {
error!("event_manager: failed to read API eventfd, {:?}", e);
}
vmm.event_ctx.api_event_triggered = true;
self.vmm_event_count.fetch_add(1, Ordering::AcqRel);
}
EPOLL_EVENT_EXIT => {
let vm = vmm.get_vm().unwrap();
match vm.get_reset_eventfd() {
Some(ev) => {
if let Err(e) = ev.read() {
error!("event_manager: failed to read exit eventfd, {:?}", e);
}
}
None => warn!("event_manager: leftover exit event in epoll context!"),
}
vmm.event_ctx.exit_evt_triggered = true;
self.vmm_event_count.fetch_add(1, Ordering::AcqRel);
}
_ => error!("event_manager: unknown epoll slot number {}", events.data()),
}
}
fn init(&mut self, ops: &mut EventOps) {
// Do not expect poisoned lock.
let vmm = self.vmm.lock().unwrap();
let events = Events::with_data(
&vmm.event_ctx.api_event_fd,
EPOLL_EVENT_API_REQUEST,
EventSet::IN,
);
if let Err(e) = ops.add(events) {
error!(
"event_manager: failed to register epoll event for API server, {:?}",
e
);
}
}
}

View File

@@ -32,8 +32,13 @@ pub mod vcpu;
/// Virtual machine manager for virtual machines.
pub mod vm;
mod event_manager;
mod io_manager;
mod vmm;
pub use self::error::StartMicroVmError;
pub use self::io_manager::IoManagerCached;
pub use self::vmm::Vmm;
/// Success exit code.
pub const EXIT_CODE_OK: u8 = 0;

View File

@@ -6,20 +6,20 @@
// Use of this source code is governed by a BSD-style license that can be
// found in the THIRD-PARTY file.
use std::ops::Deref;
use std::sync::mpsc::{channel, Sender};
use std::sync::Arc;
use std::ops::Deref;
use crate::IoManagerCached;
use dbs_utils::time::TimestampUs;
use dbs_arch::regs;
use dbs_boot::get_fdt_addr;
use dbs_utils::time::TimestampUs;
use kvm_ioctls::{VcpuFd, VmFd};
use vm_memory::{Address, GuestAddress, GuestAddressSpace};
use vmm_sys_util::eventfd::EventFd;
use crate::address_space_manager::GuestAddressSpaceImpl;
use crate::vcpu::vcpu_impl::{Result, Vcpu, VcpuStateEvent, VcpuError};
use crate::vcpu::vcpu_impl::{Result, Vcpu, VcpuError, VcpuStateEvent};
use crate::vcpu::VcpuConfig;
#[allow(unused)]
@@ -111,8 +111,7 @@ impl Vcpu {
.map_err(VcpuError::REGSConfiguration)?;
}
self.mpidr =
regs::read_mpidr(&self.fd).map_err(VcpuError::REGSConfiguration)?;
self.mpidr = regs::read_mpidr(&self.fd).map_err(VcpuError::REGSConfiguration)?;
Ok(())
}

View File

@@ -4,12 +4,14 @@
// SPDX-License-Identifier: Apache-2.0
mod sm;
pub mod vcpu_impl;
pub mod vcpu_manager;
mod vcpu_impl;
mod vcpu_manager;
#[cfg(target_arch = "x86_64")]
use dbs_arch::cpuid::VpmuFeatureLevel;
pub use vcpu_manager::{VcpuManager, VcpuManagerError};
/// vcpu config collection
pub struct VcpuConfig {
/// initial vcpu count

View File

@@ -964,16 +964,7 @@ pub mod tests {
#[cfg(target_arch = "x86_64")]
#[test]
fn test_vcpu_check_io_port_info() {
let (vcpu, receiver) = create_vcpu();
// boot complete signal
let res = vcpu
.check_io_port_info(
MAGIC_IOPORT_SIGNAL_GUEST_BOOT_COMPLETE,
&[MAGIC_VALUE_SIGNAL_GUEST_BOOT_COMPLETE],
)
.unwrap();
assert!(res);
let (vcpu, _receiver) = create_vcpu();
// debug info signal
let res = vcpu

View File

@@ -147,6 +147,7 @@ pub enum VcpuResizeError {
#[error("Removable vcpu not enough, removable vcpu num: {0}, number to remove: {1}, present vcpu count {2}")]
LackRemovableVcpus(u16, u16, u16),
#[cfg(all(feature = "hotplug", feature = "dbs-upcall"))]
/// Cannot update the configuration by upcall channel.
#[error("cannot update the configuration by upcall channel: {0}")]
Upcall(#[source] dbs_upcall::UpcallClientError),
@@ -782,16 +783,16 @@ impl VcpuManager {
#[cfg(feature = "hotplug")]
mod hotplug {
#[cfg(feature = "dbs-upcall")]
use super::*;
#[cfg(feature = "dbs-upcall")]
use dbs_upcall::{CpuDevRequest, DevMgrRequest};
#[cfg(feature = "dbs-upcall")]
use std::cmp::Ordering;
use super::*;
#[cfg(not(test))]
use dbs_upcall::CpuDevRequest;
use dbs_upcall::{DevMgrRequest, DevMgrResponse, UpcallClientRequest, UpcallClientResponse};
#[cfg(all(target_arch = "x86_64", not(test)))]
#[cfg(all(target_arch = "x86_64", feature = "dbs-upcall"))]
use dbs_boot::mptable::APIC_VERSION;
#[cfg(all(target_arch = "aarch64", not(test)))]
#[cfg(all(target_arch = "aarch64"))]
const APIC_VERSION: u8 = 0;
#[cfg(feature = "dbs-upcall")]
@@ -933,6 +934,9 @@ mod hotplug {
upcall_client: Arc<UpcallClient<DevMgrService>>,
request: DevMgrRequest,
) -> std::result::Result<(), VcpuManagerError> {
// This is used to fix clippy warnings.
use dbs_upcall::{DevMgrResponse, UpcallClientRequest, UpcallClientResponse};
let vcpu_state_event = self.vcpu_state_event.try_clone().unwrap();
let vcpu_state_sender = self.vcpu_state_sender.clone();

View File

@@ -0,0 +1,158 @@
// Copyright (C) 2022 Alibaba Cloud. All rights reserved.
// Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
// SPDX-License-Identifier: Apache-2.0
//
// Portions Copyright 2017 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 THIRD-PARTY file.
use std::collections::HashMap;
use std::fmt::Debug;
use std::ops::Deref;
use dbs_arch::gic::GICDevice;
use dbs_arch::{DeviceInfoForFDT, DeviceType};
use dbs_boot::InitrdConfig;
use dbs_utils::epoll_manager::EpollManager;
use dbs_utils::time::TimestampUs;
use linux_loader::loader::Cmdline;
use vm_memory::{GuestAddressSpace, GuestMemory};
use vmm_sys_util::eventfd::EventFd;
use super::{Vm, VmError};
use crate::address_space_manager::{GuestAddressSpaceImpl, GuestMemoryImpl};
use crate::error::{Error, StartMicrovmError};
use crate::event_manager::EventManager;
/// Configures the system and should be called once per vm before starting vcpu threads.
/// For aarch64, we only setup the FDT.
///
/// # Arguments
///
/// * `guest_mem` - The memory to be used by the guest.
/// * `cmdline` - The kernel commandline.
/// * `vcpu_mpidr` - Array of MPIDR register values per vcpu.
/// * `device_info` - A hashmap containing the attached devices for building FDT device nodes.
/// * `gic_device` - The GIC device.
/// * `initrd` - Information about an optional initrd.
fn configure_system<T: DeviceInfoForFDT + Clone + Debug, M: GuestMemory>(
guest_mem: &M,
cmdline: &str,
vcpu_mpidr: Vec<u64>,
device_info: Option<&HashMap<(DeviceType, String), T>>,
gic_device: &Box<dyn GICDevice>,
initrd: &Option<super::InitrdConfig>,
) -> super::Result<()> {
dbs_boot::fdt::create_fdt(
guest_mem,
vcpu_mpidr,
cmdline,
device_info,
gic_device,
initrd,
)
.map_err(Error::BootSystem)?;
Ok(())
}
#[cfg(target_arch = "aarch64")]
impl Vm {
/// Gets a reference to the irqchip of the VM
pub fn get_irqchip(&self) -> &Box<dyn GICDevice> {
&self.irqchip_handle.as_ref().unwrap()
}
/// Creates the irq chip in-kernel device model.
pub fn setup_interrupt_controller(&mut self) -> std::result::Result<(), StartMicrovmError> {
let vcpu_count = self.vm_config.vcpu_count;
self.irqchip_handle = Some(
dbs_arch::gic::create_gic(&self.vm_fd, vcpu_count.into())
.map_err(|e| StartMicrovmError::ConfigureVm(VmError::SetupGIC(e)))?,
);
Ok(())
}
/// Initialize the virtual machine instance.
///
/// It initialize the virtual machine instance by:
/// 1) initialize virtual machine global state and configuration.
/// 2) create system devices, such as interrupt controller.
/// 3) create and start IO devices, such as serial, console, block, net, vsock etc.
/// 4) create and initialize vCPUs.
/// 5) configure CPU power management features.
/// 6) load guest kernel image.
pub fn init_microvm(
&mut self,
epoll_mgr: EpollManager,
vm_as: GuestAddressSpaceImpl,
request_ts: TimestampUs,
) -> Result<(), StartMicrovmError> {
let reset_eventfd =
EventFd::new(libc::EFD_NONBLOCK).map_err(|_| StartMicrovmError::EventFd)?;
self.reset_eventfd = Some(
reset_eventfd
.try_clone()
.map_err(|_| StartMicrovmError::EventFd)?,
);
self.vcpu_manager()
.map_err(StartMicrovmError::Vcpu)?
.set_reset_event_fd(reset_eventfd);
// On aarch64, the vCPUs need to be created (i.e call KVM_CREATE_VCPU) and configured before
// setting up the IRQ chip because the `KVM_CREATE_VCPU` ioctl will return error if the IRQCHIP
// was already initialized.
// Search for `kvm_arch_vcpu_create` in arch/arm/kvm/arm.c.
let kernel_loader_result = self.load_kernel(vm_as.memory().deref())?;
self.vcpu_manager()
.map_err(StartMicrovmError::Vcpu)?
.create_boot_vcpus(request_ts, kernel_loader_result.kernel_load)
.map_err(StartMicrovmError::Vcpu)?;
self.setup_interrupt_controller()?;
self.init_devices(epoll_mgr)?;
Ok(())
}
/// Execute system architecture specific configurations.
///
/// 1) set guest kernel boot parameters
/// 2) setup FDT data structs.
pub fn configure_system_arch(
&self,
vm_memory: &GuestMemoryImpl,
cmdline: &Cmdline,
initrd: Option<InitrdConfig>,
) -> std::result::Result<(), StartMicrovmError> {
let vcpu_manager = self.vcpu_manager().map_err(StartMicrovmError::Vcpu)?;
let vcpu_mpidr = vcpu_manager
.vcpus()
.into_iter()
.map(|cpu| cpu.get_mpidr())
.collect();
let guest_memory = vm_memory.memory();
configure_system(
guest_memory,
cmdline.as_str(),
vcpu_mpidr,
self.device_manager.get_mmio_device_info(),
self.get_irqchip(),
&initrd,
)
.map_err(StartMicrovmError::ConfigureSystem)
}
pub(crate) fn register_events(
&mut self,
event_mgr: &mut EventManager,
) -> std::result::Result<(), StartMicrovmError> {
let reset_evt = self.get_reset_eventfd().ok_or(StartMicrovmError::EventFd)?;
event_mgr
.register_exit_eventfd(reset_evt)
.map_err(|_| StartMicrovmError::RegisterEvent)?;
Ok(())
}
}

View File

@@ -32,6 +32,11 @@ impl KernelConfigInfo {
&mut self.kernel_file
}
/// Get an immutable reference to the initrd file.
pub fn initrd_file(&self) -> Option<&File> {
self.initrd_file.as_ref()
}
/// Get a mutable reference to the initrd file.
pub fn initrd_file_mut(&mut self) -> Option<&mut File> {
self.initrd_file.as_mut()

View File

@@ -1,11 +1,71 @@
// Copyright (C) 2021 Alibaba Cloud. All rights reserved.
// SPDX-License-Identifier: Apache-2.0
use std::io::{self, Read, Seek, SeekFrom};
use std::ops::Deref;
use std::os::unix::io::RawFd;
use std::sync::{Arc, Mutex, RwLock};
use dbs_address_space::AddressSpace;
#[cfg(target_arch = "aarch64")]
use dbs_arch::gic::GICDevice;
use dbs_boot::InitrdConfig;
use dbs_utils::epoll_manager::EpollManager;
use dbs_utils::time::TimestampUs;
use kvm_ioctls::VmFd;
use linux_loader::loader::{KernelLoader, KernelLoaderResult};
use seccompiler::BpfProgram;
use serde_derive::{Deserialize, Serialize};
use slog::{error, info};
use vm_memory::{Bytes, GuestAddress, GuestAddressSpace};
use vmm_sys_util::eventfd::EventFd;
#[cfg(all(feature = "hotplug", feature = "dbs-upcall"))]
use dbs_upcall::{DevMgrService, UpcallClient};
use crate::address_space_manager::{
AddressManagerError, AddressSpaceMgr, AddressSpaceMgrBuilder, GuestAddressSpaceImpl,
GuestMemoryImpl,
};
use crate::api::v1::{InstanceInfo, InstanceState};
use crate::device_manager::console_manager::DmesgWriter;
use crate::device_manager::{DeviceManager, DeviceMgrError, DeviceOpContext};
use crate::error::{LoadInitrdError, Result, StartMicroVmError, StopMicrovmError};
use crate::event_manager::EventManager;
use crate::kvm_context::KvmContext;
use crate::resource_manager::ResourceManager;
use crate::vcpu::{VcpuManager, VcpuManagerError};
#[cfg(target_arch = "aarch64")]
use dbs_arch::gic::Error as GICError;
mod kernel_config;
pub use self::kernel_config::KernelConfigInfo;
#[cfg(target_arch = "aarch64")]
#[path = "aarch64.rs"]
mod aarch64;
#[cfg(target_arch = "x86_64")]
#[path = "x86_64.rs"]
mod x86_64;
/// Errors associated with virtual machine instance related operations.
#[derive(Debug, thiserror::Error)]
pub enum VmError {
/// Cannot configure the IRQ.
#[error("failed to configure IRQ fot the virtual machine: {0}")]
Irq(#[source] kvm_ioctls::Error),
/// Cannot configure the microvm.
#[error("failed to initialize the virtual machine: {0}")]
VmSetup(#[source] kvm_ioctls::Error),
/// Cannot setup GIC
#[cfg(target_arch = "aarch64")]
#[error("failed to configure GIC")]
SetupGIC(dbs_arch::gic::Error),
}
/// Configuration information for user defined NUMA nodes.
#[derive(Clone, Debug, Default, Serialize, Deserialize, PartialEq)]
pub struct NumaRegionInfo {
@@ -50,8 +110,6 @@ pub struct VmConfigInfo {
pub vcpu_count: u8,
/// Max number of vcpu can be added
pub max_vcpu_count: u8,
/// Enable or disable hyperthreading.
pub ht_enabled: bool,
/// cpu power management.
pub cpu_pm: String,
/// cpu topology information
@@ -65,8 +123,6 @@ pub struct VmConfigInfo {
pub mem_file_path: String,
/// The memory size in MiB.
pub mem_size_mib: usize,
/// reserve memory bytes
pub reserve_memory_bytes: u64,
/// sock path
pub serial_path: Option<String>,
@@ -77,7 +133,6 @@ impl Default for VmConfigInfo {
VmConfigInfo {
vcpu_count: 1,
max_vcpu_count: 1,
ht_enabled: false,
cpu_pm: String::from("on"),
cpu_topology: CpuTopology {
threads_per_core: 1,
@@ -89,8 +144,671 @@ impl Default for VmConfigInfo {
mem_type: String::from("shmem"),
mem_file_path: String::from(""),
mem_size_mib: 128,
reserve_memory_bytes: 0,
serial_path: None,
}
}
}
/// Struct to manage resources and control states of an virtual machine instance.
///
/// An `Vm` instance holds a resources assigned to a virtual machine instance, such as CPU, memory,
/// devices etc. When an `Vm` instance gets deconstructed, all resources assigned should be
/// released.
///
/// We have explicit build the object model as:
/// |---Vmm API Server--<-1:1-> HTTP API Server
/// | |----------<-1:1-> Shimv2/CRI API Server
/// |
/// Vmm <-1:N-> Vm <-1:1-> Address Space Manager <-1:N-> GuestMemory
/// ^ ^---1:1-> Device Manager <-1:N-> Device
/// | ^---1:1-> Resource Manager
/// | ^---1:N-> Vcpu
/// |---<-1:N-> Event Manager
pub struct Vm {
epoll_manager: EpollManager,
kvm: KvmContext,
shared_info: Arc<RwLock<InstanceInfo>>,
address_space: AddressSpaceMgr,
device_manager: DeviceManager,
dmesg_fifo: Option<Box<dyn io::Write + Send>>,
kernel_config: Option<KernelConfigInfo>,
logger: slog::Logger,
reset_eventfd: Option<EventFd>,
resource_manager: Arc<ResourceManager>,
vcpu_manager: Option<Arc<Mutex<VcpuManager>>>,
vm_config: VmConfigInfo,
vm_fd: Arc<VmFd>,
start_instance_request_ts: u64,
start_instance_request_cpu_ts: u64,
start_instance_downtime: u64,
// Arm specific fields.
// On aarch64 we need to keep around the fd obtained by creating the VGIC device.
#[cfg(target_arch = "aarch64")]
irqchip_handle: Option<Box<dyn dbs_arch::gic::GICDevice>>,
#[cfg(all(feature = "hotplug", feature = "dbs-upcall"))]
upcall_client: Option<Arc<UpcallClient<DevMgrService>>>,
}
impl Vm {
/// Constructs a new `Vm` instance using the given `Kvm` instance.
pub fn new(
kvm_fd: Option<RawFd>,
api_shared_info: Arc<RwLock<InstanceInfo>>,
epoll_manager: EpollManager,
) -> Result<Self> {
let id = api_shared_info.read().unwrap().id.clone();
let logger = slog_scope::logger().new(slog::o!("id" => id));
let kvm = KvmContext::new(kvm_fd)?;
let vm_fd = Arc::new(kvm.create_vm()?);
let resource_manager = Arc::new(ResourceManager::new(Some(kvm.max_memslots())));
let device_manager = DeviceManager::new(
vm_fd.clone(),
resource_manager.clone(),
epoll_manager.clone(),
&logger,
);
Ok(Vm {
epoll_manager,
kvm,
shared_info: api_shared_info,
address_space: AddressSpaceMgr::default(),
device_manager,
dmesg_fifo: None,
kernel_config: None,
logger,
reset_eventfd: None,
resource_manager,
vcpu_manager: None,
vm_config: Default::default(),
vm_fd,
start_instance_request_ts: 0,
start_instance_request_cpu_ts: 0,
start_instance_downtime: 0,
#[cfg(target_arch = "aarch64")]
irqchip_handle: None,
#[cfg(all(feature = "hotplug", feature = "dbs-upcall"))]
upcall_client: None,
})
}
/// Gets a reference to the device manager by this VM.
pub fn device_manager(&self) -> &DeviceManager {
&self.device_manager
}
/// Gets a mutable reference to the device manager by this VM.
pub fn device_manager_mut(&mut self) -> &mut DeviceManager {
&mut self.device_manager
}
/// Get a reference to EpollManager.
pub fn epoll_manager(&self) -> &EpollManager {
&self.epoll_manager
}
/// Get eventfd for exit notification.
pub fn get_reset_eventfd(&self) -> Option<&EventFd> {
self.reset_eventfd.as_ref()
}
/// Set guest kernel boot configurations.
pub fn set_kernel_config(&mut self, kernel_config: KernelConfigInfo) {
self.kernel_config = Some(kernel_config);
}
/// Get virtual machine shared instance information.
pub fn shared_info(&self) -> &Arc<RwLock<InstanceInfo>> {
&self.shared_info
}
/// Gets a reference to the address_space.address_space for guest memory owned by this VM.
pub fn vm_address_space(&self) -> Option<&AddressSpace> {
self.address_space.get_address_space()
}
/// Gets a reference to the address space for guest memory owned by this VM.
///
/// Note that `GuestMemory` does not include any device memory that may have been added after
/// this VM was constructed.
pub fn vm_as(&self) -> Option<&GuestAddressSpaceImpl> {
self.address_space.get_vm_as()
}
/// Get a immutable reference to the virtual machine configuration information.
pub fn vm_config(&self) -> &VmConfigInfo {
&self.vm_config
}
/// Set the virtual machine configuration information.
pub fn set_vm_config(&mut self, config: VmConfigInfo) {
self.vm_config = config;
}
/// Gets a reference to the kvm file descriptor owned by this VM.
pub fn vm_fd(&self) -> &VmFd {
&self.vm_fd
}
/// returns true if system upcall service is ready
pub fn is_upcall_client_ready(&self) -> bool {
#[cfg(all(feature = "hotplug", feature = "dbs-upcall"))]
{
if let Some(upcall_client) = self.upcall_client() {
return upcall_client.is_ready();
}
}
false
}
/// Check whether the VM has been initialized.
pub fn is_vm_initialized(&self) -> bool {
let instance_state = {
// Use expect() to crash if the other thread poisoned this lock.
let shared_info = self.shared_info.read()
.expect("Failed to determine if instance is initialized because shared info couldn't be read due to poisoned lock");
shared_info.state
};
instance_state != InstanceState::Uninitialized
}
/// Check whether the VM instance is running.
pub fn is_vm_running(&self) -> bool {
let instance_state = {
// Use expect() to crash if the other thread poisoned this lock.
let shared_info = self.shared_info.read()
.expect("Failed to determine if instance is initialized because shared info couldn't be read due to poisoned lock");
shared_info.state
};
instance_state == InstanceState::Running
}
/// Save VM instance exit state
pub fn vm_exit(&self, exit_code: i32) {
if let Ok(mut info) = self.shared_info.write() {
info.state = InstanceState::Exited(exit_code);
} else {
error!(
self.logger,
"Failed to save exit state, couldn't be written due to poisoned lock"
);
}
}
/// Create device operation context.
/// vm is not running, return false
/// vm is running, but hotplug feature is not enable, return error
/// vm is running, but upcall initialize failed, return error
/// vm is running, upcall initialize OK, return true
pub fn create_device_op_context(
&mut self,
epoll_mgr: Option<EpollManager>,
) -> std::result::Result<DeviceOpContext, StartMicroVmError> {
if !self.is_vm_initialized() {
Ok(DeviceOpContext::create_boot_ctx(self, epoll_mgr))
} else {
self.create_device_hotplug_context(epoll_mgr)
}
}
pub(crate) fn check_health(&self) -> std::result::Result<(), StartMicroVmError> {
if self.kernel_config.is_none() {
return Err(StartMicroVmError::MissingKernelConfig);
}
Ok(())
}
pub(crate) fn get_dragonball_info(&self) -> (String, String) {
let guard = self.shared_info.read().unwrap();
let instance_id = guard.id.clone();
let dragonball_version = guard.vmm_version.clone();
(dragonball_version, instance_id)
}
}
impl Vm {
pub(crate) fn init_vcpu_manager(
&mut self,
vm_as: GuestAddressSpaceImpl,
vcpu_seccomp_filter: BpfProgram,
) -> std::result::Result<(), VcpuManagerError> {
let vcpu_manager = VcpuManager::new(
self.vm_fd.clone(),
&self.kvm,
&self.vm_config,
vm_as,
vcpu_seccomp_filter,
self.shared_info.clone(),
self.device_manager.io_manager(),
self.epoll_manager.clone(),
)?;
self.vcpu_manager = Some(vcpu_manager);
Ok(())
}
/// get the cpu manager's reference
pub(crate) fn vcpu_manager(
&self,
) -> std::result::Result<std::sync::MutexGuard<'_, VcpuManager>, VcpuManagerError> {
self.vcpu_manager
.as_ref()
.ok_or(VcpuManagerError::VcpuManagerNotInitialized)
.map(|mgr| mgr.lock().unwrap())
}
/// Pause all vcpus and record the instance downtime
pub fn pause_all_vcpus_with_downtime(&mut self) -> std::result::Result<(), VcpuManagerError> {
let ts = TimestampUs::default();
self.start_instance_downtime = ts.time_us;
self.vcpu_manager()?.pause_all_vcpus()?;
Ok(())
}
/// Resume all vcpus and calc the intance downtime
pub fn resume_all_vcpus_with_downtime(&mut self) -> std::result::Result<(), VcpuManagerError> {
self.vcpu_manager()?.resume_all_vcpus()?;
if self.start_instance_downtime != 0 {
let now = TimestampUs::default();
let downtime = now.time_us - self.start_instance_downtime;
info!(self.logger, "VM: instance downtime: {} us", downtime);
self.start_instance_downtime = 0;
if let Ok(mut info) = self.shared_info.write() {
info.last_instance_downtime = downtime;
} else {
error!(self.logger, "Failed to update live upgrade downtime, couldn't be written due to poisoned lock");
}
}
Ok(())
}
pub(crate) fn init_devices(
&mut self,
epoll_manager: EpollManager,
) -> std::result::Result<(), StartMicroVmError> {
info!(self.logger, "VM: initializing devices ...");
let com1_sock_path = self.vm_config.serial_path.clone();
let kernel_config = self
.kernel_config
.as_mut()
.ok_or(StartMicroVmError::MissingKernelConfig)?;
info!(self.logger, "VM: create interrupt manager");
self.device_manager
.create_interrupt_manager()
.map_err(StartMicroVmError::DeviceManager)?;
info!(self.logger, "VM: create devices");
let vm_as =
self.address_space
.get_vm_as()
.ok_or(StartMicroVmError::AddressManagerError(
AddressManagerError::GuestMemoryNotInitialized,
))?;
self.device_manager.create_devices(
vm_as.clone(),
epoll_manager,
kernel_config,
com1_sock_path,
self.dmesg_fifo.take(),
self.address_space.address_space(),
)?;
info!(self.logger, "VM: start devices");
self.device_manager.start_devices()?;
info!(self.logger, "VM: initializing devices done");
Ok(())
}
/// Remove devices when shutdown vm
pub fn remove_devices(&mut self) -> std::result::Result<(), StopMicrovmError> {
info!(self.logger, "VM: remove devices");
let vm_as = self
.address_space
.get_vm_as()
.ok_or(StopMicrovmError::GuestMemoryNotInitialized)?;
self.device_manager
.remove_devices(
vm_as.clone(),
self.epoll_manager.clone(),
self.address_space.address_space(),
)
.map_err(StopMicrovmError::DeviceManager)
}
/// Reset the console into canonical mode.
pub fn reset_console(&self) -> std::result::Result<(), DeviceMgrError> {
self.device_manager.reset_console()
}
pub(crate) fn init_dmesg_logger(&mut self) {
let writer = self.dmesg_logger();
self.dmesg_fifo = Some(writer);
}
/// dmesg write to logger
fn dmesg_logger(&self) -> Box<dyn io::Write + Send> {
Box::new(DmesgWriter::new(&self.logger))
}
pub(crate) fn init_guest_memory(&mut self) -> std::result::Result<(), StartMicroVmError> {
info!(self.logger, "VM: initializing guest memory...");
// We are not allowing reinitialization of vm guest memory.
if self.address_space.is_initialized() {
return Ok(());
}
// vcpu boot up require local memory. reserve 100 MiB memory
let mem_size = (self.vm_config.mem_size_mib as u64) << 20;
let mem_type = self.vm_config.mem_type.clone();
let mut mem_file_path = String::from("");
if mem_type == "hugetlbfs" {
let shared_info = self.shared_info.read()
.expect("Failed to determine if instance is initialized because shared info couldn't be read due to poisoned lock");
mem_file_path.push_str("/dragonball/");
mem_file_path.push_str(shared_info.id.as_str());
}
let mut vcpu_ids: Vec<u32> = Vec::new();
for i in 0..self.vm_config().max_vcpu_count {
vcpu_ids.push(i as u32);
}
// init default regions.
let mut numa_regions = Vec::with_capacity(1);
let numa_node = NumaRegionInfo {
size: self.vm_config.mem_size_mib as u64,
host_numa_node_id: None,
guest_numa_node_id: Some(0),
vcpu_ids,
};
numa_regions.push(numa_node);
info!(
self.logger,
"VM: mem_type:{} mem_file_path:{}, mem_size:{}, numa_regions:{:?}",
mem_type,
mem_file_path,
mem_size,
numa_regions,
);
let mut address_space_param = AddressSpaceMgrBuilder::new(&mem_type, &mem_file_path)
.map_err(StartMicroVmError::AddressManagerError)?;
address_space_param.set_kvm_vm_fd(self.vm_fd.clone());
self.address_space
.create_address_space(&self.resource_manager, &numa_regions, address_space_param)
.map_err(StartMicroVmError::AddressManagerError)?;
info!(self.logger, "VM: initializing guest memory done");
Ok(())
}
fn init_configure_system(
&mut self,
vm_as: &GuestAddressSpaceImpl,
) -> std::result::Result<(), StartMicroVmError> {
let vm_memory = vm_as.memory();
let kernel_config = self
.kernel_config
.as_ref()
.ok_or(StartMicroVmError::MissingKernelConfig)?;
//let cmdline = kernel_config.cmdline.clone();
let initrd: Option<InitrdConfig> = match kernel_config.initrd_file() {
Some(f) => {
let initrd_file = f.try_clone();
if initrd_file.is_err() {
return Err(StartMicroVmError::InitrdLoader(
LoadInitrdError::ReadInitrd(io::Error::from(io::ErrorKind::InvalidData)),
));
}
let res = self.load_initrd(vm_memory.deref(), &mut initrd_file.unwrap())?;
Some(res)
}
None => None,
};
self.configure_system_arch(vm_memory.deref(), kernel_config.kernel_cmdline(), initrd)
}
/// Loads the initrd from a file into the given memory slice.
///
/// * `vm_memory` - The guest memory the initrd is written to.
/// * `image` - The initrd image.
///
/// Returns the result of initrd loading
fn load_initrd<F>(
&self,
vm_memory: &GuestMemoryImpl,
image: &mut F,
) -> std::result::Result<InitrdConfig, LoadInitrdError>
where
F: Read + Seek,
{
use crate::error::LoadInitrdError::*;
let size: usize;
// Get the image size
match image.seek(SeekFrom::End(0)) {
Err(e) => return Err(ReadInitrd(e)),
Ok(0) => {
return Err(ReadInitrd(io::Error::new(
io::ErrorKind::InvalidData,
"Initrd image seek returned a size of zero",
)))
}
Ok(s) => size = s as usize,
};
// Go back to the image start
image.seek(SeekFrom::Start(0)).map_err(ReadInitrd)?;
// Get the target address
let address = dbs_boot::initrd_load_addr(vm_memory, size as u64).map_err(|_| LoadInitrd)?;
// Load the image into memory
vm_memory
.read_from(GuestAddress(address), image, size)
.map_err(|_| LoadInitrd)?;
Ok(InitrdConfig {
address: GuestAddress(address),
size,
})
}
fn load_kernel(
&mut self,
vm_memory: &GuestMemoryImpl,
) -> std::result::Result<KernelLoaderResult, StartMicroVmError> {
// This is the easy way out of consuming the value of the kernel_cmdline.
let kernel_config = self
.kernel_config
.as_mut()
.ok_or(StartMicroVmError::MissingKernelConfig)?;
let high_mem_addr = GuestAddress(dbs_boot::get_kernel_start());
#[cfg(target_arch = "x86_64")]
return linux_loader::loader::elf::Elf::load(
vm_memory,
None,
kernel_config.kernel_file_mut(),
Some(high_mem_addr),
)
.map_err(StartMicroVmError::KernelLoader);
#[cfg(target_arch = "aarch64")]
return linux_loader::loader::pe::PE::load(
vm_memory,
Some(GuestAddress(dbs_boot::get_kernel_start())),
kernel_config.kernel_file_mut(),
Some(high_mem_addr),
)
.map_err(StartMicroVmError::KernelLoader);
}
/// Set up the initial microVM state and start the vCPU threads.
///
/// This is the main entrance of the Vm object, to bring up the virtual machine instance into
/// running state.
pub fn start_microvm(
&mut self,
event_mgr: &mut EventManager,
vmm_seccomp_filter: BpfProgram,
vcpu_seccomp_filter: BpfProgram,
) -> std::result::Result<(), StartMicroVmError> {
info!(self.logger, "VM: received instance start command");
if self.is_vm_initialized() {
return Err(StartMicroVmError::MicroVMAlreadyRunning);
}
let request_ts = TimestampUs::default();
self.start_instance_request_ts = request_ts.time_us;
self.start_instance_request_cpu_ts = request_ts.cputime_us;
self.init_dmesg_logger();
self.check_health()?;
// Use expect() to crash if the other thread poisoned this lock.
self.shared_info
.write()
.expect("Failed to start microVM because shared info couldn't be written due to poisoned lock")
.state = InstanceState::Starting;
self.init_guest_memory()?;
let vm_as = self
.vm_as()
.cloned()
.ok_or(StartMicroVmError::AddressManagerError(
AddressManagerError::GuestMemoryNotInitialized,
))?;
self.init_vcpu_manager(vm_as.clone(), vcpu_seccomp_filter)
.map_err(StartMicroVmError::Vcpu)?;
self.init_microvm(event_mgr.epoll_manager(), vm_as.clone(), request_ts)?;
self.init_configure_system(&vm_as)?;
#[cfg(feature = "dbs-upcall")]
self.init_upcall()?;
info!(self.logger, "VM: register events");
self.register_events(event_mgr)?;
info!(self.logger, "VM: start vcpus");
self.vcpu_manager()
.map_err(StartMicroVmError::Vcpu)?
.start_boot_vcpus(vmm_seccomp_filter)
.map_err(StartMicroVmError::Vcpu)?;
// Use expect() to crash if the other thread poisoned this lock.
self.shared_info
.write()
.expect("Failed to start microVM because shared info couldn't be written due to poisoned lock")
.state = InstanceState::Running;
info!(self.logger, "VM started");
Ok(())
}
}
#[cfg(feature = "hotplug")]
impl Vm {
/// initialize upcall client for guest os
#[cfg(feature = "dbs-upcall")]
fn new_upcall(&mut self) -> std::result::Result<(), StartMicroVmError> {
// get vsock inner connector for upcall
let inner_connector = self
.device_manager
.get_vsock_inner_connector()
.ok_or(StartMicroVmError::UpcallMissVsock)?;
let mut upcall_client = UpcallClient::new(
inner_connector,
self.epoll_manager.clone(),
DevMgrService::default(),
)
.map_err(StartMicroVmError::UpcallInitError)?;
upcall_client
.connect()
.map_err(StartMicroVmError::UpcallConnectError)?;
self.upcall_client = Some(Arc::new(upcall_client));
info!(self.logger, "upcall client init success");
Ok(())
}
#[cfg(feature = "dbs-upcall")]
fn init_upcall(&mut self) -> std::result::Result<(), StartMicroVmError> {
info!(self.logger, "VM upcall init");
if let Err(e) = self.new_upcall() {
info!(
self.logger,
"VM upcall init failed, no support hotplug: {}", e
);
Err(e)
} else {
self.vcpu_manager()
.map_err(StartMicroVmError::Vcpu)?
.set_upcall_channel(self.upcall_client().clone());
Ok(())
}
}
/// Get upcall client.
#[cfg(feature = "dbs-upcall")]
pub fn upcall_client(&self) -> &Option<Arc<UpcallClient<DevMgrService>>> {
&self.upcall_client
}
#[cfg(feature = "dbs-upcall")]
fn create_device_hotplug_context(
&self,
epoll_mgr: Option<EpollManager>,
) -> std::result::Result<DeviceOpContext, StartMicroVmError> {
if self.upcall_client().is_none() {
Err(StartMicroVmError::UpcallMissVsock)
} else if self.is_upcall_client_ready() {
Ok(DeviceOpContext::create_hotplug_ctx(self, epoll_mgr))
} else {
Err(StartMicroVmError::UpcallNotReady)
}
}
// We will support hotplug without upcall in future stages.
#[cfg(not(feature = "dbs-upcall"))]
fn create_device_hotplug_context(
&self,
_epoll_mgr: Option<EpollManager>,
) -> std::result::Result<DeviceOpContext, StartMicroVmError> {
Err(StartMicroVmError::MicroVMAlreadyRunning)
}
}
#[cfg(not(feature = "hotplug"))]
impl Vm {
fn init_upcall(&mut self) -> std::result::Result<(), StartMicroVmError> {
Ok(())
}
fn create_device_hotplug_context(
&self,
_epoll_mgr: Option<EpollManager>,
) -> std::result::Result<DeviceOpContext, StartMicroVmError> {
Err(StartMicroVmError::MicroVMAlreadyRunning)
}
}

View File

@@ -0,0 +1,280 @@
// Copyright (C) 2020-2022 Alibaba Cloud. All rights reserved.
// Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
// SPDX-License-Identifier: Apache-2.0
//
// Portions Copyright 2017 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 THIRD-PARTY file.
use std::convert::TryInto;
use std::mem;
use std::ops::Deref;
use dbs_address_space::AddressSpace;
use dbs_boot::{add_e820_entry, bootparam, layout, mptable, BootParamsWrapper, InitrdConfig};
use dbs_utils::epoll_manager::EpollManager;
use dbs_utils::time::TimestampUs;
use kvm_bindings::{kvm_irqchip, kvm_pit_config, kvm_pit_state2, KVM_PIT_SPEAKER_DUMMY};
use linux_loader::cmdline::Cmdline;
use slog::info;
use vm_memory::{Address, Bytes, GuestAddress, GuestAddressSpace, GuestMemory};
use crate::address_space_manager::{GuestAddressSpaceImpl, GuestMemoryImpl};
use crate::error::{Error, Result, StartMicroVmError};
use crate::event_manager::EventManager;
use crate::vm::{Vm, VmError};
/// Configures the system and should be called once per vm before starting vcpu
/// threads.
///
/// # Arguments
///
/// * `guest_mem` - The memory to be used by the guest.
/// * `cmdline_addr` - Address in `guest_mem` where the kernel command line was
/// loaded.
/// * `cmdline_size` - Size of the kernel command line in bytes including the
/// null terminator.
/// * `initrd` - Information about where the ramdisk image was loaded in the
/// `guest_mem`.
/// * `boot_cpus` - Number of virtual CPUs the guest will have at boot time.
/// * `max_cpus` - Max number of virtual CPUs the guest will have.
/// * `rsv_mem_bytes` - Reserve memory from microVM..
#[allow(clippy::too_many_arguments)]
fn configure_system<M: GuestMemory>(
guest_mem: &M,
address_space: Option<&AddressSpace>,
cmdline_addr: GuestAddress,
cmdline_size: usize,
initrd: &Option<InitrdConfig>,
boot_cpus: u8,
max_cpus: u8,
) -> super::Result<()> {
const KERNEL_BOOT_FLAG_MAGIC: u16 = 0xaa55;
const KERNEL_HDR_MAGIC: u32 = 0x5372_6448;
const KERNEL_LOADER_OTHER: u8 = 0xff;
const KERNEL_MIN_ALIGNMENT_BYTES: u32 = 0x0100_0000; // Must be non-zero.
let mmio_start = GuestAddress(layout::MMIO_LOW_START);
let mmio_end = GuestAddress(layout::MMIO_LOW_END);
let himem_start = GuestAddress(layout::HIMEM_START);
// Note that this puts the mptable at the last 1k of Linux's 640k base RAM
mptable::setup_mptable(guest_mem, boot_cpus, max_cpus).map_err(Error::MpTableSetup)?;
let mut params: BootParamsWrapper = BootParamsWrapper(bootparam::boot_params::default());
params.0.hdr.type_of_loader = KERNEL_LOADER_OTHER;
params.0.hdr.boot_flag = KERNEL_BOOT_FLAG_MAGIC;
params.0.hdr.header = KERNEL_HDR_MAGIC;
params.0.hdr.cmd_line_ptr = cmdline_addr.raw_value() as u32;
params.0.hdr.cmdline_size = cmdline_size as u32;
params.0.hdr.kernel_alignment = KERNEL_MIN_ALIGNMENT_BYTES;
if let Some(initrd_config) = initrd {
params.0.hdr.ramdisk_image = initrd_config.address.raw_value() as u32;
params.0.hdr.ramdisk_size = initrd_config.size as u32;
}
add_e820_entry(&mut params.0, 0, layout::EBDA_START, bootparam::E820_RAM)
.map_err(Error::BootSystem)?;
let mem_end = address_space.ok_or(Error::AddressSpace)?.last_addr();
if mem_end < mmio_start {
add_e820_entry(
&mut params.0,
himem_start.raw_value() as u64,
// it's safe to use unchecked_offset_from because
// mem_end > himem_start
mem_end.unchecked_offset_from(himem_start) as u64 + 1,
bootparam::E820_RAM,
)
.map_err(Error::BootSystem)?;
} else {
add_e820_entry(
&mut params.0,
himem_start.raw_value(),
// it's safe to use unchecked_offset_from because
// end_32bit_gap_start > himem_start
mmio_start.unchecked_offset_from(himem_start),
bootparam::E820_RAM,
)
.map_err(Error::BootSystem)?;
if mem_end > mmio_end {
add_e820_entry(
&mut params.0,
mmio_end.raw_value() + 1,
// it's safe to use unchecked_offset_from because mem_end > mmio_end
mem_end.unchecked_offset_from(mmio_end) as u64,
bootparam::E820_RAM,
)
.map_err(Error::BootSystem)?;
}
}
let zero_page_addr = GuestAddress(layout::ZERO_PAGE_START);
guest_mem
.checked_offset(zero_page_addr, mem::size_of::<bootparam::boot_params>())
.ok_or(Error::ZeroPagePastRamEnd)?;
guest_mem
.write_obj(params, zero_page_addr)
.map_err(|_| Error::ZeroPageSetup)?;
Ok(())
}
impl Vm {
/// Get the status of in-kernel PIT.
pub fn get_pit_state(&self) -> Result<kvm_pit_state2> {
self.vm_fd
.get_pit2()
.map_err(|e| Error::Vm(VmError::Irq(e)))
}
/// Set the status of in-kernel PIT.
pub fn set_pit_state(&self, pit_state: &kvm_pit_state2) -> Result<()> {
self.vm_fd
.set_pit2(pit_state)
.map_err(|e| Error::Vm(VmError::Irq(e)))
}
/// Get the status of in-kernel ioapic.
pub fn get_irqchip_state(&self, chip_id: u32) -> Result<kvm_irqchip> {
let mut irqchip: kvm_irqchip = kvm_irqchip {
chip_id,
..kvm_irqchip::default()
};
self.vm_fd
.get_irqchip(&mut irqchip)
.map(|_| irqchip)
.map_err(|e| Error::Vm(VmError::Irq(e)))
}
/// Set the status of in-kernel ioapic.
pub fn set_irqchip_state(&self, irqchip: &kvm_irqchip) -> Result<()> {
self.vm_fd
.set_irqchip(irqchip)
.map_err(|e| Error::Vm(VmError::Irq(e)))
}
}
impl Vm {
/// Initialize the virtual machine instance.
///
/// It initialize the virtual machine instance by:
/// 1) initialize virtual machine global state and configuration.
/// 2) create system devices, such as interrupt controller, PIT etc.
/// 3) create and start IO devices, such as serial, console, block, net, vsock etc.
/// 4) create and initialize vCPUs.
/// 5) configure CPU power management features.
/// 6) load guest kernel image.
pub fn init_microvm(
&mut self,
epoll_mgr: EpollManager,
vm_as: GuestAddressSpaceImpl,
request_ts: TimestampUs,
) -> std::result::Result<(), StartMicroVmError> {
info!(self.logger, "VM: start initializing microvm ...");
self.init_tss()?;
// For x86_64 we need to create the interrupt controller before calling `KVM_CREATE_VCPUS`
// while on aarch64 we need to do it the other way around.
self.setup_interrupt_controller()?;
self.create_pit()?;
self.init_devices(epoll_mgr)?;
let reset_event_fd = self.device_manager.get_reset_eventfd().unwrap();
self.vcpu_manager()
.map_err(StartMicroVmError::Vcpu)?
.set_reset_event_fd(reset_event_fd)
.map_err(StartMicroVmError::Vcpu)?;
if self.vm_config.cpu_pm == "on" {
// TODO: add cpu_pm support. issue #4590.
info!(self.logger, "VM: enable CPU disable_idle_exits capability");
}
let vm_memory = vm_as.memory();
let kernel_loader_result = self.load_kernel(vm_memory.deref())?;
self.vcpu_manager()
.map_err(StartMicroVmError::Vcpu)?
.create_boot_vcpus(request_ts, kernel_loader_result.kernel_load)
.map_err(StartMicroVmError::Vcpu)?;
info!(self.logger, "VM: initializing microvm done");
Ok(())
}
/// Execute system architecture specific configurations.
///
/// 1) set guest kernel boot parameters
/// 2) setup BIOS configuration data structs, mainly implement the MPSpec.
pub fn configure_system_arch(
&self,
vm_memory: &GuestMemoryImpl,
cmdline: &Cmdline,
initrd: Option<InitrdConfig>,
) -> std::result::Result<(), StartMicroVmError> {
let cmdline_addr = GuestAddress(dbs_boot::layout::CMDLINE_START);
linux_loader::loader::load_cmdline(vm_memory, cmdline_addr, cmdline)
.map_err(StartMicroVmError::LoadCommandline)?;
configure_system(
vm_memory,
self.address_space.address_space(),
cmdline_addr,
cmdline.as_str().len() + 1,
&initrd,
self.vm_config.vcpu_count,
self.vm_config.max_vcpu_count,
)
.map_err(StartMicroVmError::ConfigureSystem)
}
/// Initializes the guest memory.
pub(crate) fn init_tss(&mut self) -> std::result::Result<(), StartMicroVmError> {
self.vm_fd
.set_tss_address(dbs_boot::layout::KVM_TSS_ADDRESS.try_into().unwrap())
.map_err(|e| StartMicroVmError::ConfigureVm(VmError::VmSetup(e)))
}
/// Creates the irq chip and an in-kernel device model for the PIT.
pub(crate) fn setup_interrupt_controller(
&mut self,
) -> std::result::Result<(), StartMicroVmError> {
self.vm_fd
.create_irq_chip()
.map_err(|e| StartMicroVmError::ConfigureVm(VmError::VmSetup(e)))
}
/// Creates an in-kernel device model for the PIT.
pub(crate) fn create_pit(&self) -> std::result::Result<(), StartMicroVmError> {
info!(self.logger, "VM: create pit");
// We need to enable the emulation of a dummy speaker port stub so that writing to port 0x61
// (i.e. KVM_SPEAKER_BASE_ADDRESS) does not trigger an exit to user space.
let pit_config = kvm_pit_config {
flags: KVM_PIT_SPEAKER_DUMMY,
..kvm_pit_config::default()
};
// Safe because we know that our file is a VM fd, we know the kernel will only read the
// correct amount of memory from our pointer, and we verify the return result.
self.vm_fd
.create_pit2(pit_config)
.map_err(|e| StartMicroVmError::ConfigureVm(VmError::VmSetup(e)))
}
pub(crate) fn register_events(
&mut self,
event_mgr: &mut EventManager,
) -> std::result::Result<(), StartMicroVmError> {
let reset_evt = self
.device_manager
.get_reset_eventfd()
.map_err(StartMicroVmError::DeviceManager)?;
event_mgr
.register_exit_eventfd(&reset_evt)
.map_err(|_| StartMicroVmError::RegisterEvent)?;
self.reset_eventfd = Some(reset_evt);
Ok(())
}
}

215
src/dragonball/src/vmm.rs Normal file
View File

@@ -0,0 +1,215 @@
// Copyright (C) 2020-2022 Alibaba Cloud. All rights reserved.
// Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
// SPDX-License-Identifier: Apache-2.0
//
// Portions Copyright 2017 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 THIRD-PARTY file.
use std::os::unix::io::RawFd;
use std::sync::{Arc, Mutex, RwLock};
use dbs_utils::epoll_manager::EpollManager;
use log::{error, info, warn};
use seccompiler::BpfProgram;
use vmm_sys_util::eventfd::EventFd;
use crate::api::v1::{InstanceInfo, VmmService};
use crate::error::{EpollError, Result};
use crate::event_manager::{EventContext, EventManager};
use crate::vm::Vm;
use crate::{EXIT_CODE_GENERIC_ERROR, EXIT_CODE_OK};
/// Global coordinator to manage API servers, virtual machines, upgrade etc.
///
/// Originally firecracker assumes an VMM only manages an VM, and doesn't distinguish VMM and VM.
/// Thus caused a mixed and confusion design. Now we have explicit build the object model as:
/// |---Vmm API Server--<-1:1-> HTTP API Server
/// | |----------<-1:1-> Shimv2/CRI API Server
/// |
/// Vmm <-1:N-> Vm <-1:1-> Address Space Manager <-1:N-> GuestMemory
/// ^ ^---1:1-> Device Manager <-1:N-> Device
/// | ^---1:1-> Resource Manager
/// | ^---1:N-> Vcpu
/// |---<-1:N-> Event Manager
pub struct Vmm {
pub(crate) event_ctx: EventContext,
epoll_manager: EpollManager,
// Will change to a HashMap when enabling 1 VMM with multiple VMs.
vm: Vm,
vcpu_seccomp_filter: BpfProgram,
vmm_seccomp_filter: BpfProgram,
}
impl Vmm {
/// Create a Virtual Machine Monitor instance.
pub fn new(
api_shared_info: Arc<RwLock<InstanceInfo>>,
api_event_fd: EventFd,
vmm_seccomp_filter: BpfProgram,
vcpu_seccomp_filter: BpfProgram,
kvm_fd: Option<RawFd>,
) -> Result<Self> {
let epoll_manager = EpollManager::default();
Self::new_with_epoll_manager(
api_shared_info,
api_event_fd,
epoll_manager,
vmm_seccomp_filter,
vcpu_seccomp_filter,
kvm_fd,
)
}
/// Create a Virtual Machine Monitor instance with a epoll_manager.
pub fn new_with_epoll_manager(
api_shared_info: Arc<RwLock<InstanceInfo>>,
api_event_fd: EventFd,
epoll_manager: EpollManager,
vmm_seccomp_filter: BpfProgram,
vcpu_seccomp_filter: BpfProgram,
kvm_fd: Option<RawFd>,
) -> Result<Self> {
let vm = Vm::new(kvm_fd, api_shared_info, epoll_manager.clone())?;
let event_ctx = EventContext::new(api_event_fd)?;
Ok(Vmm {
event_ctx,
epoll_manager,
vm,
vcpu_seccomp_filter,
vmm_seccomp_filter,
})
}
/// Get a reference to a virtual machine managed by the VMM.
pub fn get_vm(&self) -> Option<&Vm> {
Some(&self.vm)
}
/// Get a mutable reference to a virtual machine managed by the VMM.
pub fn get_vm_mut(&mut self) -> Option<&mut Vm> {
Some(&mut self.vm)
}
/// Get the seccomp rules for vCPU threads.
pub fn vcpu_seccomp_filter(&self) -> BpfProgram {
self.vcpu_seccomp_filter.clone()
}
/// Get the seccomp rules for VMM threads.
pub fn vmm_seccomp_filter(&self) -> BpfProgram {
self.vmm_seccomp_filter.clone()
}
/// Run the event loop to service API requests.
///
/// # Arguments
///
/// * `vmm` - An Arc reference to the global Vmm instance.
/// * `service` - VMM Service provider.
pub fn run_vmm_event_loop(vmm: Arc<Mutex<Vmm>>, mut service: VmmService) -> i32 {
let epoll_mgr = vmm.lock().unwrap().epoll_manager.clone();
let mut event_mgr =
EventManager::new(&vmm, epoll_mgr).expect("Cannot create epoll manager");
'poll: loop {
match event_mgr.handle_events(-1) {
Ok(_) => {
// Check whether there are pending vmm events.
if event_mgr.fetch_vmm_event_count() == 0 {
continue;
}
let mut v = vmm.lock().unwrap();
if v.event_ctx.api_event_triggered {
// The run_vmm_action() needs to access event_mgr, so it could
// not be handled in EpollHandler::handle_events(). It has been
// delayed to the main loop.
v.event_ctx.api_event_triggered = false;
service
.run_vmm_action(&mut v, &mut event_mgr)
.unwrap_or_else(|_| {
warn!("got spurious notification from api thread");
});
}
if v.event_ctx.exit_evt_triggered {
info!("Gracefully terminated VMM control loop");
return v.stop(EXIT_CODE_OK as i32);
}
}
Err(e) => {
error!("Abruptly exited VMM control loop: {:?}", e);
if let EpollError::EpollMgr(dbs_utils::epoll_manager::Error::Epoll(e)) = e {
if e.errno() == libc::EAGAIN || e.errno() == libc::EINTR {
continue 'poll;
}
}
return vmm.lock().unwrap().stop(EXIT_CODE_GENERIC_ERROR as i32);
}
}
}
}
/// Waits for all vCPUs to exit and terminates the Dragonball process.
fn stop(&mut self, exit_code: i32) -> i32 {
info!("Vmm is stopping.");
if let Some(vm) = self.get_vm_mut() {
if vm.is_vm_initialized() {
if let Err(e) = vm.remove_devices() {
warn!("failed to remove devices: {:?}", e);
}
if let Err(e) = vm.reset_console() {
warn!("Cannot set canonical mode for the terminal. {:?}", e);
}
// Now, we use exit_code instead of invoking _exit to
// terminate process, so all of vcpu threads should be stopped
// prior to vmm event loop.
match vm.vcpu_manager() {
Ok(mut mgr) => {
if let Err(e) = mgr.exit_all_vcpus() {
warn!("Failed to exit vcpu thread. {:?}", e);
}
}
Err(e) => warn!("Failed to get vcpu manager {:?}", e),
}
// save exit state to VM, instead of exit process.
vm.vm_exit(exit_code);
}
}
exit_code
}
}
#[cfg(test)]
pub(crate) mod tests {
use super::*;
pub fn create_vmm_instance() -> Vmm {
let info = Arc::new(RwLock::new(InstanceInfo::default()));
let event_fd = EventFd::new(libc::EFD_NONBLOCK).unwrap();
let seccomp_filter: BpfProgram = Vec::new();
let epoll_manager = EpollManager::default();
Vmm::new_with_epoll_manager(
info,
event_fd,
epoll_manager,
seccomp_filter.clone(),
seccomp_filter,
None,
)
.unwrap()
}
#[test]
fn test_create_vmm_instance() {
create_vmm_instance();
}
}