mirror of
https://github.com/kata-containers/kata-containers.git
synced 2025-09-17 23:07:55 +00:00
dragonball: add pci hotplug / hot-unplug support
Introduce two new vmm action to implement pci hotplug and pci hot-unplug: PrepareRemoveHostDevice and RemoveHostDevice. PrepareRemoveHostDevice is to call upcall to unregister the pci device in the guest kernel. RemoveHostDevice should be called after PrepareRemoveHostDevice, it is used to clean the PCI resource in the Dragonball side. fixes: #8741 Signed-off-by: Gerry Liu <gerry@linux.alibaba.com> Signed-off-by: Zizheng Bian <zizheng.bian@linux.alibaba.com> Signed-off-by: Zha Bin <zhabin@linux.alibaba.com> Signed-off-by: Helin Guo <helinguo@linux.alibaba.com> Signed-off-by: Chao Wu <chaowu@linux.alibaba.com>
This commit is contained in:
@@ -9,13 +9,14 @@
|
||||
use std::fs::File;
|
||||
use std::sync::{Arc, Mutex};
|
||||
|
||||
use crossbeam_channel::{Receiver, Sender, TryRecvError};
|
||||
use crossbeam_channel::{unbounded, Receiver, Sender, TryRecvError};
|
||||
use log::{debug, error, info, warn};
|
||||
use tracing::instrument;
|
||||
|
||||
use crate::error::{Result, StartMicroVmError, StopMicrovmError};
|
||||
use crate::event_manager::EventManager;
|
||||
use crate::tracer::{DragonballTracer, TraceError, TraceInfo};
|
||||
use crate::vcpu::VcpuManagerError;
|
||||
use crate::vm::{CpuTopology, KernelConfigInfo, VmConfigInfo};
|
||||
use crate::vmm::Vmm;
|
||||
|
||||
@@ -155,6 +156,10 @@ pub enum VmmActionError {
|
||||
/// The action `InsertHostDevice` failed either because of bad user input or an internal error.
|
||||
#[error("failed to add VFIO passthrough device: {0:?}")]
|
||||
HostDeviceConfig(#[source] VfioDeviceError),
|
||||
#[cfg(feature = "host-device")]
|
||||
/// The action 'RemoveHostDevice' failed because of vcpu manager internal error.
|
||||
#[error("remove host device error: {0}")]
|
||||
RemoveHostDevice(#[source] VcpuManagerError),
|
||||
}
|
||||
|
||||
/// This enum represents the public interface of the VMM. Each action contains various
|
||||
@@ -256,6 +261,14 @@ pub enum VmmAction {
|
||||
#[cfg(feature = "host-device")]
|
||||
/// Add a VFIO assignment host device or update that already exists
|
||||
InsertHostDevice(HostDeviceConfig),
|
||||
|
||||
#[cfg(feature = "host-device")]
|
||||
/// Prepare to remove a VFIO assignment host device that already exists
|
||||
PrepareRemoveHostDevice(String),
|
||||
|
||||
#[cfg(feature = "host-device")]
|
||||
/// Add a VFIO assignment host device or update that already exists
|
||||
RemoveHostDevice(String),
|
||||
}
|
||||
|
||||
/// The enum represents the response sent by the VMM in case of success. The response is either
|
||||
@@ -268,6 +281,8 @@ pub enum VmmData {
|
||||
MachineConfiguration(Box<VmConfigInfo>),
|
||||
/// Prometheus Metrics represented by String.
|
||||
HypervisorMetrics(String),
|
||||
/// Sync Hotplug
|
||||
SyncHotplug((Sender<Option<i32>>, Receiver<Option<i32>>)),
|
||||
}
|
||||
|
||||
/// Request data type used to communicate between the API and the VMM.
|
||||
@@ -384,6 +399,12 @@ impl VmmService {
|
||||
}
|
||||
#[cfg(feature = "host-device")]
|
||||
VmmAction::InsertHostDevice(hostdev_cfg) => self.add_vfio_device(vmm, hostdev_cfg),
|
||||
#[cfg(feature = "host-device")]
|
||||
VmmAction::PrepareRemoveHostDevice(hostdev_id) => {
|
||||
self.prepare_remove_vfio_device(vmm, &hostdev_id)
|
||||
}
|
||||
#[cfg(feature = "host-device")]
|
||||
VmmAction::RemoveHostDevice(hostdev_cfg) => self.remove_vfio_device(vmm, &hostdev_cfg),
|
||||
};
|
||||
|
||||
debug!("send vmm response: {:?}", response);
|
||||
@@ -552,6 +573,8 @@ impl VmmService {
|
||||
// - Some(path), legacy_manager will create_socket_console on that path.
|
||||
config.serial_path = machine_config.serial_path;
|
||||
|
||||
config.pci_hotplug_enabled = machine_config.pci_hotplug_enabled;
|
||||
|
||||
vm.set_vm_config(config.clone());
|
||||
self.machine_config = config;
|
||||
|
||||
@@ -833,15 +856,94 @@ impl VmmService {
|
||||
))?;
|
||||
info!("add_vfio_device: {:?}", config);
|
||||
|
||||
let mut ctx = vm.create_device_op_context(None).map_err(|e| {
|
||||
info!("create device op context error: {:?}", e);
|
||||
if let StartMicroVmError::MicroVMAlreadyRunning = e {
|
||||
VmmActionError::HostDeviceConfig(VfioDeviceError::UpdateNotAllowedPostBoot)
|
||||
} else if let StartMicroVmError::UpcallServerNotReady = e {
|
||||
VmmActionError::UpcallServerNotReady
|
||||
} else {
|
||||
VmmActionError::StartMicroVm(e)
|
||||
}
|
||||
})?;
|
||||
|
||||
vm.device_manager()
|
||||
.vfio_manager
|
||||
.lock()
|
||||
.unwrap()
|
||||
.insert_device(config.into())
|
||||
.insert_device(&mut ctx, config.into())
|
||||
.map_err(VmmActionError::HostDeviceConfig)?;
|
||||
Ok(VmmData::Empty)
|
||||
}
|
||||
|
||||
// using upcall to unplug the pci device in the guest
|
||||
#[cfg(feature = "host-device")]
|
||||
fn prepare_remove_vfio_device(&mut self, vmm: &mut Vmm, hostdev_id: &str) -> VmmRequestResult {
|
||||
let vm = vmm.get_vm_mut().ok_or(VmmActionError::HostDeviceConfig(
|
||||
VfioDeviceError::InvalidVMID,
|
||||
))?;
|
||||
|
||||
info!("prepare_remove_vfio_device: {:?}", hostdev_id);
|
||||
let ctx = vm.create_device_op_context(None).map_err(|e| {
|
||||
info!("create device op context error: {:?}", e);
|
||||
if let StartMicroVmError::MicroVMAlreadyRunning = e {
|
||||
VmmActionError::HostDeviceConfig(VfioDeviceError::UpdateNotAllowedPostBoot)
|
||||
} else if let StartMicroVmError::UpcallServerNotReady = e {
|
||||
VmmActionError::UpcallServerNotReady
|
||||
} else {
|
||||
VmmActionError::StartMicroVm(e)
|
||||
}
|
||||
})?;
|
||||
|
||||
let (sender, receiver) = unbounded();
|
||||
|
||||
// It is safe because we don't expect poison lock.
|
||||
let vfio_manager = vm.device_manager.vfio_manager.lock().unwrap();
|
||||
|
||||
vfio_manager
|
||||
.prepare_remove_device(&ctx, hostdev_id, sender.clone())
|
||||
.map(|_| VmmData::SyncHotplug((sender, receiver)))
|
||||
.map_err(VmmActionError::HostDeviceConfig)
|
||||
}
|
||||
|
||||
#[cfg(feature = "host-device")]
|
||||
fn remove_vfio_device(&self, vmm: &mut Vmm, hostdev_id: &str) -> VmmRequestResult {
|
||||
let vm = vmm.get_vm_mut().ok_or(VmmActionError::HostDeviceConfig(
|
||||
VfioDeviceError::InvalidVMID,
|
||||
))?;
|
||||
|
||||
info!("remove_vfio_device: {:?}", hostdev_id);
|
||||
let mut ctx = vm.create_device_op_context(None).map_err(|e| {
|
||||
info!("create device op context error: {:?}", e);
|
||||
if let StartMicroVmError::MicroVMAlreadyRunning = e {
|
||||
VmmActionError::HostDeviceConfig(VfioDeviceError::UpdateNotAllowedPostBoot)
|
||||
} else if let StartMicroVmError::UpcallServerNotReady = e {
|
||||
VmmActionError::UpcallServerNotReady
|
||||
} else {
|
||||
VmmActionError::StartMicroVm(e)
|
||||
}
|
||||
})?;
|
||||
|
||||
// It is safe because we don't expect poison lock.
|
||||
let mut vfio_manager = vm.device_manager.vfio_manager.lock().unwrap();
|
||||
|
||||
vfio_manager
|
||||
.remove_device(&mut ctx, hostdev_id)
|
||||
.map_err(VmmActionError::HostDeviceConfig)?;
|
||||
|
||||
// we need to revalidate io_manager cache in all vcpus
|
||||
// in order to drop old io_manager and close device's fd
|
||||
vm.vcpu_manager()
|
||||
.map_err(VmmActionError::RemoveHostDevice)?
|
||||
.revalidate_all_vcpus_cache()
|
||||
.map_err(VmmActionError::RemoveHostDevice)?;
|
||||
|
||||
// FIXME: we should clear corresponding information because vfio module in
|
||||
// host kernel will clear iommu table in this scenario.
|
||||
|
||||
Ok(VmmData::Empty)
|
||||
}
|
||||
|
||||
#[cfg(feature = "hotplug")]
|
||||
#[instrument(skip(self))]
|
||||
fn resize_vcpu(&mut self, vmm: &mut Vmm, config: VcpuResizeInfo) -> VmmRequestResult {
|
||||
|
@@ -3,6 +3,7 @@
|
||||
//
|
||||
// SPDX-License-Identifier: Apache-2.0
|
||||
|
||||
use std::any::Any;
|
||||
use std::io;
|
||||
use std::os::unix::io::AsRawFd;
|
||||
use std::ptr::null_mut;
|
||||
@@ -884,7 +885,7 @@ impl Region {
|
||||
}
|
||||
}
|
||||
|
||||
struct VfioPciDeviceState<C: PciSystemContext> {
|
||||
pub struct VfioPciDeviceState<C: PciSystemContext> {
|
||||
vfio_path: String,
|
||||
interrupt: Interrupt,
|
||||
vfio_dev: Arc<VfioDevice>,
|
||||
@@ -947,6 +948,10 @@ impl<C: PciSystemContext> VfioPciDeviceState<C> {
|
||||
})
|
||||
}
|
||||
|
||||
pub fn vfio_dev(&self) -> &Arc<VfioDevice> {
|
||||
&self.vfio_dev
|
||||
}
|
||||
|
||||
fn read_config_byte(&self, offset: u32) -> u8 {
|
||||
let mut data: [u8; 1] = [0];
|
||||
self.vfio_dev
|
||||
@@ -1314,6 +1319,23 @@ impl<C: PciSystemContext> VfioPciDeviceState<C> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn free_register_resources(&self) -> Result<()> {
|
||||
let mut register_resources = DeviceResources::new();
|
||||
for region in self.regions.iter() {
|
||||
let resources = region.to_resources();
|
||||
for res in resources.get_all_resources() {
|
||||
register_resources.append(res.clone());
|
||||
}
|
||||
}
|
||||
|
||||
self.bus
|
||||
.upgrade()
|
||||
.ok_or(VfioPciError::BusIsDropped)?
|
||||
.free_resources(register_resources);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn unregister_regions(&mut self, vm: &Arc<VmFd>) -> Result<()> {
|
||||
// This routine handle VfioPciDevice dropped but not unmap memory
|
||||
if self.context.upgrade().is_none() {
|
||||
@@ -1661,7 +1683,7 @@ impl<C: PciSystemContext> VfioPciDevice<C> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn state(&self) -> MutexGuard<VfioPciDeviceState<C>> {
|
||||
pub fn state(&self) -> MutexGuard<VfioPciDeviceState<C>> {
|
||||
// Don't expect poisoned lock
|
||||
self.state
|
||||
.lock()
|
||||
@@ -1687,6 +1709,14 @@ impl<C: PciSystemContext> VfioPciDevice<C> {
|
||||
.expect("poisoned lock for VFIO PCI device")
|
||||
.read_config_word(PCI_CONFIG_VENDOR_OFFSET)
|
||||
}
|
||||
|
||||
pub fn clear_device(&self) -> Result<()> {
|
||||
let mut state = self.state();
|
||||
state.free_register_resources()?;
|
||||
let _ = state.unregister_regions(&self.vm_fd);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
impl<C: 'static + PciSystemContext> DeviceIo for VfioPciDevice<C> {
|
||||
@@ -1784,7 +1814,8 @@ impl<C: 'static + PciSystemContext> DeviceIo for VfioPciDevice<C> {
|
||||
fn get_trapped_io_resources(&self) -> DeviceResources {
|
||||
self.state().trapped_resources.clone()
|
||||
}
|
||||
fn as_any(&self) -> &dyn std::any::Any {
|
||||
|
||||
fn as_any(&self) -> &dyn Any {
|
||||
self
|
||||
}
|
||||
}
|
||||
|
@@ -45,6 +45,16 @@ struct DevMgrMsgHeader {
|
||||
pub msg_flags: u32,
|
||||
}
|
||||
|
||||
/// Command struct to add/del a PCI Device.
|
||||
#[repr(C)]
|
||||
#[derive(Copy, Clone, Debug, PartialEq)]
|
||||
pub struct PciDevRequest {
|
||||
/// PCI bus number
|
||||
pub busno: u8,
|
||||
/// Combined device number and function number
|
||||
pub devfn: u8,
|
||||
}
|
||||
|
||||
/// Command struct to add/del a MMIO Virtio Device.
|
||||
#[repr(C)]
|
||||
#[derive(Copy, Clone, Debug, Eq, PartialEq)]
|
||||
@@ -128,6 +138,10 @@ pub enum DevMgrRequest {
|
||||
AddVcpu(CpuDevRequest),
|
||||
/// Del a VCPU
|
||||
DelVcpu(CpuDevRequest),
|
||||
/// Add a PCI device
|
||||
AddPciDev(PciDevRequest),
|
||||
/// Delete a PCI device
|
||||
DelPciDev(PciDevRequest),
|
||||
}
|
||||
|
||||
impl DevMgrRequest {
|
||||
@@ -167,6 +181,18 @@ impl DevMgrRequest {
|
||||
let vcpu_dev = unsafe { &mut *(buffer[size_hdr..].as_ptr() as *mut CpuDevRequest) };
|
||||
*vcpu_dev = s.clone();
|
||||
}
|
||||
DevMgrRequest::AddPciDev(s) => {
|
||||
msg_hdr.msg_type = DevMgrMsgType::AddPci as u32;
|
||||
msg_hdr.msg_size = mem::size_of::<PciDevRequest>() as u32;
|
||||
let pci_dev = unsafe { &mut *(buffer[size_hdr..].as_ptr() as *mut PciDevRequest) };
|
||||
*pci_dev = *s;
|
||||
}
|
||||
DevMgrRequest::DelPciDev(s) => {
|
||||
msg_hdr.msg_type = DevMgrMsgType::DelPci as u32;
|
||||
msg_hdr.msg_size = mem::size_of::<PciDevRequest>() as u32;
|
||||
let pci_dev = unsafe { &mut *(buffer[size_hdr..].as_ptr() as *mut PciDevRequest) };
|
||||
*pci_dev = *s;
|
||||
}
|
||||
}
|
||||
|
||||
buffer
|
||||
|
@@ -23,7 +23,7 @@ use log::{debug, error, info, trace, warn};
|
||||
use timerfd::{SetTimeFlags, TimerFd, TimerState};
|
||||
|
||||
pub use crate::dev_mgr_service::{
|
||||
CpuDevRequest, DevMgrRequest, DevMgrResponse, DevMgrService, MmioDevRequest,
|
||||
CpuDevRequest, DevMgrRequest, DevMgrResponse, DevMgrService, MmioDevRequest, PciDevRequest,
|
||||
};
|
||||
|
||||
const SERVER_PORT: u32 = 0xDB;
|
||||
|
@@ -2,6 +2,7 @@
|
||||
//
|
||||
// SPDX-License-Identifier: Apache-2.0 AND BSD-3-Clause
|
||||
|
||||
use std::any::Any;
|
||||
use std::sync::atomic::{AtomicU32, Ordering};
|
||||
use std::sync::{Arc, Mutex, MutexGuard};
|
||||
|
||||
@@ -484,7 +485,7 @@ where
|
||||
resources
|
||||
}
|
||||
|
||||
fn as_any(&self) -> &dyn std::any::Any {
|
||||
fn as_any(&self) -> &dyn Any {
|
||||
self
|
||||
}
|
||||
}
|
||||
|
@@ -36,9 +36,11 @@ use dbs_virtio_devices::{
|
||||
VirtioDevice,
|
||||
};
|
||||
|
||||
#[cfg(feature = "host-device")]
|
||||
use dbs_pci::VfioPciDevice;
|
||||
#[cfg(all(feature = "hotplug", feature = "dbs-upcall"))]
|
||||
use dbs_upcall::{
|
||||
DevMgrRequest, DevMgrService, MmioDevRequest, UpcallClient, UpcallClientError,
|
||||
DevMgrRequest, DevMgrService, MmioDevRequest, PciDevRequest, UpcallClient, UpcallClientError,
|
||||
UpcallClientRequest, UpcallClientResponse,
|
||||
};
|
||||
#[cfg(feature = "hotplug")]
|
||||
@@ -46,6 +48,8 @@ use dbs_virtio_devices::vsock::backend::VsockInnerConnector;
|
||||
|
||||
use crate::address_space_manager::GuestAddressSpaceImpl;
|
||||
use crate::api::v1::InstanceInfo;
|
||||
#[cfg(feature = "host-device")]
|
||||
use crate::device_manager::vfio_dev_mgr::PciSystemManager;
|
||||
use crate::error::StartMicroVmError;
|
||||
use crate::resource_manager::ResourceManager;
|
||||
use crate::vm::{KernelConfigInfo, Vm, VmConfigInfo};
|
||||
@@ -169,6 +173,11 @@ pub enum DeviceMgrError {
|
||||
/// Failed to free device resource.
|
||||
#[error("failed to free device resources: {0}")]
|
||||
ResourceError(#[source] crate::resource_manager::ResourceError),
|
||||
|
||||
#[cfg(feature = "host-device")]
|
||||
/// Error from Vfio Pci
|
||||
#[error("failed to do vfio pci operation: {0:?}")]
|
||||
VfioPci(#[source] dbs_pci::VfioPciError),
|
||||
}
|
||||
|
||||
/// Specialized version of `std::result::Result` for device manager operations.
|
||||
@@ -273,6 +282,8 @@ pub struct DeviceOpContext {
|
||||
address_space: Option<AddressSpace>,
|
||||
logger: slog::Logger,
|
||||
is_hotplug: bool,
|
||||
#[cfg(all(feature = "hotplug", feature = "host-device"))]
|
||||
pci_hotplug_enabled: bool,
|
||||
|
||||
#[cfg(all(feature = "hotplug", feature = "dbs-upcall"))]
|
||||
upcall_client: Option<Arc<UpcallClient<DevMgrService>>>,
|
||||
@@ -304,6 +315,12 @@ impl DeviceOpContext {
|
||||
};
|
||||
let logger = device_mgr.logger.new(slog::o!());
|
||||
|
||||
#[cfg(all(feature = "hotplug", feature = "host-device"))]
|
||||
let pci_hotplug_enabled = vm_config
|
||||
.clone()
|
||||
.map(|c| c.pci_hotplug_enabled)
|
||||
.unwrap_or(false);
|
||||
|
||||
DeviceOpContext {
|
||||
epoll_mgr,
|
||||
io_context,
|
||||
@@ -314,6 +331,8 @@ impl DeviceOpContext {
|
||||
address_space,
|
||||
logger,
|
||||
is_hotplug,
|
||||
#[cfg(all(feature = "hotplug", feature = "host-device"))]
|
||||
pci_hotplug_enabled,
|
||||
#[cfg(all(feature = "hotplug", feature = "dbs-upcall"))]
|
||||
upcall_client: None,
|
||||
#[cfg(feature = "dbs-virtio-devices")]
|
||||
@@ -526,6 +545,37 @@ impl DeviceOpContext {
|
||||
|
||||
self.call_hotplug_device(req, callback)
|
||||
}
|
||||
|
||||
#[cfg(feature = "host-device")]
|
||||
pub(crate) fn insert_hotplug_pci_device(
|
||||
&self,
|
||||
dev: &Arc<dyn DeviceIo>,
|
||||
callback: Option<Box<dyn Fn(UpcallClientResponse) + Send>>,
|
||||
) -> Result<()> {
|
||||
if !self.is_hotplug || !self.pci_hotplug_enabled {
|
||||
return Err(DeviceMgrError::InvalidOperation);
|
||||
}
|
||||
|
||||
let (busno, devfn) = DeviceManager::get_pci_device_info(dev)?;
|
||||
let req = DevMgrRequest::AddPciDev(PciDevRequest { busno, devfn });
|
||||
|
||||
self.call_hotplug_device(req, callback)
|
||||
}
|
||||
|
||||
#[cfg(feature = "host-device")]
|
||||
pub(crate) fn remove_hotplug_pci_device(
|
||||
&self,
|
||||
dev: &Arc<dyn DeviceIo>,
|
||||
callback: Option<Box<dyn Fn(UpcallClientResponse) + Send>>,
|
||||
) -> Result<()> {
|
||||
if !self.is_hotplug || !self.pci_hotplug_enabled {
|
||||
return Err(DeviceMgrError::InvalidOperation);
|
||||
}
|
||||
let (busno, devfn) = DeviceManager::get_pci_device_info(dev)?;
|
||||
let req = DevMgrRequest::DelPciDev(PciDevRequest { busno, devfn });
|
||||
|
||||
self.call_hotplug_device(req, callback)
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(all(feature = "hotplug", feature = "acpi"))]
|
||||
@@ -1152,6 +1202,30 @@ impl DeviceManager {
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(feature = "host-device")]
|
||||
fn get_pci_device_info(device: &Arc<dyn DeviceIo>) -> Result<(u8, u8)> {
|
||||
if let Some(pci_dev) = device
|
||||
.as_any()
|
||||
.downcast_ref::<VfioPciDevice<PciSystemManager>>()
|
||||
{
|
||||
// reference from kernel: include/uapi/linux/pci.h
|
||||
let busno = pci_dev.bus_id().map_err(DeviceMgrError::VfioPci)?;
|
||||
let slot = pci_dev.device_id();
|
||||
let func = 0;
|
||||
// The slot/function address of each device is encoded
|
||||
// in a single byte as follows:
|
||||
//
|
||||
// 7:3 = slot
|
||||
// 2:0 = function
|
||||
// together those 8 bits combined as devfn value
|
||||
let devfn = (((slot) & 0x1f) << 3) | ((func) & 0x07);
|
||||
|
||||
return Ok((busno, devfn));
|
||||
}
|
||||
|
||||
Err(DeviceMgrError::GetDeviceResource)
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(feature = "hotplug")]
|
||||
@@ -1282,6 +1356,7 @@ mod tests {
|
||||
sockets: 1,
|
||||
},
|
||||
vpmu_feature: 0,
|
||||
pci_hotplug_enabled: false,
|
||||
};
|
||||
vm.set_vm_config(vm_config.clone());
|
||||
vm.init_guest_memory().unwrap();
|
||||
|
@@ -18,19 +18,17 @@ use std::os::fd::RawFd;
|
||||
use std::path::Path;
|
||||
use std::sync::{Arc, Weak};
|
||||
|
||||
use super::StartMicroVmError;
|
||||
use crate::address_space_manager::{GuestAddressSpaceImpl, GuestMemoryImpl};
|
||||
use crate::config_manager::{ConfigItem, DeviceConfigInfo, DeviceConfigInfos};
|
||||
use crate::device_manager::{DeviceManagerContext, DeviceMgrError, DeviceOpContext};
|
||||
use crate::resource_manager::{ResourceError, ResourceManager};
|
||||
use crossbeam_channel::Sender;
|
||||
use dbs_device::resources::Resource::LegacyIrq;
|
||||
use dbs_device::resources::ResourceConstraint;
|
||||
use dbs_device::resources::{DeviceResources, Resource, ResourceConstraint};
|
||||
use dbs_device::DeviceIo;
|
||||
use dbs_interrupt::KvmIrqManager;
|
||||
#[cfg(target_arch = "aarch64")]
|
||||
use dbs_pci::ECAM_SPACE_LENGTH;
|
||||
use dbs_pci::{VfioPciDevice, VENDOR_NVIDIA};
|
||||
use dbs_upcall::{DevMgrResponse, UpcallClientResponse};
|
||||
use kvm_ioctls::{DeviceFd, VmFd};
|
||||
use log::error;
|
||||
use serde_derive::{Deserialize, Serialize};
|
||||
use vfio_ioctls::{VfioContainer, VfioDevice};
|
||||
use vm_memory::{
|
||||
@@ -38,6 +36,12 @@ use vm_memory::{
|
||||
MemoryRegionAddress,
|
||||
};
|
||||
|
||||
use super::StartMicroVmError;
|
||||
use crate::address_space_manager::{GuestAddressSpaceImpl, GuestMemoryImpl};
|
||||
use crate::config_manager::{ConfigItem, DeviceConfigInfo, DeviceConfigInfos};
|
||||
use crate::device_manager::{DeviceManagerContext, DeviceMgrError, DeviceOpContext};
|
||||
use crate::resource_manager::{ResourceError, ResourceManager};
|
||||
|
||||
// The flag of whether to use the shared irq.
|
||||
const USE_SHARED_IRQ: bool = true;
|
||||
|
||||
@@ -245,9 +249,35 @@ impl VfioDeviceMgr {
|
||||
}
|
||||
}
|
||||
|
||||
/// Insert or update a VFIO device into the manager.
|
||||
pub fn insert_device(&mut self, config: HostDeviceConfig) -> Result<()> {
|
||||
let _device_index = self.info_list.insert_or_update(&config)?;
|
||||
/// Insert or update a VFIO device into the manager.ig)?;
|
||||
pub fn insert_device(
|
||||
&mut self,
|
||||
ctx: &mut DeviceOpContext,
|
||||
config: HostDeviceConfig,
|
||||
) -> Result<()> {
|
||||
if !cfg!(feature = "hotplug") && ctx.is_hotplug {
|
||||
return Err(VfioDeviceError::UpdateNotAllowedPostBoot);
|
||||
}
|
||||
slog::info!(
|
||||
ctx.logger(),
|
||||
"add VFIO device configuration";
|
||||
"subsystem" => "vfio_dev_mgr",
|
||||
"hostdev_id" => &config.hostdev_id,
|
||||
"bdf" => &config.dev_config.bus_slot_func,
|
||||
);
|
||||
let device_index = self.info_list.insert_or_update(&config)?;
|
||||
// Handle device hotplug case
|
||||
if ctx.is_hotplug {
|
||||
slog::info!(
|
||||
ctx.logger(),
|
||||
"attach VFIO device";
|
||||
"subsystem" => "vfio_dev_mgr",
|
||||
"hostdev_id" => &config.hostdev_id,
|
||||
"bdf" => &config.dev_config.bus_slot_func,
|
||||
);
|
||||
self.add_device(ctx, &config, device_index)?;
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
@@ -256,6 +286,17 @@ impl VfioDeviceMgr {
|
||||
&mut self,
|
||||
ctx: &mut DeviceOpContext,
|
||||
) -> std::result::Result<(), StartMicroVmError> {
|
||||
// create and attach pci root bus
|
||||
#[cfg(all(feature = "hotplug", feature = "host-device"))]
|
||||
if ctx.pci_hotplug_enabled {
|
||||
let _ = self
|
||||
.create_pci_manager(
|
||||
ctx.irq_manager.clone(),
|
||||
ctx.io_context.clone(),
|
||||
ctx.res_manager.clone(),
|
||||
)
|
||||
.map_err(StartMicroVmError::CreateVfioDevice)?;
|
||||
}
|
||||
for (idx, info) in self.info_list.clone().iter().enumerate() {
|
||||
self.create_device(&info.config, ctx, idx)
|
||||
.map_err(StartMicroVmError::CreateVfioDevice)?;
|
||||
@@ -263,6 +304,85 @@ impl VfioDeviceMgr {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub fn remove_device(&mut self, ctx: &mut DeviceOpContext, hostdev_id: &str) -> Result<()> {
|
||||
if !cfg!(feature = "hotplug") {
|
||||
return Err(VfioDeviceError::UpdateNotAllowedPostBoot);
|
||||
}
|
||||
|
||||
slog::info!(
|
||||
ctx.logger(),
|
||||
"remove VFIO device";
|
||||
"subsystem" => "vfio_dev_mgr",
|
||||
"hostdev_id" => hostdev_id,
|
||||
);
|
||||
let device_index = self
|
||||
.get_index_of_hostdev_id(hostdev_id)
|
||||
.ok_or(VfioDeviceError::InvalidConfig)?;
|
||||
let mut info = self
|
||||
.info_list
|
||||
.remove(device_index)
|
||||
.ok_or(VfioDeviceError::InvalidConfig)?;
|
||||
|
||||
self.remove_vfio_device(ctx, &mut info)
|
||||
}
|
||||
|
||||
/// prepare to remove device
|
||||
pub fn prepare_remove_device(
|
||||
&self,
|
||||
ctx: &DeviceOpContext,
|
||||
hostdev_id: &str,
|
||||
result_sender: Sender<Option<i32>>,
|
||||
) -> Result<()> {
|
||||
if !cfg!(feature = "hotplug") {
|
||||
return Err(VfioDeviceError::UpdateNotAllowedPostBoot);
|
||||
}
|
||||
|
||||
slog::info!(
|
||||
ctx.logger(),
|
||||
"prepare remove VFIO device";
|
||||
"subsystem" => "vfio_dev_mgr",
|
||||
"hostdev_id" => hostdev_id,
|
||||
);
|
||||
|
||||
let device_index = self
|
||||
.get_index_of_hostdev_id(hostdev_id)
|
||||
.ok_or(VfioDeviceError::InvalidConfig)?;
|
||||
|
||||
let info = &self.info_list[device_index];
|
||||
if let Some(dev) = info.device.as_ref() {
|
||||
let callback: Option<Box<dyn Fn(UpcallClientResponse) + Send>> =
|
||||
Some(Box::new(move |result| match result {
|
||||
UpcallClientResponse::DevMgr(response) => {
|
||||
if let DevMgrResponse::Other(resp) = response {
|
||||
if let Err(e) = result_sender.send(Some(resp.result)) {
|
||||
error!("send upcall result failed, due to {:?}!", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
UpcallClientResponse::UpcallReset => {
|
||||
if let Err(e) = result_sender.send(None) {
|
||||
error!("send upcall result failed, due to {:?}!", e);
|
||||
}
|
||||
}
|
||||
#[cfg(test)]
|
||||
UpcallClientResponse::FakeResponse => {}
|
||||
}));
|
||||
ctx.remove_hotplug_pci_device(dev, callback)
|
||||
.map_err(VfioDeviceError::VfioDeviceMgr)?
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn remove_vfio_device(
|
||||
&mut self,
|
||||
ctx: &mut DeviceOpContext,
|
||||
info: &mut DeviceConfigInfo<HostDeviceConfig>,
|
||||
) -> Result<()> {
|
||||
let device = info.device.take().ok_or(VfioDeviceError::InvalidConfig)?;
|
||||
self.remove_pci_vfio_device(&device, ctx)?;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Start all VFIO devices.
|
||||
pub fn start_devices(&mut self, vm_as: &GuestAddressSpaceImpl) -> Result<()> {
|
||||
if self.vfio_container.is_some() {
|
||||
@@ -310,6 +430,26 @@ impl VfioDeviceMgr {
|
||||
self.info_list[idx].device = Some(device.clone());
|
||||
Ok(device)
|
||||
}
|
||||
|
||||
fn add_device(
|
||||
&mut self,
|
||||
ctx: &mut DeviceOpContext,
|
||||
cfg: &HostDeviceConfig,
|
||||
idx: usize,
|
||||
) -> Result<()> {
|
||||
let dev = self.create_device(cfg, ctx, idx)?;
|
||||
if self.locked_vm_size == 0 && self.vfio_container.is_some() {
|
||||
let vm_as = ctx
|
||||
.get_vm_as()
|
||||
.map_err(|_| VfioDeviceError::InternalError)?;
|
||||
let vm_memory = vm_as.memory();
|
||||
|
||||
self.register_memory(vm_memory.deref())?;
|
||||
}
|
||||
ctx.insert_hotplug_pci_device(&dev, None)
|
||||
.map_err(VfioDeviceError::VfioDeviceMgr)
|
||||
}
|
||||
|
||||
/// Gets the index of the device with the specified `hostdev_id` if it exists in the list.
|
||||
fn get_index_of_hostdev_id(&self, id: &str) -> Option<usize> {
|
||||
self.info_list
|
||||
@@ -379,12 +519,14 @@ impl VfioDeviceMgr {
|
||||
self.locked_vm_size -= size;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub(crate) fn update_memory(&mut self, region: &GuestRegionMmap) -> Result<()> {
|
||||
if self.locked_vm_size != 0 {
|
||||
self.register_memory_region(region)?;
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub(crate) fn build_sysfs_path(cfg: &HostDeviceConfig) -> Result<String> {
|
||||
if cfg.sysfs_path.is_empty() {
|
||||
let (bdf, domain) = (
|
||||
@@ -499,6 +641,52 @@ impl VfioDeviceMgr {
|
||||
Ok(vfio_pci_device)
|
||||
}
|
||||
|
||||
fn remove_pci_vfio_device(
|
||||
&mut self,
|
||||
device: &Arc<dyn DeviceIo>,
|
||||
ctx: &mut DeviceOpContext,
|
||||
) -> Result<()> {
|
||||
// safe to unwrap because type is decided
|
||||
let vfio_pci_device = device
|
||||
.as_any()
|
||||
.downcast_ref::<VfioPciDevice<PciSystemManager>>()
|
||||
.unwrap();
|
||||
|
||||
let device_id = vfio_pci_device.device_id() as u32;
|
||||
|
||||
// safe to unwrap because pci vfio manager is already created
|
||||
let _ = self
|
||||
.pci_vfio_manager
|
||||
.as_mut()
|
||||
.unwrap()
|
||||
.free_device_id(device_id)
|
||||
.ok_or(VfioDeviceError::InvalidDeviceID(device_id))?;
|
||||
|
||||
let resources = vfio_pci_device.get_assigned_resources();
|
||||
let vendor_id = vfio_pci_device.vendor_id();
|
||||
let filtered_resources = if vendor_id == VENDOR_NVIDIA {
|
||||
let mut filtered_resources = DeviceResources::new();
|
||||
for resource in resources.get_all_resources() {
|
||||
if let Resource::LegacyIrq(_) = resource {
|
||||
continue;
|
||||
} else {
|
||||
filtered_resources.append(resource.clone())
|
||||
}
|
||||
}
|
||||
filtered_resources
|
||||
} else {
|
||||
resources
|
||||
};
|
||||
|
||||
ctx.res_manager.free_device_resources(&filtered_resources);
|
||||
|
||||
vfio_pci_device
|
||||
.clear_device()
|
||||
.map_err(VfioDeviceError::VfioPciError)?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub(crate) fn create_pci_manager(
|
||||
&mut self,
|
||||
irq_manager: Arc<KvmIrqManager>,
|
||||
@@ -516,7 +704,7 @@ impl VfioDeviceMgr {
|
||||
}
|
||||
Ok(self.pci_vfio_manager.as_mut().unwrap())
|
||||
}
|
||||
|
||||
|
||||
/// Get the PCI manager to support PCI device passthrough
|
||||
pub fn get_pci_manager(&mut self) -> Option<&mut Arc<PciSystemManager>> {
|
||||
self.pci_vfio_manager.as_mut()
|
||||
|
@@ -138,6 +138,9 @@ pub struct VmConfigInfo {
|
||||
|
||||
/// sock path
|
||||
pub serial_path: Option<String>,
|
||||
|
||||
/// Enable PCI device hotplug or not
|
||||
pub pci_hotplug_enabled: bool,
|
||||
}
|
||||
|
||||
impl Default for VmConfigInfo {
|
||||
@@ -157,6 +160,7 @@ impl Default for VmConfigInfo {
|
||||
mem_file_path: String::from(""),
|
||||
mem_size_mib: 128,
|
||||
serial_path: None,
|
||||
pci_hotplug_enabled: false,
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -182,7 +186,7 @@ pub struct Vm {
|
||||
shared_info: Arc<RwLock<InstanceInfo>>,
|
||||
|
||||
address_space: AddressSpaceMgr,
|
||||
device_manager: DeviceManager,
|
||||
pub device_manager: DeviceManager,
|
||||
dmesg_fifo: Option<Box<dyn io::Write + Send>>,
|
||||
kernel_config: Option<KernelConfigInfo>,
|
||||
logger: slog::Logger,
|
||||
@@ -928,6 +932,7 @@ pub mod tests {
|
||||
sockets: 1,
|
||||
},
|
||||
vpmu_feature: 0,
|
||||
pci_hotplug_enabled: false,
|
||||
};
|
||||
|
||||
let mut vm = create_vm_instance();
|
||||
@@ -960,6 +965,7 @@ pub mod tests {
|
||||
sockets: 1,
|
||||
},
|
||||
vpmu_feature: 0,
|
||||
pci_hotplug_enabled: false,
|
||||
};
|
||||
vm.set_vm_config(vm_config);
|
||||
assert!(vm.init_guest_memory().is_ok());
|
||||
@@ -1008,6 +1014,7 @@ pub mod tests {
|
||||
sockets: 1,
|
||||
},
|
||||
vpmu_feature: 0,
|
||||
pci_hotplug_enabled: false,
|
||||
};
|
||||
|
||||
vm.set_vm_config(vm_config);
|
||||
@@ -1084,6 +1091,7 @@ pub mod tests {
|
||||
sockets: 1,
|
||||
},
|
||||
vpmu_feature: 0,
|
||||
pci_hotplug_enabled: false,
|
||||
};
|
||||
|
||||
vm.set_vm_config(vm_config);
|
||||
|
Reference in New Issue
Block a user