vm-migration: Define the Snapshottable and Transportable traits

A Snapshottable component can snapshot itself and
provide a MigrationSnapshot payload as a result.

A MigrationSnapshot payload is a map of component IDs to a list of
migration sections (MigrationSection). As component can be made of
several Migratable sub-components (e.g. the DeviceManager and its
device objects), a migration snapshot can be made of multiple snapshot
itself.
A snapshot is a list of migration sections, each section being a
component state snapshot. Having multiple sections allows for easier and
backward compatible migration payload extensions.

Once created, a migratable component snapshot may be transported and this
is what the Transportable trait defines, through 2 methods: send and recv.

Signed-off-by: Samuel Ortiz <sameo@linux.intel.com>
Signed-off-by: Yi Sun <yi.y.sun@linux.intel.com>
This commit is contained in:
Samuel Ortiz 2019-05-02 00:59:51 +08:00 committed by Rob Bradford
parent 2d17f4384a
commit 1b1a2175ca
25 changed files with 788 additions and 615 deletions

1105
Cargo.lock generated

File diff suppressed because it is too large Load Diff

View File

@ -55,6 +55,7 @@ members = [
"vmm",
"vm-virtio",
"vm-device",
"vm-migration",
"vhost_user_block",
"vhost_user_backend",
"vhost_user_fs",

View File

@ -1,5 +1,3 @@
extern crate serde;
extern crate thiserror;
extern crate vm_memory;
pub mod interrupt;
@ -9,8 +7,6 @@ use vm_memory::{
MemoryRegionAddress,
};
use thiserror::Error;
/// Trait meant for triggering the DMA mapping update related to an external
/// device not managed fully through virtio. It is dedicated to virtio-iommu
/// in order to trigger the map update anytime the mapping is updated from the
@ -23,34 +19,6 @@ pub trait ExternalDmaMapping: Send + Sync {
fn unmap(&self, iova: u64, size: u64) -> std::result::Result<(), std::io::Error>;
}
#[derive(Error, Debug)]
pub enum MigratableError {
#[error("Failed to pause migratable component: {0}")]
Pause(#[source] anyhow::Error),
#[error("Failed to resume migratable component: {0}")]
Resume(#[source] anyhow::Error),
}
/// A Pausable component can be paused and resumed.
pub trait Pausable {
/// Pause the component.
fn pause(&mut self) -> std::result::Result<(), MigratableError>;
/// Resume the component.
fn resume(&mut self) -> std::result::Result<(), MigratableError>;
}
/// A snapshotable component can be snapshoted.
pub trait Snapshotable {}
/// Trait to be implemented by any component (device, CPU, RAM, etc) that
/// can be migrated.
/// All migratable components are paused before being snapshotted, and then
/// eventually resumed. Thus any Migratable component must be both Pausable
/// and Snapshotable.
pub trait Migratable: Send + Pausable + Snapshotable {}
fn get_region_host_address_range(
region: &GuestRegionMmap,
addr: MemoryRegionAddress,

12
vm-migration/Cargo.toml Normal file
View File

@ -0,0 +1,12 @@
[package]
name = "vm-migration"
version = "0.1.0"
authors = ["The Cloud Hypervisor Authors"]
edition = "2018"
[dependencies]
anyhow = "1.0"
thiserror = "1.0"
serde = {version = ">=1.0.27", features = ["rc"] }
serde_derive = ">=1.0.27"
serde_json = ">=1.0.9"

162
vm-migration/src/lib.rs Normal file
View File

@ -0,0 +1,162 @@
// Copyright © 2019 Intel Corporation
//
// SPDX-License-Identifier: Apache-2.0 AND BSD-3-Clause
//
extern crate serde;
extern crate thiserror;
#[macro_use]
extern crate serde_derive;
use thiserror::Error;
#[derive(Error, Debug)]
pub enum MigratableError {
#[error("Failed to pause migratable component: {0}")]
Pause(#[source] anyhow::Error),
#[error("Failed to resume migratable component: {0}")]
Resume(#[source] anyhow::Error),
#[error("Failed to snapshot migratable component: {0}")]
Snapshot(#[source] anyhow::Error),
#[error("Failed to restore migratable component: {0}")]
Restore(#[source] anyhow::Error),
#[error("Failed to send migratable component snapshot: {0}")]
MigrateSend(#[source] anyhow::Error),
#[error("Failed to receive migratable component snapshot: {0}")]
MigrateReceive(#[source] anyhow::Error),
}
/// A Pausable component can be paused and resumed.
pub trait Pausable {
/// Pause the component.
fn pause(&mut self) -> std::result::Result<(), MigratableError> {
Ok(())
}
/// Resume the component.
fn resume(&mut self) -> std::result::Result<(), MigratableError> {
Ok(())
}
}
/// A Snapshottable component snapshot section.
/// Migratable component can split their migration snapshot into
/// separate sections.
/// Splitting a component migration data into different sections
/// allows for easier and forward compatible extensions.
#[derive(Clone, Default, Deserialize, Serialize)]
pub struct SnapshotDataSection {
/// The section id.
pub id: String,
/// The section serialized snapshot.
pub snapshot: Vec<u8>,
}
/// A Snapshottable component's snapshot is a tree of snapshots, where leafs
/// contain the snapshot data. Nodes of this tree track all their children
/// through the snapshots field, which is basically their sub-components.
/// Leaves will typically have an empty snapshots map, while nodes usually
/// carry an empty snapshot_data.
///
/// For example, a device manager snapshot is the composition of all its
/// devices snapshots. The device manager Snapshot would have no snapshot_data
/// but one Snapshot child per tracked device. Then each device's Snapshot
/// would carry an empty snapshots map but a map of SnapshotDataSection, i.e.
/// the actual device snapshot data.
#[derive(Clone, Default, Deserialize, Serialize)]
pub struct Snapshot {
/// The Snapshottable component id.
pub id: String,
/// The Snapshottable component snapshots.
pub snapshots: std::collections::HashMap<String, Box<Snapshot>>,
/// The Snapshottable component's snapshot data.
/// A map of snapshot sections, indexed by the section ids.
pub snapshot_data: std::collections::HashMap<String, SnapshotDataSection>,
}
impl Snapshot {
/// Create an empty Snapshot.
pub fn new(id: &str) -> Self {
Snapshot {
id: id.to_string(),
..Default::default()
}
}
/// Add a sub-component's Snapshot to the Snapshot.
pub fn add_snapshot(&mut self, snapshot: Snapshot) {
self.snapshots
.insert(snapshot.id.clone(), Box::new(snapshot));
}
/// Add a SnapshotDatasection to the component snapshot data.
pub fn add_data_section(&mut self, section: SnapshotDataSection) {
self.snapshot_data.insert(section.id.clone(), section);
}
}
/// A snapshottable component can be snapshotted.
pub trait Snapshottable: Pausable {
/// The snapshottable component id.
fn id(&self) -> String {
String::new()
}
/// Take a component snapshot.
fn snapshot(&self) -> std::result::Result<Snapshot, MigratableError> {
Ok(Snapshot::new(""))
}
/// Restore a component from its snapshot.
fn restore(&mut self, _snapshot: Snapshot) -> std::result::Result<(), MigratableError> {
Ok(())
}
}
/// A transportable component can be sent or receive to a specific URL.
///
/// This trait is meant to be used for component that have custom
/// transport handlers.
pub trait Transportable: Pausable + Snapshottable {
/// Send a component snapshot.
///
/// # Arguments
///
/// * `snapshot` - The migratable component snapshot to send.
/// * `destination_url` - The destination URL to send the snapshot to. This
/// could be an HTTP endpoint, a TCP address or a local file.
fn send(
&self,
_snapshot: &Snapshot,
_destination_url: &str,
) -> std::result::Result<(), MigratableError> {
Ok(())
}
/// Receive a component snapshot.
///
/// # Arguments
///
/// * `source_url` - The source URL to fetch the snapshot from. This could be an HTTP
/// endpoint, a TCP address or a local file.
fn recv(&self, _source_url: &str) -> std::result::Result<Snapshot, MigratableError> {
Ok(Snapshot::new(""))
}
}
/// Trait to be implemented by any component (device, CPU, RAM, etc) that
/// can be migrated.
/// All migratable components are paused before being snapshotted, and then
/// eventually resumed. Thus any Migratable component must be both Pausable
/// and Snapshottable.
/// Moreover a migratable component can be transported to a remote or local
/// destination and thus must be Transportable.
pub trait Migratable: Send + Pausable + Snapshottable + Transportable {}

View File

@ -24,5 +24,6 @@ virtio-bindings = { git = "https://github.com/rust-vmm/virtio-bindings", version
vm-allocator = { path = "../vm-allocator" }
vm-device = { path = "../vm-device" }
vm-memory = { version = "0.2.0", features = ["backend-mmap", "backend-atomic"] }
vm-migration = { path = "../vm-migration" }
vmm-sys-util = ">=0.3.1"
vhost_rs = { git = "https://github.com/cloud-hypervisor/vhost", branch = "dragonball", package = "vhost", features = ["vhost-user-master", "vhost-user-slave"] }

View File

@ -31,11 +31,11 @@ use std::sync::atomic::{AtomicBool, Ordering};
use std::sync::{Arc, Mutex};
use std::thread;
use virtio_bindings::bindings::virtio_blk::*;
use vm_device::{Migratable, MigratableError, Pausable, Snapshotable};
use vm_memory::{
ByteValued, Bytes, GuestAddress, GuestAddressSpace, GuestMemory, GuestMemoryAtomic,
GuestMemoryError, GuestMemoryMmap,
};
use vm_migration::{Migratable, MigratableError, Pausable, Snapshottable, Transportable};
use vmm_sys_util::{eventfd::EventFd, seek_hole::SeekHole, write_zeroes::PunchHole};
const SECTOR_SHIFT: u8 = 9;
@ -1051,5 +1051,6 @@ impl<T: 'static + DiskFile + Send> VirtioDevice for Block<T> {
}
virtio_pausable!(Block, T: 'static + DiskFile + Send);
impl<T: 'static + DiskFile + Send> Snapshotable for Block<T> {}
impl<T: 'static + DiskFile + Send> Snapshottable for Block<T> {}
impl<T: 'static + DiskFile + Send> Transportable for Block<T> {}
impl<T: 'static + DiskFile + Send> Migratable for Block<T> {}

View File

@ -20,8 +20,8 @@ use std::result;
use std::sync::atomic::{AtomicBool, AtomicU64, Ordering};
use std::sync::{Arc, Mutex};
use std::thread;
use vm_device::{Migratable, MigratableError, Pausable, Snapshotable};
use vm_memory::{ByteValued, Bytes, GuestAddressSpace, GuestMemoryAtomic, GuestMemoryMmap};
use vm_migration::{Migratable, MigratableError, Pausable, Snapshottable, Transportable};
use vmm_sys_util::eventfd::EventFd;
const QUEUE_SIZE: u16 = 256;
@ -563,5 +563,6 @@ impl VirtioDevice for Console {
}
virtio_pausable!(Console);
impl Snapshotable for Console {}
impl Snapshottable for Console {}
impl Transportable for Console {}
impl Migratable for Console {}

View File

@ -21,11 +21,12 @@ use std::result;
use std::sync::atomic::{AtomicBool, Ordering};
use std::sync::{Arc, RwLock};
use std::thread;
use vm_device::{ExternalDmaMapping, Migratable, MigratableError, Pausable, Snapshotable};
use vm_device::ExternalDmaMapping;
use vm_memory::{
Address, ByteValued, Bytes, GuestAddress, GuestAddressSpace, GuestMemoryAtomic,
GuestMemoryError, GuestMemoryMmap,
};
use vm_migration::{Migratable, MigratableError, Pausable, Snapshottable, Transportable};
use vmm_sys_util::eventfd::EventFd;
/// Queues sizes
@ -1038,5 +1039,6 @@ impl VirtioDevice for Iommu {
}
virtio_pausable!(Iommu);
impl Snapshotable for Iommu {}
impl Snapshottable for Iommu {}
impl Transportable for Iommu {}
impl Migratable for Iommu {}

View File

@ -30,11 +30,11 @@ use std::sync::atomic::{AtomicBool, AtomicU64, Ordering};
use std::sync::mpsc;
use std::sync::{Arc, Mutex};
use std::thread;
use vm_device::{Migratable, MigratableError, Pausable, Snapshotable};
use vm_memory::{
Address, ByteValued, Bytes, GuestAddress, GuestAddressSpace, GuestMemoryAtomic,
GuestMemoryError, GuestMemoryMmap, GuestMemoryRegion, GuestRegionMmap,
};
use vm_migration::{Migratable, MigratableError, Pausable, Snapshottable, Transportable};
use vmm_sys_util::eventfd::EventFd;
const QUEUE_SIZE: u16 = 128;
@ -955,5 +955,6 @@ impl VirtioDevice for Mem {
}
virtio_pausable!(Mem);
impl Snapshotable for Mem {}
impl Snapshottable for Mem {}
impl Transportable for Mem {}
impl Migratable for Mem {}

View File

@ -30,8 +30,8 @@ use std::sync::Arc;
use std::thread;
use std::vec::Vec;
use virtio_bindings::bindings::virtio_net::*;
use vm_device::{Migratable, MigratableError, Pausable, Snapshotable};
use vm_memory::{ByteValued, GuestAddressSpace, GuestMemoryAtomic, GuestMemoryMmap};
use vm_migration::{Migratable, MigratableError, Pausable, Snapshottable, Transportable};
use vmm_sys_util::eventfd::EventFd;
#[derive(Debug)]
@ -567,5 +567,6 @@ impl VirtioDevice for Net {
}
virtio_ctrl_q_pausable!(Net);
impl Snapshotable for Net {}
impl Snapshottable for Net {}
impl Transportable for Net {}
impl Migratable for Net {}

View File

@ -24,11 +24,11 @@ use std::result;
use std::sync::atomic::{AtomicBool, Ordering};
use std::sync::Arc;
use std::thread;
use vm_device::{Migratable, MigratableError, Pausable, Snapshotable};
use vm_memory::{
Address, ByteValued, Bytes, GuestAddress, GuestAddressSpace, GuestMemoryAtomic,
GuestMemoryError, GuestMemoryMmap, GuestUsize,
};
use vm_migration::{Migratable, MigratableError, Pausable, Snapshottable, Transportable};
use vmm_sys_util::eventfd::EventFd;
const QUEUE_SIZE: u16 = 256;
@ -506,5 +506,6 @@ impl VirtioDevice for Pmem {
}
virtio_pausable!(Pmem);
impl Snapshotable for Pmem {}
impl Snapshottable for Pmem {}
impl Transportable for Pmem {}
impl Migratable for Pmem {}

View File

@ -18,8 +18,8 @@ use std::result;
use std::sync::atomic::{AtomicBool, Ordering};
use std::sync::Arc;
use std::thread;
use vm_device::{Migratable, MigratableError, Pausable, Snapshotable};
use vm_memory::{Bytes, GuestAddressSpace, GuestMemoryAtomic, GuestMemoryMmap};
use vm_migration::{Migratable, MigratableError, Pausable, Snapshottable, Transportable};
use vmm_sys_util::eventfd::EventFd;
const QUEUE_SIZE: u16 = 256;
@ -357,5 +357,6 @@ impl VirtioDevice for Rng {
}
virtio_pausable!(Rng);
impl Snapshotable for Rng {}
impl Snapshottable for Rng {}
impl Transportable for Rng {}
impl Migratable for Rng {}

View File

@ -15,8 +15,8 @@ use std::result;
use std::sync::atomic::{AtomicUsize, Ordering};
use std::sync::{Arc, Mutex};
use vm_device::interrupt::InterruptSourceGroup;
use vm_device::{Migratable, MigratableError, Pausable, Snapshotable};
use vm_memory::{GuestAddress, GuestAddressSpace, GuestMemoryAtomic, GuestMemoryMmap};
use vm_migration::{Migratable, MigratableError, Pausable, Snapshottable, Transportable};
use vmm_sys_util::{errno::Result, eventfd::EventFd};
const VENDOR_ID: u32 = 0;
@ -354,5 +354,6 @@ impl Pausable for MmioDevice {
}
}
impl Snapshotable for MmioDevice {}
impl Snapshottable for MmioDevice {}
impl Transportable for MmioDevice {}
impl Migratable for MmioDevice {}

View File

@ -37,11 +37,11 @@ use vm_allocator::SystemAllocator;
use vm_device::interrupt::{
InterruptIndex, InterruptManager, InterruptSourceGroup, MsiIrqGroupConfig,
};
use vm_device::{Migratable, MigratableError, Pausable, Snapshotable};
use vm_memory::{
Address, ByteValued, GuestAddress, GuestAddressSpace, GuestMemoryAtomic, GuestMemoryMmap,
GuestUsize, Le32,
};
use vm_migration::{Migratable, MigratableError, Pausable, Snapshottable, Transportable};
use vmm_sys_util::{errno::Result, eventfd::EventFd};
#[allow(clippy::enum_variant_names)]
@ -916,5 +916,6 @@ impl Pausable for VirtioPciDevice {
}
}
impl Snapshotable for VirtioPciDevice {}
impl Snapshottable for VirtioPciDevice {}
impl Transportable for VirtioPciDevice {}
impl Migratable for VirtioPciDevice {}

View File

@ -26,8 +26,8 @@ use vhost_rs::vhost_user::{Master, VhostUserMaster, VhostUserMasterReqHandler};
use vhost_rs::VhostBackend;
use virtio_bindings::bindings::virtio_blk::*;
use virtio_bindings::bindings::virtio_ring::VIRTIO_RING_F_EVENT_IDX;
use vm_device::{Migratable, MigratableError, Pausable, Snapshotable};
use vm_memory::{ByteValued, GuestAddressSpace, GuestMemoryAtomic, GuestMemoryMmap};
use vm_migration::{Migratable, MigratableError, Pausable, Snapshottable, Transportable};
use vmm_sys_util::eventfd::EventFd;
struct SlaveReqHandler {}
@ -328,5 +328,6 @@ impl VirtioDevice for Blk {
}
virtio_pausable!(Blk);
impl Snapshotable for Blk {}
impl Snapshottable for Blk {}
impl Transportable for Blk {}
impl Migratable for Blk {}

View File

@ -26,10 +26,10 @@ use vhost_rs::vhost_user::{
HandlerResult, Master, MasterReqHandler, VhostUserMaster, VhostUserMasterReqHandler,
};
use vhost_rs::VhostBackend;
use vm_device::{Migratable, MigratableError, Pausable, Snapshotable};
use vm_memory::{
Address, ByteValued, GuestAddress, GuestAddressSpace, GuestMemoryAtomic, GuestMemoryMmap,
};
use vm_migration::{Migratable, MigratableError, Pausable, Snapshottable, Transportable};
use vmm_sys_util::eventfd::EventFd;
const NUM_QUEUE_OFFSET: usize = 1;
@ -555,5 +555,6 @@ impl VirtioDevice for Fs {
}
virtio_pausable!(Fs);
impl Snapshotable for Fs {}
impl Snapshottable for Fs {}
impl Transportable for Fs {}
impl Migratable for Fs {}

View File

@ -27,8 +27,8 @@ use vhost_rs::vhost_user::{Master, VhostUserMaster, VhostUserMasterReqHandler};
use vhost_rs::VhostBackend;
use virtio_bindings::bindings::virtio_net;
use virtio_bindings::bindings::virtio_ring;
use vm_device::{Migratable, MigratableError, Pausable, Snapshotable};
use vm_memory::{ByteValued, GuestAddressSpace, GuestMemoryAtomic, GuestMemoryMmap};
use vm_migration::{Migratable, MigratableError, Pausable, Snapshottable, Transportable};
use vmm_sys_util::eventfd::EventFd;
const DEFAULT_QUEUE_NUMBER: usize = 2;
@ -365,5 +365,6 @@ impl VirtioDevice for Net {
}
virtio_ctrl_q_pausable!(Net);
impl Snapshotable for Net {}
impl Snapshottable for Net {}
impl Transportable for Net {}
impl Migratable for Net {}

View File

@ -44,8 +44,8 @@ use std::result;
use std::sync::atomic::{AtomicBool, Ordering};
use std::sync::{Arc, RwLock};
use std::thread;
use vm_device::{Migratable, MigratableError, Pausable, Snapshotable};
use vm_memory::{GuestAddressSpace, GuestMemoryAtomic, GuestMemoryMmap};
use vm_migration::{Migratable, MigratableError, Pausable, Snapshottable, Transportable};
use vmm_sys_util::eventfd::EventFd;
const QUEUE_SIZE: u16 = 256;
@ -574,7 +574,8 @@ where
virtio_pausable!(Vsock, T: 'static + VsockBackend + Sync);
impl<B> Snapshotable for Vsock<B> where B: VsockBackend + Sync + 'static {}
impl<B> Snapshottable for Vsock<B> where B: VsockBackend + Sync + 'static {}
impl<B> Transportable for Vsock<B> where B: VsockBackend + Sync + 'static {}
impl<B> Migratable for Vsock<B> where B: VsockBackend + Sync + 'static {}
#[cfg(test)]

View File

@ -36,6 +36,7 @@ vfio = { path = "../vfio", optional = true }
vm-allocator = { path = "../vm-allocator" }
vm-device = { path = "../vm-device" }
vm-memory = { version = "0.2.0", features = ["backend-mmap", "backend-atomic"] }
vm-migration = { path = "../vm-migration" }
vm-virtio = { path = "../vm-virtio" }
vmm-sys-util = ">=0.3.1"
signal-hook = "0.1.13"

View File

@ -26,8 +26,8 @@ use std::sync::atomic::{AtomicBool, Ordering};
use std::sync::{Arc, Barrier, Mutex};
use std::thread;
use std::{fmt, io, result};
use vm_device::{Migratable, MigratableError, Pausable, Snapshotable};
use vm_memory::{Address, GuestAddress, GuestAddressSpace, GuestMemoryAtomic, GuestMemoryMmap};
use vm_migration::{Migratable, MigratableError, Pausable, Snapshottable, Transportable};
use vmm_sys_util::eventfd::EventFd;
use vmm_sys_util::signal::{register_signal_handler, SIGRTMIN};
@ -1068,5 +1068,6 @@ impl Pausable for CpuManager {
}
}
impl Snapshotable for CpuManager {}
impl Snapshottable for CpuManager {}
impl Transportable for CpuManager {}
impl Migratable for CpuManager {}

View File

@ -51,11 +51,11 @@ use vm_allocator::SystemAllocator;
use vm_device::interrupt::{
InterruptIndex, InterruptManager, LegacyIrqGroupConfig, MsiIrqGroupConfig,
};
use vm_device::{Migratable, MigratableError, Pausable, Snapshotable};
use vm_memory::guest_memory::FileOffset;
use vm_memory::{
Address, GuestAddress, GuestAddressSpace, GuestRegionMmap, GuestUsize, MmapRegion,
};
use vm_migration::{Migratable, MigratableError, Pausable, Snapshottable, Transportable};
#[cfg(feature = "pci_support")]
use vm_virtio::transport::VirtioPciDevice;
use vm_virtio::transport::VirtioTransport;
@ -2373,7 +2373,8 @@ impl Pausable for DeviceManager {
}
}
impl Snapshotable for DeviceManager {}
impl Snapshottable for DeviceManager {}
impl Transportable for DeviceManager {}
impl Migratable for DeviceManager {}
#[cfg(feature = "pci_support")]

View File

@ -27,7 +27,7 @@ use std::path::PathBuf;
use std::sync::mpsc::{Receiver, RecvError, SendError, Sender};
use std::sync::{Arc, Mutex};
use std::{result, thread};
use vm_device::Pausable;
use vm_migration::Pausable;
use vmm_sys_util::eventfd::EventFd;
pub mod api;

View File

@ -23,6 +23,7 @@ use vm_memory::{
GuestMemory, GuestMemoryAtomic, GuestMemoryMmap, GuestMemoryRegion, GuestRegionMmap,
GuestUsize, MmapRegion,
};
use vm_migration::{Migratable, Pausable, Snapshottable, Transportable};
const HOTPLUG_COUNT: usize = 8;
@ -884,3 +885,8 @@ impl Aml for MemoryManager {
bytes
}
}
impl Pausable for MemoryManager {}
impl Snapshottable for MemoryManager {}
impl Transportable for MemoryManager {}
impl Migratable for MemoryManager {}

View File

@ -48,11 +48,11 @@ use std::path::PathBuf;
use std::sync::{Arc, Mutex, RwLock};
use std::{result, str, thread};
use vm_allocator::{GsiApic, SystemAllocator};
use vm_device::{Migratable, MigratableError, Pausable, Snapshotable};
use vm_memory::{
Address, Bytes, GuestAddress, GuestAddressSpace, GuestMemory, GuestMemoryMmap,
GuestMemoryRegion, GuestUsize,
};
use vm_migration::{Migratable, MigratableError, Pausable, Snapshottable, Transportable};
use vmm_sys_util::eventfd::EventFd;
use vmm_sys_util::terminal::Terminal;
@ -966,7 +966,8 @@ impl Pausable for Vm {
}
}
impl Snapshotable for Vm {}
impl Snapshottable for Vm {}
impl Transportable for Vm {}
impl Migratable for Vm {}
#[cfg(test)]