diff --git a/Cargo.lock b/Cargo.lock index 3dd861e12..8921fe9d2 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3072,6 +3072,7 @@ dependencies = [ "flate2", "futures", "hex", + "http 0.2.12", "libc", "propolis-client", "propolis-server-config", @@ -3421,6 +3422,7 @@ name = "propolis" version = "0.1.0" dependencies = [ "anyhow", + "async-trait", "bhyve_api", "bitflags 2.5.0", "bitstruct", diff --git a/bin/propolis-server/src/lib/initializer.rs b/bin/propolis-server/src/lib/initializer.rs index e738d6337..060934d74 100644 --- a/bin/propolis-server/src/lib/initializer.rs +++ b/bin/propolis-server/src/lib/initializer.rs @@ -11,8 +11,8 @@ use std::sync::Arc; use std::time::{SystemTime, UNIX_EPOCH}; use crate::serial::Serial; -use crate::server::{BlockBackendMap, CrucibleBackendMap, DeviceMap}; use crate::stats::virtual_machine::VirtualMachine; +use crate::vm::{BlockBackendMap, CrucibleBackendMap, DeviceMap}; use anyhow::{Context, Result}; use crucible_client_types::VolumeConstructionRequest; pub use nexus_client::Client as NexusClient; @@ -185,7 +185,7 @@ impl<'a> MachineInitializer<'a> { pub fn initialize_chipset( &mut self, - event_handler: &Arc, + event_handler: &Arc, ) -> Result { let mut pci_builder = pci::topology::Builder::new(); for (name, bridge) in &self.spec.devices.pci_pci_bridges { @@ -371,7 +371,7 @@ impl<'a> MachineInitializer<'a> { Ok(()) } - fn create_storage_backend_from_spec( + async fn create_storage_backend_from_spec( &self, backend_spec: &instance_spec::v0::StorageBackendV0, backend_name: &str, @@ -409,9 +409,10 @@ impl<'a> MachineInitializer<'a> { self.log.new( slog::o!("component" => format!("crucible-{cru_id}")), ), - )?; + ) + .await?; - let crucible = Some((be.get_uuid()?, be.clone())); + let crucible = Some((be.get_uuid().await?, be.clone())); Ok(StorageBackendInstance { be, crucible }) } instance_spec::v0::StorageBackendV0::File(spec) => { @@ -480,7 +481,7 @@ impl<'a> MachineInitializer<'a> { /// /// On success, returns a map from Crucible backend IDs to Crucible /// backends. - pub fn initialize_storage_devices( + pub async fn initialize_storage_devices( &mut self, chipset: &RegisteredChipset, nexus_client: Option, @@ -537,7 +538,8 @@ impl<'a> MachineInitializer<'a> { backend_spec, backend_name, &nexus_client, - )?; + ) + .await?; self.block_backends.insert(backend_name.clone(), backend.clone()); match device_interface { diff --git a/bin/propolis-server/src/lib/migrate/destination.rs b/bin/propolis-server/src/lib/migrate/destination.rs index 18ba3f1ef..7f23beb15 100644 --- a/bin/propolis-server/src/lib/migrate/destination.rs +++ b/bin/propolis-server/src/lib/migrate/destination.rs @@ -9,13 +9,16 @@ use propolis::migrate::{ MigrateCtx, MigrateStateError, Migrator, PayloadOffer, PayloadOffers, }; use propolis::vmm; +use propolis_api_types::InstanceMigrateInitiateRequest; use slog::{error, info, trace, warn}; use std::convert::TryInto; use std::io; use std::net::SocketAddr; use std::sync::Arc; -use tokio::io::{AsyncRead, AsyncWrite}; -use tokio_tungstenite::WebSocketStream; +use tokio_tungstenite::tungstenite::protocol::frame::coding::CloseCode; +use tokio_tungstenite::tungstenite::protocol::CloseFrame; +use tokio_tungstenite::{tungstenite, WebSocketStream}; +use uuid::Uuid; use crate::migrate::codec; use crate::migrate::memx; @@ -24,57 +27,132 @@ use crate::migrate::probes; use crate::migrate::{ Device, MigrateError, MigratePhase, MigrateRole, MigrationState, PageIter, }; -use crate::vm::{MigrateTargetCommand, VmController}; +use crate::vm::ensure::{VmEnsureActive, VmEnsureNotStarted}; +use crate::vm::state_publisher::{ + ExternalStateUpdate, MigrationStateUpdate, StatePublisher, +}; use super::protocol::Protocol; +use super::MigrateConn; + +/// The interface to an arbitrary version of the target half of the live +/// migration protocol. +// +// Use `async_trait` here to help generate a `Send` bound on the futures +// returned by the functions in this trait. +#[async_trait::async_trait] +pub(crate) trait DestinationProtocol { + /// Runs live migration as a target, attempting to create a set of VM + /// objects in the process. On success, returns an "active VM" placeholder + /// that the caller can use to set up and start a state driver loop. + async fn run<'ensure>( + mut self, + ensure: VmEnsureNotStarted<'ensure>, + ) -> Result, MigrateError>; +} -/// Launches an attempt to migrate into a supplied instance using the supplied -/// source connection. -pub async fn migrate( - vm_controller: Arc, - command_tx: tokio::sync::mpsc::Sender, - conn: WebSocketStream, +/// Connects to a live migration source using the migration request information +/// in `migrate_info`, then negotiates a protocol version with that source. +/// Returns a [`DestinationProtocol`] implementation for the negotiated version +/// that the caller can use to run the migration. +pub(crate) async fn initiate( + log: &slog::Logger, + migrate_info: &InstanceMigrateInitiateRequest, local_addr: SocketAddr, - protocol: Protocol, -) -> Result<(), MigrateError> { - let err_tx = command_tx.clone(); - let mut proto = match protocol { - Protocol::RonV0 => DestinationProtocol::new( - vm_controller, - command_tx, - conn, - local_addr, - ), - }; +) -> Result { + let migration_id = migrate_info.migration_id; + + let log = log.new(slog::o!( + "migration_id" => migration_id.to_string(), + "migrate_role" => "destination", + "migrate_src_addr" => migrate_info.src_addr + )); + + info!(log, "negotiating migration as destination"); + + // Build upgrade request to the source instance + // (we do this by hand because it's hidden from the OpenAPI spec) + // TODO(#165): https (wss) + // TODO: We need to make sure the src_addr is a valid target + let src_migrate_url = format!( + "ws://{}/instance/migrate/{}/start", + migrate_info.src_addr, migration_id, + ); + info!(log, "Begin migration"; "src_migrate_url" => &src_migrate_url); + let (mut conn, _) = + tokio_tungstenite::connect_async(src_migrate_url).await?; + + // Generate a list of protocols that this target supports, then send them to + // the source and allow it to choose its favorite. + let dst_protocols = super::protocol::make_protocol_offer(); + conn.send(tungstenite::Message::Text(dst_protocols)).await?; + let src_selected = match conn.next().await { + Some(Ok(tungstenite::Message::Text(selected))) => selected, + x => { + error!( + log, + "source instance failed to negotiate protocol version: {:?}", x + ); - if let Err(err) = proto.run().await { - err_tx - .send(MigrateTargetCommand::UpdateState(MigrationState::Error)) - .await - .unwrap(); - - // We encountered an error, try to inform the remote before bailing - // Note, we don't use `?` here as this is a best effort and we don't - // want an error encountered during this send to shadow the run error - // from the caller. - if let Ok(e) = codec::Message::Error(err.clone()).try_into() { - let _ = proto.conn.send(e).await; + // Tell the source about its mistake. This is best-effort. + if let Err(e) = conn + .send(tungstenite::Message::Close(Some(CloseFrame { + code: CloseCode::Protocol, + reason: "did not respond to version handshake.".into(), + }))) + .await + { + warn!(log, "failed to send handshake failure to source"; + "error" => ?e); + } + + return Err(MigrateError::Initiate); } - return Err(err); - } + }; - Ok(()) + // Make sure the source's selected protocol parses correctly and is in the + // list of protocols this target supports. If the source's choice is valid, + // use the protocol it picked. + let selected = + match super::protocol::select_protocol_from_offer(&src_selected) { + Ok(Some(selected)) => selected, + Ok(None) => { + let offered = super::protocol::make_protocol_offer(); + error!(log, "source selected protocol not on offer"; + "offered" => &offered, + "selected" => &src_selected); + + return Err(MigrateError::NoMatchingProtocol( + src_selected, + offered, + )); + } + Err(e) => { + error!(log, "source selected protocol failed to parse"; + "selected" => &src_selected); + + return Err(MigrateError::ProtocolParse( + src_selected, + e.to_string(), + )); + } + }; + + Ok(match selected { + Protocol::RonV0 => RonV0::new(log, migration_id, conn, local_addr), + }) } -struct DestinationProtocol { - /// The VM controller for the instance of interest. - vm_controller: Arc, +/// The runner for version 0 of the LM protocol, using RON encoding. +struct RonV0 { + /// The ID for this migration. + migration_id: Uuid, + + /// The logger for messages from this protocol. + log: slog::Logger, /// The channel to use to send messages to the state worker coordinating /// this migration. - command_tx: tokio::sync::mpsc::Sender, - - /// Transport to the source Instance. conn: WebSocketStream, /// Local propolis-server address @@ -82,84 +160,178 @@ struct DestinationProtocol { local_addr: SocketAddr, } -impl DestinationProtocol { +#[async_trait::async_trait] +impl DestinationProtocol for RonV0 { + async fn run<'ensure>( + mut self, + mut ensure: VmEnsureNotStarted<'ensure>, + ) -> Result, MigrateError> { + info!(self.log(), "entering destination migration task"); + + let result = async { + // Run the sync phase to ensure that the source's instance spec is + // compatible with the spec supplied in the ensure parameters. + if let Err(e) = self.run_sync_phases(&mut ensure).await { + self.update_state( + ensure.state_publisher(), + MigrationState::Error, + ); + let e = ensure.fail(e.into()).await; + return Err(e + .downcast::() + .expect("original error was a MigrateError")); + } + + // The sync phase succeeded, so it's OK to go ahead with creating + // the objects in the target's instance spec. + let mut objects_created = + ensure.create_objects().await.map_err(|e| { + MigrateError::TargetInstanceInitializationFailed( + e.to_string(), + ) + })?; + objects_created.prepare_for_migration().await; + let mut ensure = objects_created.ensure_active().await; + + // Now that the VM's objects exist, run the rest of the protocol to + // import state into them. + if let Err(e) = self.run_import_phases(&mut ensure).await { + self.update_state( + ensure.state_publisher(), + MigrationState::Error, + ); + ensure.fail().await; + return Err(e); + } + + Ok(ensure) + } + .await; + + match result { + Ok(vm) => { + info!(self.log(), "migration in succeeded"); + Ok(vm) + } + Err(err) => { + error!(self.log(), "migration in failed"; "error" => ?err); + + // We encountered an error, try to inform the remote before + // bailing Note, we don't use `?` here as this is a best effort + // and we don't want an error encountered during this send to + // shadow the run error from the caller. + if let Ok(e) = codec::Message::Error(err.clone()).try_into() { + let _ = self.conn.send(e).await; + } + Err(err) + } + } + } +} + +impl RonV0 { fn new( - vm_controller: Arc, - command_tx: tokio::sync::mpsc::Sender, + log: slog::Logger, + migration_id: Uuid, conn: WebSocketStream, local_addr: SocketAddr, ) -> Self { - Self { vm_controller, command_tx, conn, local_addr } + Self { log, migration_id, conn, local_addr } } fn log(&self) -> &slog::Logger { - self.vm_controller.log() + &self.log } - async fn update_state(&mut self, state: MigrationState) { - // When migrating into an instance, the VM state worker blocks waiting - // for the disposition of the migration attempt, so the channel should - // never be closed before the attempt completes. - self.command_tx - .send(MigrateTargetCommand::UpdateState(state)) - .await - .unwrap(); + fn update_state( + &self, + publisher: &mut StatePublisher, + state: MigrationState, + ) { + publisher.update(ExternalStateUpdate::Migration( + MigrationStateUpdate { + state, + id: self.migration_id, + role: MigrateRole::Destination, + }, + )); } - async fn run_phase( + async fn run_sync_phases( &mut self, - step: MigratePhase, + ensure_ctx: &mut VmEnsureNotStarted<'_>, ) -> Result<(), MigrateError> { - probes::migrate_phase_begin!(|| { step.to_string() }); - - let res = match step { - MigratePhase::MigrateSync => self.sync().await, - - // no pause step on the dest side - MigratePhase::Pause => unreachable!(), - MigratePhase::RamPushPrePause | MigratePhase::RamPushPostPause => { - self.ram_push(&step).await - } - MigratePhase::DeviceState => self.device_state().await, - MigratePhase::TimeData => self.time_data().await, - MigratePhase::RamPull => self.ram_pull().await, - MigratePhase::ServerState => self.server_state().await, - MigratePhase::Finish => self.finish().await, - }; + let step = MigratePhase::MigrateSync; + probes::migrate_phase_begin!(|| { step.to_string() }); + self.sync(ensure_ctx).await?; probes::migrate_phase_end!(|| { step.to_string() }); - res + Ok(()) } - async fn run(&mut self) -> Result<(), MigrateError> { - info!(self.log(), "Entering Destination Migration Task"); - - self.run_phase(MigratePhase::MigrateSync).await?; - + async fn run_import_phases( + &mut self, + ensure_ctx: &mut VmEnsureActive<'_>, + ) -> Result<(), MigrateError> { // The RAM transfer phase runs twice, once before the source pauses and // once after. There is no explicit pause phase on the destination, // though, so that step does not appear here even though there are // pre- and post-pause steps. - self.run_phase(MigratePhase::RamPushPrePause).await?; - self.run_phase(MigratePhase::RamPushPostPause).await?; + self.run_import_phase(MigratePhase::RamPushPrePause, ensure_ctx) + .await?; + self.run_import_phase(MigratePhase::RamPushPostPause, ensure_ctx) + .await?; // Import of the time data *must* be done before we import device // state: the proper functioning of device timers depends on an adjusted // boot_hrtime. - self.run_phase(MigratePhase::TimeData).await?; - self.run_phase(MigratePhase::DeviceState).await?; - self.run_phase(MigratePhase::RamPull).await?; - self.run_phase(MigratePhase::ServerState).await?; - self.run_phase(MigratePhase::Finish).await?; - - info!(self.log(), "Destination Migration Successful"); + self.run_import_phase(MigratePhase::TimeData, ensure_ctx).await?; + self.run_import_phase(MigratePhase::DeviceState, ensure_ctx).await?; + self.run_import_phase(MigratePhase::RamPull, ensure_ctx).await?; + self.run_import_phase(MigratePhase::ServerState, ensure_ctx).await?; + self.run_import_phase(MigratePhase::Finish, ensure_ctx).await?; Ok(()) } - async fn sync(&mut self) -> Result<(), MigrateError> { - self.update_state(MigrationState::Sync).await; + async fn run_import_phase( + &mut self, + step: MigratePhase, + ensure_ctx: &mut VmEnsureActive<'_>, + ) -> Result<(), MigrateError> { + probes::migrate_phase_begin!(|| { step.to_string() }); + + let res = match step { + MigratePhase::MigrateSync => { + unreachable!("sync phase runs before import") + } + + // no pause step on the dest side + MigratePhase::Pause => { + unreachable!("no explicit pause phase on dest") + } + + MigratePhase::RamPushPrePause | MigratePhase::RamPushPostPause => { + self.ram_push(&step, ensure_ctx).await + } + MigratePhase::DeviceState => self.device_state(ensure_ctx).await, + MigratePhase::TimeData => self.time_data(ensure_ctx).await, + MigratePhase::RamPull => self.ram_pull(ensure_ctx).await, + MigratePhase::ServerState => self.server_state(ensure_ctx).await, + MigratePhase::Finish => self.finish(ensure_ctx).await, + }; + + probes::migrate_phase_end!(|| { step.to_string() }); + + res + } + + async fn sync( + &mut self, + ensure_ctx: &mut VmEnsureNotStarted<'_>, + ) -> Result<(), MigrateError> { + self.update_state(ensure_ctx.state_publisher(), MigrationState::Sync); let preamble: Preamble = match self.read_msg().await? { codec::Message::Serialized(s) => { Ok(ron::de::from_str(&s).map_err(codec::ProtocolError::from)?) @@ -173,8 +345,9 @@ impl DestinationProtocol { } }?; info!(self.log(), "Destination read Preamble: {:?}", preamble); - if let Err(e) = preamble - .is_migration_compatible(self.vm_controller.instance_spec().await) + + if let Err(e) = + preamble.is_migration_compatible(ensure_ctx.instance_spec()) { error!( self.log(), @@ -189,17 +362,15 @@ impl DestinationProtocol { async fn ram_push( &mut self, phase: &MigratePhase, + ensure_ctx: &mut VmEnsureActive<'_>, ) -> Result<(), MigrateError> { - match phase { - MigratePhase::RamPushPrePause => { - self.update_state(MigrationState::RamPush).await - } - MigratePhase::RamPushPostPause => { - self.update_state(MigrationState::RamPushDirty).await - } + let state = match phase { + MigratePhase::RamPushPrePause => MigrationState::RamPush, + MigratePhase::RamPushPostPause => MigrationState::RamPushDirty, _ => unreachable!("should only push RAM in a RAM push phase"), - } + }; + self.update_state(ensure_ctx.state_publisher(), state); let (dirty, highest) = self.query_ram().await?; for (k, region) in dirty.as_raw_slice().chunks(4096).enumerate() { if region.iter().all(|&b| b == 0) { @@ -236,13 +407,13 @@ impl DestinationProtocol { // space or non-existent RAM regions. While we de facto // do not because of the way access is implemented, we // should probably disallow it at the protocol level. - self.xfer_ram(start, end, &bits).await?; + self.xfer_ram(ensure_ctx, start, end, &bits).await?; } _ => return Err(MigrateError::UnexpectedMessage), }; } self.send_msg(codec::Message::MemDone).await?; - self.update_state(MigrationState::Pause).await; + self.update_state(ensure_ctx.state_publisher(), MigrationState::Pause); Ok(()) } @@ -289,6 +460,7 @@ impl DestinationProtocol { async fn xfer_ram( &mut self, + ensure_ctx: &VmEnsureActive<'_>, start: u64, end: u64, bits: &[u8], @@ -296,13 +468,16 @@ impl DestinationProtocol { info!(self.log(), "ram_push: xfer RAM between {} and {}", start, end); for addr in PageIter::new(start, end, bits) { let bytes = self.read_page().await?; - self.write_guest_ram(GuestAddr(addr), &bytes).await?; + self.write_guest_ram(ensure_ctx, GuestAddr(addr), &bytes).await?; } Ok(()) } - async fn device_state(&mut self) -> Result<(), MigrateError> { - self.update_state(MigrationState::Device).await; + async fn device_state( + &mut self, + ensure_ctx: &mut VmEnsureActive<'_>, + ) -> Result<(), MigrateError> { + self.update_state(ensure_ctx.state_publisher(), MigrationState::Device); let devices: Vec = match self.read_msg().await? { codec::Message::Serialized(encoded) => { @@ -319,32 +494,33 @@ impl DestinationProtocol { info!(self.log(), "Devices: {devices:#?}"); { - let machine = self.vm_controller.machine(); + let vm_objects = ensure_ctx.vm_objects().lock_shared().await; let migrate_ctx = - MigrateCtx { mem: &machine.acc_mem.access().unwrap() }; + MigrateCtx { mem: &vm_objects.access_mem().unwrap() }; for device in devices { info!( self.log(), "Applying state to device {}", device.instance_name ); - let target = self - .vm_controller + let target = vm_objects .device_by_name(&device.instance_name) .ok_or_else(|| { - MigrateError::UnknownDevice( - device.instance_name.clone(), - ) - })?; + MigrateError::UnknownDevice(device.instance_name.clone()) + })?; self.import_device(&target, &device, &migrate_ctx)?; } } + self.send_msg(codec::Message::Okay).await } // Get the guest time data from the source, make updates to it based on the // new host, and write the data out to bhvye. - async fn time_data(&mut self) -> Result<(), MigrateError> { + async fn time_data( + &mut self, + ensure_ctx: &VmEnsureActive<'_>, + ) -> Result<(), MigrateError> { // Read time data sent by the source and deserialize let raw: String = match self.read_msg().await? { codec::Message::Serialized(encoded) => encoded, @@ -371,7 +547,9 @@ impl DestinationProtocol { // Take a snapshot of the host hrtime/wall clock time, then adjust // time data appropriately. - let vmm_hdl = &self.vm_controller.machine().hdl.clone(); + let vmm_hdl = + &ensure_ctx.vm_objects().lock_shared().await.vmm_hdl().clone(); + let (dst_hrt, dst_wc) = vmm::time::host_time_snapshot(vmm_hdl) .map_err(|e| { MigrateError::TimeData(format!( @@ -541,16 +719,25 @@ impl DestinationProtocol { Ok(()) } - async fn ram_pull(&mut self) -> Result<(), MigrateError> { - self.update_state(MigrationState::RamPull).await; + async fn ram_pull( + &mut self, + ensure_ctx: &mut VmEnsureActive<'_>, + ) -> Result<(), MigrateError> { + self.update_state( + ensure_ctx.state_publisher(), + MigrationState::RamPull, + ); self.send_msg(codec::Message::MemQuery(0, !0)).await?; let m = self.read_msg().await?; info!(self.log(), "ram_pull: got end {:?}", m); self.send_msg(codec::Message::MemDone).await } - async fn server_state(&mut self) -> Result<(), MigrateError> { - self.update_state(MigrationState::Server).await; + async fn server_state( + &mut self, + ensure_ctx: &mut VmEnsureActive<'_>, + ) -> Result<(), MigrateError> { + self.update_state(ensure_ctx.state_publisher(), MigrationState::Server); self.send_msg(codec::Message::Serialized( ron::to_string(&self.local_addr) .map_err(codec::ProtocolError::from)?, @@ -564,15 +751,22 @@ impl DestinationProtocol { } }; - self.vm_controller + ensure_ctx + .vm_objects() + .lock_shared() + .await .com1() .import(&com1_history) .await .map_err(|e| MigrateError::Codec(e.to_string()))?; + self.send_msg(codec::Message::Okay).await } - async fn finish(&mut self) -> Result<(), MigrateError> { + async fn finish( + &mut self, + ensure_ctx: &mut VmEnsureActive<'_>, + ) -> Result<(), MigrateError> { // Tell the source this destination is ready to run the VM. self.send_msg(codec::Message::Okay).await?; @@ -582,9 +776,9 @@ impl DestinationProtocol { // that it should resume the VM. self.read_ok().await?; - // Now that control is definitely being transferred, publish that the - // migration has succeeded. - self.update_state(MigrationState::Finish).await; + // The source has acknowledged the migration is complete, so it's safe + // to declare victory publicly. + self.update_state(ensure_ctx.state_publisher(), MigrationState::Finish); Ok(()) } @@ -600,7 +794,10 @@ impl DestinationProtocol { // If this is an error message, lift that out .map(|msg| match msg.try_into()? { codec::Message::Error(err) => { - error!(self.log(), "remote error: {err}"); + error!( + self.log(), + "migration failed due to error from source: {err}" + ); Err(MigrateError::RemoteError( MigrateRole::Source, err.to_string(), @@ -636,11 +833,12 @@ impl DestinationProtocol { async fn write_guest_ram( &mut self, + ensure_ctx: &VmEnsureActive<'_>, addr: GuestAddr, buf: &[u8], ) -> Result<(), MigrateError> { - let machine = self.vm_controller.machine(); - let memctx = machine.acc_mem.access().unwrap(); + let objects = ensure_ctx.vm_objects().lock_shared().await; + let memctx = objects.access_mem().unwrap(); let len = buf.len(); memctx.write_from(addr, buf, len); Ok(()) diff --git a/bin/propolis-server/src/lib/migrate/mod.rs b/bin/propolis-server/src/lib/migrate/mod.rs index 5c246d0c0..a59e8c85c 100644 --- a/bin/propolis-server/src/lib/migrate/mod.rs +++ b/bin/propolis-server/src/lib/migrate/mod.rs @@ -2,26 +2,14 @@ // License, v. 2.0. If a copy of the MPL was not distributed with this // file, You can obtain one at https://mozilla.org/MPL/2.0/. -use std::sync::Arc; - use bit_field::BitField; -use dropshot::{HttpError, RequestContext}; -use futures::{SinkExt, StreamExt}; +use dropshot::HttpError; use propolis::migrate::MigrateStateError; -use propolis_api_types::{self as api, MigrationState}; +use propolis_api_types::MigrationState; use serde::{Deserialize, Serialize}; -use slog::{error, info, o}; +use slog::error; use thiserror::Error; use tokio::io::{AsyncRead, AsyncWrite}; -use tokio_tungstenite::tungstenite::protocol::frame::coding::CloseCode; -use tokio_tungstenite::tungstenite::protocol::CloseFrame; -use tokio_tungstenite::{tungstenite, WebSocketStream}; -use uuid::Uuid; - -use crate::{ - server::{DropshotEndpointContext, VmControllerState}, - vm::{VmController, VmControllerError}, -}; mod codec; pub mod destination; @@ -30,6 +18,15 @@ mod preamble; pub mod protocol; pub mod source; +/// Trait bounds for connection objects used in live migrations. +pub(crate) trait MigrateConn: + AsyncRead + AsyncWrite + Unpin + Send +{ +} + +impl MigrateConn for tokio_tungstenite::MaybeTlsStream {} +impl MigrateConn for hyper::upgrade::Upgraded {} + #[derive(Copy, Clone, Debug, Eq, PartialEq, Serialize, Deserialize)] pub enum MigrateRole { Source, @@ -37,7 +34,7 @@ pub enum MigrateRole { } // N.B. Keep in sync with scripts/live-migration-times.d. -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq)] enum MigratePhase { MigrateSync, Pause, @@ -92,8 +89,8 @@ pub enum MigrateError { UpgradeExpected, /// Attempted to migrate an uninitialized instance - #[error("instance is not initialized")] - InstanceNotInitialized, + #[error("failed to initialize the target VM: {0}")] + TargetInstanceInitializationFailed(String), /// The given UUID does not match the existing instance/migration UUID #[error("unexpected Uuid")] @@ -160,16 +157,6 @@ impl From for MigrateError { } } -impl From for MigrateError { - fn from(err: VmControllerError) -> Self { - match err { - VmControllerError::AlreadyMigrationSource => { - MigrateError::MigrationAlreadyInProgress - } - _ => MigrateError::StateMachine(err.to_string()), - } - } -} impl From for MigrateError { fn from(value: MigrateStateError) -> Self { Self::DeviceState(value.to_string()) @@ -184,7 +171,7 @@ impl From for HttpError { | MigrateError::Initiate | MigrateError::ProtocolParse(_, _) | MigrateError::NoMatchingProtocol(_, _) - | MigrateError::InstanceNotInitialized + | MigrateError::TargetInstanceInitializationFailed(_) | MigrateError::InvalidInstanceState | MigrateError::Codec(_) | MigrateError::UnexpectedMessage @@ -228,178 +215,6 @@ struct DevicePayload { pub data: String, } -/// Begin the migration process (source-side). -/// -/// This will check protocol version and then begin the migration in a separate task. -pub async fn source_start< - T: AsyncRead + AsyncWrite + Unpin + Send + 'static, ->( - rqctx: RequestContext>, - migration_id: Uuid, - mut conn: WebSocketStream, -) -> Result<(), MigrateError> { - // Create a new log context for the migration - let log = rqctx.log.new(o!( - "migration_id" => migration_id.to_string(), - "migrate_role" => "source" - )); - info!(log, "Migration Source"); - - let controller = tokio::sync::MutexGuard::try_map( - rqctx.context().services.vm.lock().await, - VmControllerState::as_controller, - ) - .map_err(|_| MigrateError::InstanceNotInitialized)?; - - let selected = match conn.next().await { - Some(Ok(tungstenite::Message::Text(dst_protocols))) => { - info!(log, "destination offered protocols: {}", dst_protocols); - match protocol::select_protocol_from_offer(&dst_protocols) { - Ok(Some(selected)) => { - info!(log, "selected protocol {:?}", selected); - conn.send(tungstenite::Message::Text( - selected.offer_string(), - )) - .await?; - selected - } - Ok(None) => { - let src_protocols = protocol::make_protocol_offer(); - error!( - log, - "no compatible destination protocols"; - "dst_protocols" => &dst_protocols, - "src_protocols" => &src_protocols, - ); - return Err(MigrateError::NoMatchingProtocol( - src_protocols, - dst_protocols, - )); - } - Err(e) => { - error!(log, "failed to parse destination protocol offer"; - "dst_protocols" => &dst_protocols, - "error" => %e); - return Err(MigrateError::ProtocolParse( - dst_protocols, - e.to_string(), - )); - } - } - } - x => { - conn.send(tungstenite::Message::Close(Some(CloseFrame { - code: CloseCode::Protocol, - reason: "did not begin with version handshake.".into(), - }))) - .await?; - error!( - log, - "destination side did not begin migration version handshake: \ - {:?}", - x - ); - return Err(MigrateError::Initiate); - } - }; - - controller.request_migration_from(migration_id, conn, selected)?; - Ok(()) -} - -/// Initiate a migration to the given source instance. -/// -/// This will attempt to open a websocket to the given source instance and -/// check that the migrate protocol version is compatible ("equal" presently). -/// Once we've successfully established the connection, we can begin the -/// migration process (destination-side). -pub(crate) async fn dest_initiate( - rqctx: &RequestContext>, - controller: Arc, - migrate_info: api::InstanceMigrateInitiateRequest, -) -> Result { - let migration_id = migrate_info.migration_id; - - // Create a new log context for the migration - let log = rqctx.log.new(o!( - "migration_id" => migration_id.to_string(), - "migrate_role" => "destination", - "migrate_src_addr" => migrate_info.src_addr - )); - info!(log, "Migration Destination"); - - // Build upgrade request to the source instance - // (we do this by hand because it's hidden from the OpenAPI spec) - // TODO(#165): https (wss) - // TODO: We need to make sure the src_addr is a valid target - let src_migrate_url = format!( - "ws://{}/instance/migrate/{}/start", - migrate_info.src_addr, migration_id, - ); - info!(log, "Begin migration"; "src_migrate_url" => &src_migrate_url); - let (mut conn, _) = - tokio_tungstenite::connect_async(src_migrate_url).await?; - - let dst_protocols = protocol::make_protocol_offer(); - conn.send(tungstenite::Message::Text(dst_protocols)).await?; - let selected = match conn.next().await { - Some(Ok(tungstenite::Message::Text(selected_protocol))) => { - info!(log, "source negotiated protocol {}", selected_protocol); - match protocol::select_protocol_from_offer(&selected_protocol) { - Ok(Some(selected)) => selected, - Ok(None) => { - let offered = protocol::make_protocol_offer(); - error!(log, "source selected protocol not on offer"; - "offered" => &offered, - "selected" => &selected_protocol); - - return Err(MigrateError::NoMatchingProtocol( - selected_protocol, - offered, - )); - } - Err(e) => { - error!(log, "source selected protocol failed to parse"; - "selected" => &selected_protocol); - - return Err(MigrateError::ProtocolParse( - selected_protocol, - e.to_string(), - )); - } - } - } - x => { - conn.send(tungstenite::Message::Close(Some(CloseFrame { - code: CloseCode::Protocol, - reason: "did not respond to version handshake.".into(), - }))) - .await?; - error!( - log, - "source instance failed to negotiate protocol version: {:?}", x - ); - return Err(MigrateError::Initiate); - } - }; - let local_addr = rqctx.server.local_addr; - tokio::runtime::Handle::current() - .spawn_blocking(move || -> Result<(), MigrateError> { - // Now start using the websocket for the migration protocol - controller.request_migration_into( - migration_id, - conn, - local_addr, - selected, - )?; - Ok(()) - }) - .await - .unwrap()?; - - Ok(api::InstanceMigrateInitiateResponse { migration_id }) -} - // We should probably turn this into some kind of ValidatedBitmap // data structure, so that we're only parsing it once. struct PageIter<'a> { diff --git a/bin/propolis-server/src/lib/migrate/preamble.rs b/bin/propolis-server/src/lib/migrate/preamble.rs index 8618b0449..b45a0d9ac 100644 --- a/bin/propolis-server/src/lib/migrate/preamble.rs +++ b/bin/propolis-server/src/lib/migrate/preamble.rs @@ -10,7 +10,6 @@ use propolis_api_types::instance_spec::{ VersionedInstanceSpec, }; use serde::{Deserialize, Serialize}; -use tokio::sync::MutexGuard; #[derive(Deserialize, Serialize, Debug)] pub(crate) struct Preamble { @@ -40,10 +39,8 @@ impl Preamble { pub fn is_migration_compatible( &self, - other_spec: MutexGuard<'_, VersionedInstanceSpec>, + other_spec: &InstanceSpecV0, ) -> Result<(), MigrationCompatibilityError> { - let VersionedInstanceSpec::V0(other_spec) = &*other_spec; - self.device_spec.can_migrate_devices_from(&other_spec.devices)?; let other_keys = get_spec_backend_keys(other_spec); if self.backend_keys.len() != other_keys.len() { diff --git a/bin/propolis-server/src/lib/migrate/source.rs b/bin/propolis-server/src/lib/migrate/source.rs index 2ffd12a62..a8c3c5d9a 100644 --- a/bin/propolis-server/src/lib/migrate/source.rs +++ b/bin/propolis-server/src/lib/migrate/source.rs @@ -9,26 +9,34 @@ use propolis::migrate::{ MigrateCtx, MigrateStateError, Migrator, PayloadOutputs, }; use propolis::vmm; -use slog::{debug, error, info, trace}; +use propolis_api_types::instance_spec::VersionedInstanceSpec; +use slog::{debug, error, info, trace, warn}; use std::collections::HashMap; use std::convert::TryInto; use std::io; use std::ops::{Range, RangeInclusive}; -use std::sync::Arc; -use tokio::io::{AsyncRead, AsyncWrite}; -use tokio_tungstenite::WebSocketStream; +use tokio_tungstenite::tungstenite::protocol::frame::coding::CloseCode; +use tokio_tungstenite::tungstenite::protocol::CloseFrame; +use tokio_tungstenite::{tungstenite, WebSocketStream}; +use uuid::Uuid; -use crate::migrate::codec; use crate::migrate::codec::Message; use crate::migrate::memx; use crate::migrate::preamble::Preamble; use crate::migrate::probes; use crate::migrate::protocol::Protocol; +use crate::migrate::{codec, protocol}; use crate::migrate::{ Device, DevicePayload, MigrateError, MigratePhase, MigrateRole, MigrationState, PageIter, }; -use crate::vm::{MigrateSourceCommand, MigrateSourceResponse, VmController}; + +use crate::vm::objects::VmObjects; +use crate::vm::state_publisher::{ + ExternalStateUpdate, MigrationStateUpdate, StatePublisher, +}; + +use super::MigrateConn; /// Specifies which pages should be offered during a RAM transfer phase. /// @@ -109,74 +117,116 @@ enum RamOfferDiscipline { OfferDirty, } -pub async fn migrate( - vm_controller: Arc, - command_tx: tokio::sync::mpsc::Sender, - response_rx: tokio::sync::mpsc::Receiver, - conn: WebSocketStream, - protocol: super::protocol::Protocol, -) -> Result<(), MigrateError> { - let err_tx = command_tx.clone(); - let mut proto = match protocol { - Protocol::RonV0 => { - SourceProtocol::new(vm_controller, command_tx, response_rx, conn) - } - }; +/// The interface to an arbitrary version of the source half of the live +/// migration protocol. +// +// Use `async_trait` here to help generate a `Send` bound on the futures +// returned by the functions in this trait. +#[async_trait::async_trait] +pub(crate) trait SourceProtocol { + /// Runs live migration out of the supplied `vm_objects`, writing back any + /// state that must be saved for future migration attempts to + /// `persistent_state`. + /// + /// This routine guarantees that the supplied `vm_objects` are paused on + /// success and resumed on failure. + async fn run( + self, + vm_objects: &VmObjects, + publisher: &mut StatePublisher, + persistent_state: &mut PersistentState, + ) -> Result<(), MigrateError>; +} - if let Err(err) = proto.run().await { - err_tx - .send(MigrateSourceCommand::UpdateState(MigrationState::Error)) - .await - .unwrap(); - - // We encountered an error, try to inform the remote before bailing - // Note, we don't use `?` here as this is a best effort and we don't - // want an error encountered during this send to shadow the run error - // from the caller. - let _ = proto.send_msg(codec::Message::Error(err.clone())).await; - - // If we are capable of setting the dirty bit on guest page table - // entries, re-dirty them, so that a later migration attempt can also - // only offer dirty pages. If we can't use VM_NPT_OPERATION, a - // subsequent migration attempt will offer all pages. - // - // See the lengthy comment on `RamOfferDiscipline` above for more - // details about what's going on here. - for (&GuestAddr(gpa), dirtiness) in proto.dirt.iter().flatten() { - if let Err(e) = proto - .vm_controller - .machine() - .hdl - .set_dirty_pages(gpa, dirtiness) +/// Negotiates a live migration protocol version with a target who has connected +/// over `conn`. If this is successful, returns a `SourceProtocol` +/// implementation that can be used to run the requested migration. +pub(crate) async fn initiate( + log: &slog::Logger, + migration_id: Uuid, + mut conn: WebSocketStream, + vm_objects: &VmObjects, + persistent_state: &PersistentState, +) -> Result { + // Create a new log context for the migration + let log = log.new(slog::o!( + "migration_id" => migration_id.to_string(), + "migrate_role" => "source" + )); + info!(log, "negotiating migration as source"); + + // The protocol should start with some text from the destination identifying + // the protocol versions it supports. + let dst_protocols = match conn.next().await { + Some(Ok(tungstenite::Message::Text(dst_protocols))) => dst_protocols, + x => { + error!( + log, + "destination side did not begin migration version handshake: \ + {:?}", + x + ); + + // Tell the destination it misbehaved. This is best-effort. + if let Err(e) = conn + .send(tungstenite::Message::Close(Some(CloseFrame { + code: CloseCode::Protocol, + reason: "did not begin with version handshake.".into(), + }))) + .await { - // Bad news! Our attempt to re-set the dirty bit on these - // pages has failed! Thus, subsequent migration attempts - // /!\ CAN NO LONGER RELY ON DIRTY PAGE TRACKING /!\ - // and must always offer all pages in the initial RAM push - // phase. - // - // Record that now so we never try to do this again. - proto - .vm_controller - .migration_src_state() - .has_redirtying_ever_failed = true; - error!( - proto.log(), - "failed to restore dirty bits: {e}"; - "gpa" => gpa, - ); - // No sense continuing to try putting back any remaining - // dirty bits, as we won't be using them any longer. - break; - } else { - debug!(proto.log(), "re-dirtied pages at {gpa:#x}",); + warn!(log, "failed to send handshake failed message to source"; + "error" => ?e); } + + return Err(MigrateError::Initiate); } + }; - return Err(err); - } + // Pick the most favorable protocol from the list the destination supplied + // and send it back to the destination. + info!(log, "destination offered protocols: {}", dst_protocols); + let selected = match protocol::select_protocol_from_offer(&dst_protocols) { + Ok(Some(selected)) => { + conn.send(tungstenite::Message::Text(selected.offer_string())) + .await?; + selected + } + Ok(None) => { + let src_protocols = protocol::make_protocol_offer(); + error!( + log, + "no compatible destination protocols"; + "dst_protocols" => &dst_protocols, + "src_protocols" => &src_protocols, + ); + return Err(MigrateError::NoMatchingProtocol( + src_protocols, + dst_protocols, + )); + } + Err(e) => { + error!(log, "failed to parse destination protocol offer"; + "dst_protocols" => &dst_protocols, + "error" => %e); + return Err(MigrateError::ProtocolParse( + dst_protocols, + e.to_string(), + )); + } + }; - Ok(()) + info!(log, "selected protocol {:?}", selected); + match selected { + Protocol::RonV0 => Ok(RonV0::new( + log, + vm_objects, + migration_id, + conn, + persistent_state, + ) + .await), + } } /// State which must be stored across multiple migration attempts. @@ -187,23 +237,19 @@ pub async fn migrate( #[derive(Default)] pub(crate) struct PersistentState { /// Set if we were unable to re-set dirty bits on guest pages after a failed - /// migration attempt. If this occurs, we can no longer offer only dirty pages - /// in a subsequent migration attempt, as some pages which should be marked as - /// dirty may not be. - has_redirtying_ever_failed: bool, + /// migration attempt. If this occurs, we can no longer offer only dirty + /// pages in a subsequent migration attempt, as some pages which should be + /// marked as dirty may not be. + pub(crate) has_redirtying_ever_failed: bool, } -struct SourceProtocol { - /// The VM controller for the instance of interest. - vm_controller: Arc, - - /// The channel to use to send messages to the state worker coordinating - /// this migration. - command_tx: tokio::sync::mpsc::Sender, +/// Context for the source side of protocol version 0 using the RON encoding. +struct RonV0 { + /// The logger to which to log messages from this migration attempt. + log: slog::Logger, - /// The channel to use to receive messages from the state worker - /// coordinating this migration. - response_rx: tokio::sync::mpsc::Receiver, + /// The migration's ID. + migration_id: Uuid, /// Transport to the destination Instance. conn: WebSocketStream, @@ -233,44 +279,99 @@ struct SourceProtocol { const PAGE_BITMAP_SIZE: usize = 4096; type PageBitmap = [u8; PAGE_BITMAP_SIZE]; -impl SourceProtocol { - fn new( - vm_controller: Arc, - command_tx: tokio::sync::mpsc::Sender, - response_rx: tokio::sync::mpsc::Receiver, +impl RonV0 { + async fn new( + log: slog::Logger, + vm: &VmObjects, + migration_id: Uuid, conn: WebSocketStream, + persistent_state: &PersistentState, ) -> Self { + // Create a (prospective) dirty page map if bhyve supports the NPT + // API. If this map is present and the VM hasn't recorded that it's + // possibly unhealthy, it will be used to offer only dirty pages during + // the pre-pause RAM push. let dirt = { - let can_npt_operate = vm_controller.machine().hdl.can_npt_operate(); + let can_npt_operate = + vm.lock_shared().await.vmm_hdl().can_npt_operate(); + let has_redirtying_ever_failed = - vm_controller.migration_src_state().has_redirtying_ever_failed; + persistent_state.has_redirtying_ever_failed; if can_npt_operate && !has_redirtying_ever_failed { Some(Default::default()) } else { info!( - vm_controller.log(), - "guest pages are not redirtyable; will offer all pages in pre-pause RAM push"; + log, + "guest pages not redirtyable, will offer all pages in pre-pause"; "can_npt_operate" => can_npt_operate, "has_redirtying_ever_failed" => has_redirtying_ever_failed ); None } }; - Self { vm_controller, command_tx, response_rx, conn, dirt } + Self { log, migration_id, conn, dirt } } +} + +#[async_trait::async_trait] +impl SourceProtocol for RonV0 { + async fn run( + self, + vm_objects: &VmObjects, + publisher: &mut StatePublisher, + persistent_state: &mut PersistentState, + ) -> Result<(), MigrateError> { + let mut runner = RonV0Runner { + log: self.log, + migration_id: self.migration_id, + conn: self.conn, + dirt: self.dirt, + vm: vm_objects, + state_publisher: publisher, + persistent_state, + paused: false, + }; + runner.run().await + } +} + +struct RonV0Runner<'vm, T: MigrateConn> { + log: slog::Logger, + migration_id: Uuid, + conn: WebSocketStream, + dirt: Option>, + vm: &'vm VmObjects, + state_publisher: &'vm mut StatePublisher, + persistent_state: &'vm mut PersistentState, + paused: bool, +} + +impl<'vm, T: MigrateConn> RonV0Runner<'vm, T> { fn log(&self) -> &slog::Logger { - self.vm_controller.log() + &self.log } - async fn update_state(&mut self, state: MigrationState) { - // When migrating into an instance, the VM state worker blocks waiting - // for the disposition of the migration attempt, so the channel should - // never be closed before the attempt completes. - self.command_tx - .send(MigrateSourceCommand::UpdateState(state)) - .await - .unwrap(); + fn update_state(&mut self, state: MigrationState) { + self.state_publisher.update(ExternalStateUpdate::Migration( + MigrationStateUpdate { + state, + id: self.migration_id, + role: MigrateRole::Source, + }, + )); + } + + async fn pause_vm(&mut self) { + assert!(!self.paused); + self.paused = true; + self.vm.lock_exclusive().await.pause().await; + } + + async fn resume_vm(&mut self) { + assert!(self.paused); + self.paused = false; + self.vm.lock_exclusive().await.resume(); } async fn run_phase( @@ -300,24 +401,75 @@ impl SourceProtocol { async fn run(&mut self) -> Result<(), MigrateError> { info!(self.log(), "Entering Source Migration Task"); - self.run_phase(MigratePhase::MigrateSync).await?; - self.run_phase(MigratePhase::RamPushPrePause).await?; - self.run_phase(MigratePhase::Pause).await?; - self.run_phase(MigratePhase::RamPushPostPause).await?; - self.run_phase(MigratePhase::TimeData).await?; - self.run_phase(MigratePhase::DeviceState).await?; - self.run_phase(MigratePhase::RamPull).await?; - self.run_phase(MigratePhase::ServerState).await?; - self.run_phase(MigratePhase::Finish).await?; - - info!(self.log(), "Source Migration Successful"); - Ok(()) + let result: Result<_, MigrateError> = async { + self.run_phase(MigratePhase::MigrateSync).await?; + self.run_phase(MigratePhase::RamPushPrePause).await?; + self.run_phase(MigratePhase::Pause).await?; + self.run_phase(MigratePhase::RamPushPostPause).await?; + self.run_phase(MigratePhase::TimeData).await?; + self.run_phase(MigratePhase::DeviceState).await?; + self.run_phase(MigratePhase::RamPull).await?; + self.run_phase(MigratePhase::ServerState).await?; + self.run_phase(MigratePhase::Finish).await?; + Ok(()) + } + .await; + + if let Err(err) = result { + self.update_state(MigrationState::Error); + let _ = self.send_msg(codec::Message::Error(err.clone())).await; + + // If we are capable of setting the dirty bit on guest page table + // entries, re-dirty them, so that a later migration attempt can also + // only offer dirty pages. If we can't use VM_NPT_OPERATION, a + // subsequent migration attempt will offer all pages. + // + // See the lengthy comment on `RamOfferDiscipline` above for more + // details about what's going on here. + let vmm_hdl = self.vm.lock_shared().await.vmm_hdl().clone(); + for (&GuestAddr(gpa), dirtiness) in self.dirt.iter().flatten() { + if let Err(e) = vmm_hdl.set_dirty_pages(gpa, dirtiness) { + // Bad news! Our attempt to re-set the dirty bit on these + // pages has failed! Thus, subsequent migration attempts + // /!\ CAN NO LONGER RELY ON DIRTY PAGE TRACKING /!\ + // and must always offer all pages in the initial RAM push + // phase. + // + // Record that now so we never try to do this again. + self.persistent_state.has_redirtying_ever_failed = true; + error!( + self.log(), + "failed to restore dirty bits: {e}"; + "gpa" => gpa, + ); + // No sense continuing to try putting back any remaining + // dirty bits, as we won't be using them any longer. + break; + } else { + debug!(self.log(), "re-dirtied pages at {gpa:#x}",); + } + } + + if self.paused { + self.resume_vm().await; + } + + Err(err) + } else { + // The VM should be paused after successfully migrating out; the + // state driver assumes as much when subsequently halting the + // instance. + assert!(self.paused); + info!(self.log(), "Source Migration Successful"); + Ok(()) + } } async fn sync(&mut self) -> Result<(), MigrateError> { - self.update_state(MigrationState::Sync).await; - let preamble = - Preamble::new(self.vm_controller.instance_spec().await.clone()); + self.update_state(MigrationState::Sync); + let preamble = Preamble::new(VersionedInstanceSpec::V0( + self.vm.lock_shared().await.instance_spec().clone(), + )); let s = ron::ser::to_string(&preamble) .map_err(codec::ProtocolError::from)?; self.send_msg(codec::Message::Serialized(s)).await?; @@ -331,10 +483,10 @@ impl SourceProtocol { ) -> Result<(), MigrateError> { match phase { MigratePhase::RamPushPrePause => { - self.update_state(MigrationState::RamPush).await + self.update_state(MigrationState::RamPush) } MigratePhase::RamPushPostPause => { - self.update_state(MigrationState::RamPushDirty).await + self.update_state(MigrationState::RamPushDirty) } _ => unreachable!("should only push RAM in a RAM push phase"), } @@ -403,7 +555,7 @@ impl SourceProtocol { }; } info!(self.log(), "ram_push: done sending ram"); - self.update_state(MigrationState::Pause).await; + self.update_state(MigrationState::Pause); Ok(()) } @@ -446,7 +598,7 @@ impl SourceProtocol { // says to offer all pages. This ensures that pages that are // transferred now and not touched again will not be offered again // by a subsequent phase. - self.track_dirty(GuestAddr(gpa), &mut bits)?; + self.track_dirty(GuestAddr(gpa), &mut bits).await?; match offer_discipline { RamOfferDiscipline::OfferAll => { @@ -514,34 +666,24 @@ impl SourceProtocol { } async fn pause(&mut self) -> Result<(), MigrateError> { - self.update_state(MigrationState::Pause).await; + self.update_state(MigrationState::Pause); // Ask the instance to begin transitioning to the paused state // This will inform each device to pause. info!(self.log(), "Pausing devices"); - self.command_tx.send(MigrateSourceCommand::Pause).await.unwrap(); - let resp = self.response_rx.recv().await.unwrap(); - match resp { - MigrateSourceResponse::Pause(Ok(())) => Ok(()), - _ => { - info!( - self.log(), - "Unexpected pause response from state worker: {:?}", resp - ); - Err(MigrateError::SourcePause) - } - } + self.pause_vm().await; + Ok(()) } async fn device_state(&mut self) -> Result<(), MigrateError> { - self.update_state(MigrationState::Device).await; + self.update_state(MigrationState::Device); let mut device_states = vec![]; { - let machine = self.vm_controller.machine(); + let objects = self.vm.lock_shared().await; let migrate_ctx = - MigrateCtx { mem: &machine.acc_mem.access().unwrap() }; + MigrateCtx { mem: &objects.access_mem().unwrap() }; // Collect together the serialized state for all the devices - self.vm_controller.for_each_device_fallible(|name, devop| { + objects.for_each_device_fallible(|name, devop| { let mut dev = Device { instance_name: name.to_string(), payload: Vec::new(), @@ -599,7 +741,7 @@ impl SourceProtocol { // Read and send over the time data async fn time_data(&mut self) -> Result<(), MigrateError> { - let vmm_hdl = &self.vm_controller.machine().hdl.clone(); + let vmm_hdl = &self.vm.lock_shared().await.vmm_hdl().clone(); let vm_time_data = vmm::time::export_time_data(vmm_hdl).map_err(|e| { MigrateError::TimeData(format!( @@ -618,11 +760,11 @@ impl SourceProtocol { } async fn ram_pull(&mut self) -> Result<(), MigrateError> { - self.update_state(MigrationState::RamPush).await; + self.update_state(MigrationState::RamPush); let m = self.read_msg().await?; info!(self.log(), "ram_pull: got query {:?}", m); - self.update_state(MigrationState::Pause).await; - self.update_state(MigrationState::RamPushDirty).await; + self.update_state(MigrationState::Pause); + self.update_state(MigrationState::RamPushDirty); self.send_msg(codec::Message::MemEnd(0, !0)).await?; let m = self.read_msg().await?; info!(self.log(), "ram_pull: got done {:?}", m); @@ -630,15 +772,20 @@ impl SourceProtocol { } async fn server_state(&mut self) -> Result<(), MigrateError> { - self.update_state(MigrationState::Server).await; + self.update_state(MigrationState::Server); let remote_addr = match self.read_msg().await? { Message::Serialized(s) => { ron::from_str(&s).map_err(codec::ProtocolError::from)? } _ => return Err(MigrateError::UnexpectedMessage), }; - let com1_history = - self.vm_controller.com1().export_history(remote_addr).await?; + let com1_history = self + .vm + .lock_shared() + .await + .com1() + .export_history(remote_addr) + .await?; self.send_msg(codec::Message::Serialized(com1_history)).await?; self.read_ok().await } @@ -658,7 +805,7 @@ impl SourceProtocol { // Now that handoff is complete, publish that the migration has // succeeded. - self.update_state(MigrationState::Finish).await; + self.update_state(MigrationState::Finish); // This VMM is going away, so if any guest memory is still dirty, it // won't be transferred. Assert that there is no such memory. @@ -671,7 +818,7 @@ impl SourceProtocol { let mut bits = [0u8; PAGE_BITMAP_SIZE]; let step = bits.len() * 8 * PAGE_SIZE; for gpa in (vmm_range.start().0..vmm_range.end().0).step_by(step) { - self.track_dirty(GuestAddr(gpa), &mut bits).unwrap(); + self.track_dirty(GuestAddr(gpa), &mut bits).await.unwrap(); let pages_left_behind = BitSlice::<_, Lsb0>::from_slice(&bits).count_ones() as u64; assert_eq!( @@ -701,7 +848,10 @@ impl SourceProtocol { // If this is an error message, lift that out .map(|msg| match msg { codec::Message::Error(err) => { - error!(self.log(), "remote error: {err}"); + error!( + self.log(), + "migration failed due to error from target: {err}" + ); Err(MigrateError::RemoteError( MigrateRole::Destination, err.to_string(), @@ -748,19 +898,20 @@ impl SourceProtocol { async fn vmm_ram_bounds( &mut self, ) -> Result, MigrateError> { - let machine = self.vm_controller.machine(); - let memctx = machine.acc_mem.access().unwrap(); + let objects = self.vm.lock_shared().await; + let memctx = objects.access_mem().unwrap(); memctx.mem_bounds().ok_or(MigrateError::InvalidInstanceState) } - fn track_dirty( + async fn track_dirty( &mut self, start_gpa: GuestAddr, bits: &mut [u8], ) -> Result<(), MigrateError> { - self.vm_controller - .machine() - .hdl + self.vm + .lock_shared() + .await + .vmm_hdl() .track_dirty_pages(start_gpa.0, bits) .map_err(|_| MigrateError::InvalidInstanceState) } @@ -770,8 +921,8 @@ impl SourceProtocol { addr: GuestAddr, buf: &mut [u8], ) -> Result<(), MigrateError> { - let machine = self.vm_controller.machine(); - let memctx = machine.acc_mem.access().unwrap(); + let objects = self.vm.lock_shared().await; + let memctx = objects.access_mem().unwrap(); let len = buf.len(); memctx.direct_read_into(addr, buf, len); Ok(()) diff --git a/bin/propolis-server/src/lib/server.rs b/bin/propolis-server/src/lib/server.rs index ecaae8019..eba893858 100644 --- a/bin/propolis-server/src/lib/server.rs +++ b/bin/propolis-server/src/lib/server.rs @@ -6,17 +6,17 @@ //! //! Functions in this module verify parameters and convert between types (API //! request types to Propolis-native types and Propolis-native error types to -//! HTTP error codes) before sending operations to other components (e.g. the VM -//! controller) for processing. +//! HTTP error codes) before sending operations to the VM state machine for +//! processing. use std::convert::TryFrom; use std::net::Ipv6Addr; +use std::net::SocketAddr; use std::net::SocketAddrV6; use std::sync::Arc; -use std::{collections::BTreeMap, net::SocketAddr}; use crate::serial::history_buffer::SerialHistoryOffset; -use crate::serial::SerialTaskControlMessage; +use crate::vm::VmError; use dropshot::{ channel, endpoint, ApiDescription, HttpError, HttpResponseCreated, HttpResponseOk, HttpResponseUpdatedNoContent, Path, Query, RequestContext, @@ -28,31 +28,19 @@ use internal_dns::ServiceName; pub use nexus_client::Client as NexusClient; use oximeter::types::ProducerRegistry; use propolis_api_types as api; -use propolis_api_types::instance_spec::{ - self, components::backends::CrucibleStorageBackend, v0::StorageBackendV0, - VersionedInstanceSpec, -}; +use propolis_api_types::instance_spec::{self, VersionedInstanceSpec}; pub use propolis_server_config::Config as VmTomlConfig; use rfb::server::VncServer; -use slog::{error, info, o, warn, Logger}; +use slog::{error, warn, Logger}; use thiserror::Error; -use tokio::sync::{mpsc, oneshot, MappedMutexGuard, Mutex, MutexGuard}; +use tokio::sync::MutexGuard; use tokio_tungstenite::tungstenite::protocol::{Role, WebSocketConfig}; use tokio_tungstenite::WebSocketStream; use crate::spec::{ServerSpecBuilder, ServerSpecBuilderError}; -use crate::stats::virtual_machine::VirtualMachine; -use crate::vm::VmController; use crate::vnc::PropolisVncServer; -pub(crate) type DeviceMap = - BTreeMap>; -pub(crate) type BlockBackendMap = - BTreeMap>; -pub(crate) type CrucibleBackendMap = - BTreeMap>; - /// Configuration used to set this server up to provide Oximeter metrics. #[derive(Debug, Clone)] pub struct MetricsEndpointConfig { @@ -76,7 +64,7 @@ impl MetricsEndpointConfig { /// objects. pub struct StaticConfig { /// The TOML-driven configuration for this server's instances. - pub vm: VmTomlConfig, + pub vm: Arc, /// Whether to use the host's guest memory reservoir to back guest memory. pub use_reservoir: bool, @@ -86,164 +74,11 @@ pub struct StaticConfig { metrics: Option, } -/// The state of the current VM controller in this server, if there is one, or -/// the most recently created one, if one ever existed. -pub enum VmControllerState { - /// No VM controller has ever been constructed in this server. - NotCreated, - - /// A VM controller exists. - Created(Arc), - - /// No VM controller exists. - /// - /// Distinguishing this state from `NotCreated` allows the server to discard - /// the active `VmController` on instance stop while still being able to - /// service get requests for the instance. (If this were not needed, or the - /// server were willing to preserve the `VmController` after halt, this enum - /// could be replaced with an `Option`.) - Destroyed { - /// A copy of the instance properties recorded at the time the instance - /// was destroyed, used to serve subsequent `instance_get` requests. - last_instance: Box, - - /// A copy of the destroyed instance's spec, used to serve subsequent - /// `instance_spec_get` requests. - // - // TODO: Merge this into `api::Instance` when the migration to generated - // types is complete. - last_instance_spec: Box, - - /// A clone of the receiver side of the server's state watcher, used to - /// serve subsequent `instance_state_monitor` requests. Note that an - /// outgoing controller can publish new state changes even after the - /// server has dropped its reference to it (its state worker may - /// continue running for a time). - state_watcher: - tokio::sync::watch::Receiver, - }, -} - -impl VmControllerState { - /// Maps this `VmControllerState` into a mutable reference to its internal - /// `VmController` if a controller is active. - pub fn as_controller(&mut self) -> Option<&mut Arc> { - match self { - VmControllerState::NotCreated => None, - VmControllerState::Created(c) => Some(c), - VmControllerState::Destroyed { .. } => None, - } - } - - /// Takes the active `VmController` if one is present and replaces it with - /// `VmControllerState::Destroyed`. - pub async fn take_controller(&mut self) -> Option> { - if let VmControllerState::Created(vm) = self { - let state = vm.state_watcher().borrow().state; - let last_instance = api::Instance { - properties: vm.properties().clone(), - state, - disks: vec![], - nics: vec![], - }; - let last_instance_spec = vm.instance_spec().await.clone(); - - // Preserve the state watcher so that subsequent updates to the VM's - // state are visible to calls to query/monitor that state. Note that - // the VM's state will change at least once more after this point: - // the final transition to the "destroyed" state happens only when - // all references to the VM have been dropped, including the one - // this routine just exchanged and will return. - let state_watcher = vm.state_watcher().clone(); - if let VmControllerState::Created(vm) = std::mem::replace( - self, - VmControllerState::Destroyed { - last_instance: Box::new(last_instance), - last_instance_spec: Box::new(last_instance_spec), - state_watcher, - }, - ) { - Some(vm) - } else { - unreachable!() - } - } else { - None - } - } -} - -/// Objects related to Propolis's Oximeter metric production. -pub struct OximeterState { - /// The metric producer server. - server: Option, - - /// The metrics wrapper for "server-level" metrics, i.e., metrics that are - /// tracked by the server itself (as opposed to being tracked by a component - /// within an instance). - stats: Option, -} - -/// Objects that this server creates, owns, and manipulates in response to API -/// calls. -pub struct ServiceProviders { - /// The VM controller that manages this server's Propolis instance. This is - /// `None` until a guest is created via `instance_ensure`. - pub vm: Mutex, - - /// The currently active serial console handling task, if present. - serial_task: Mutex>, - - /// State related to the Propolis Oximeter server and actual statistics. - oximeter_state: Mutex, - - /// The VNC server hosted within this process. Note that this server always - /// exists irrespective of whether there is an instance. Creating an - /// instance hooks this server up to the instance's framebuffer. - vnc_server: Arc>, -} - -impl ServiceProviders { - /// Directs the current set of per-instance service providers to stop in an - /// orderly fashion, then drops them all. - async fn stop(&self, log: &Logger) { - // Stop the VNC server - self.vnc_server.stop().await; - - if let Some(vm) = self.vm.lock().await.take_controller().await { - slog::info!(log, "Dropping server's VM controller reference"; - "strong_refs" => Arc::strong_count(&vm), - "weak_refs" => Arc::weak_count(&vm), - ); - } - if let Some(serial_task) = self.serial_task.lock().await.take() { - let _ = serial_task - .control_ch - .send(SerialTaskControlMessage::Stopping) - .await; - // Wait for the serial task to exit - let _ = serial_task.task.await; - } - - // Clean up oximeter tasks and statistic state. - let mut oximeter_state = self.oximeter_state.lock().await; - if let Some(server) = oximeter_state.server.take() { - if let Err(e) = server.close().await { - error!( - log, - "failed to close oximeter producer server"; - "error" => ?e, - ); - }; - } - let _ = oximeter_state.stats.take(); - } -} - /// Context accessible from HTTP callbacks. pub struct DropshotEndpointContext { static_config: StaticConfig, - pub services: Arc, + vnc_server: Arc>, + pub(crate) vm: Arc, log: Logger, } @@ -258,34 +93,15 @@ impl DropshotEndpointContext { ) -> Self { Self { static_config: StaticConfig { - vm: config, + vm: Arc::new(config), use_reservoir, metrics: metric_config, }, - services: Arc::new(ServiceProviders { - vm: Mutex::new(VmControllerState::NotCreated), - serial_task: Mutex::new(None), - oximeter_state: Mutex::new(OximeterState { - server: None, - stats: None, - }), - vnc_server, - }), + vnc_server, + vm: crate::vm::Vm::new(&log), log, } } - - /// Get access to the VM controller for this context, emitting a consistent - /// error if it is absent. - pub(crate) async fn vm( - &self, - ) -> Result>, HttpError> { - MutexGuard::try_map( - self.services.vm.lock().await, - VmControllerState::as_controller, - ) - .map_err(|_| not_created_error()) - } } #[derive(Debug, Error)] @@ -330,69 +146,6 @@ fn instance_spec_from_request( Ok(VersionedInstanceSpec::V0(spec_builder.finish())) } -/// Register an Oximeter server reporting metrics from a new instance. -async fn register_oximeter_producer( - services: Arc, - cfg: MetricsEndpointConfig, - registry: &ProducerRegistry, - virtual_machine: VirtualMachine, - log: Logger, -) { - let mut oximeter_state = services.oximeter_state.lock().await; - assert!(oximeter_state.stats.is_none()); - assert!(oximeter_state.server.is_none()); - - // Create the server itself. - // - // The server manages all details of the registration with Nexus, so we - // don't need our own task for that or way to shut it down. - match crate::stats::start_oximeter_server( - virtual_machine.instance_id, - &cfg, - &log, - registry, - ) { - Ok(server) => { - info!(log, "created metric producer server"); - let old = oximeter_state.server.replace(server); - assert!(old.is_none()); - } - Err(err) => { - error!( - log, - "failed to construct metric producer server, \ - no metrics will be available for this instance."; - "error" => ?err, - ); - } - } - - // Assign our own metrics production for this VM instance to the - // registry, letting the server actually return them to oximeter when - // polled. - let stats = match crate::stats::register_server_metrics( - registry, - virtual_machine, - &log, - ) - .await - { - Ok(stats) => stats, - Err(e) => { - error!( - log, - "failed to register our server metrics with \ - the ProducerRegistry, no server stats will \ - be produced"; - "error" => ?e, - ); - return; - } - }; - let old = oximeter_state.stats.replace(stats); - assert!(old.is_none()); -} - /// Wrapper around a [`NexusClient`] object, which allows deferring /// the DNS lookup until accessed. /// @@ -474,177 +227,49 @@ async fn instance_ensure_common( request: api::InstanceSpecEnsureRequest, ) -> Result, HttpError> { let server_context = rqctx.context(); - let api::InstanceSpecEnsureRequest { properties, instance_spec, migrate } = - request; - - // Handle requests to an instance that has already been initialized. Treat - // the instances as compatible (and return Ok) if they have the same - // properties and return an appropriate error otherwise. - // - // TODO(#205): Consider whether to use this interface to change an - // instance's devices and backends at runtime. - if let VmControllerState::Created(existing) = - &*server_context.services.vm.lock().await - { - let existing_properties = existing.properties(); - if existing_properties.id != properties.id { - return Err(HttpError::for_client_error( - Some(api::ErrorCode::AlreadyInitialized.to_string()), - http::status::StatusCode::CONFLICT, - format!( - "Server already initialized with ID {}", - existing_properties.id - ), - )); - } - - if *existing_properties != properties { - return Err(HttpError::for_client_error( - Some(api::ErrorCode::AlreadyRunning.to_string()), - http::status::StatusCode::CONFLICT, - "Cannot update running server".to_string(), - )); - } - - return Ok(HttpResponseCreated(api::InstanceEnsureResponse { - migrate: None, - })); - } - - let producer_registry = - if let Some(cfg) = server_context.static_config.metrics.as_ref() { - // Create a registry and spawn tasks to register with Nexus as an - // oximeter metric producer. - // - // We create a registry here so that we can pass it through to Crucible - // below. We also spawn a task for the actual registration process - // (which may spin indefinitely) so that we can continue to initialize - // the VM instance without blocking for that to succeed. - let registry = ProducerRegistry::with_id(properties.id); - let virtual_machine = VirtualMachine::from(&properties); - register_oximeter_producer( - server_context.services.clone(), - cfg.clone(), - ®istry, - virtual_machine, - rqctx.log.clone(), - ) - .await; - Some(registry) - } else { - None - }; - - let (stop_ch, stop_recv) = oneshot::channel(); + let oximeter_registry = server_context + .static_config + .metrics + .as_ref() + .map(|_| ProducerRegistry::with_id(request.properties.id)); - // Use our current address to generate the expected Nexus client endpoint - // address. let nexus_client = find_local_nexus_client(rqctx.server.local_addr, rqctx.log.clone()) .await; - // Parts of VM initialization (namely Crucible volume attachment) make use - // of async processing, which itself is turned synchronous with `block_on` - // calls to the Tokio runtime. - // - // Since `block_on` will panic if called from an async context, as we are in - // now, the whole process is wrapped up in `spawn_blocking`. It is - // admittedly a big kludge until this can be better refactored. - let vm = { - let properties = properties.clone(); - let server_context = server_context.clone(); - let log = server_context.log.clone(); - - // Block for VM controller setup under the current (API) runtime - let cur_rt_hdl = tokio::runtime::Handle::current(); - let vm_hdl = cur_rt_hdl.spawn_blocking(move || { - VmController::new( - instance_spec, - properties, - &server_context.static_config, - producer_registry, - nexus_client, - log, - stop_ch, - ) - }); - - vm_hdl.await.unwrap() - } - .map_err(|e| { - HttpError::for_internal_error(format!("failed to create instance: {e}")) - })?; - - if let Some(ramfb) = vm.framebuffer() { - // Get a framebuffer description from the wrapped instance. - let fb_spec = ramfb.get_framebuffer_spec(); - let vnc_fb = crate::vnc::RamFb::new(fb_spec); - - // Get a reference to the PS2 controller so that we can pass keyboard input. - let ps2ctrl = vm.ps2ctrl().clone(); - - // Get a reference to the outward-facing VNC server in this process. - let vnc_server = server_context.services.vnc_server.clone(); - - // Initialize the Propolis VNC adapter with references to the VM's Instance, - // framebuffer, and PS2 controller. - vnc_server.server.initialize(vnc_fb, ps2ctrl, vm.clone()).await; - - // Hook up the framebuffer notifier to update the Propolis VNC adapter - let notifier_server_ref = vnc_server.clone(); - let rt = tokio::runtime::Handle::current(); - ramfb.set_notifier(Box::new(move |config, is_valid| { - let vnc = notifier_server_ref.clone(); - rt.block_on(vnc.server.update(config, is_valid, &vnc)); - })); - } - - let mut serial_task = server_context.services.serial_task.lock().await; - if serial_task.is_none() { - let (websocks_ch, websocks_recv) = mpsc::channel(1); - let (control_ch, control_recv) = mpsc::channel(1); - - let serial = vm.com1().clone(); - serial.set_task_control_sender(control_ch.clone()).await; - let err_log = rqctx.log.new(o!("component" => "serial task")); - let task = tokio::spawn(async move { - if let Err(e) = super::serial::instance_serial_task( - websocks_recv, - control_recv, - serial, - err_log.clone(), - ) - .await - { - error!(err_log, "Failure in serial task: {}", e); - } - }); - *serial_task = - Some(super::serial::SerialTask { task, control_ch, websocks_ch }); - } - - let log = server_context.log.clone(); - let services = Arc::clone(&server_context.services); - tokio::task::spawn(async move { - // Once the VmController has signaled that it is shutting down, - // we'll clean up the per-instance service providers as well. - let _ = stop_recv.await; - services.stop(&log).await; - }); - - *server_context.services.vm.lock().await = - VmControllerState::Created(vm.clone()); - - let migrate = if let Some(migrate_request) = migrate { - let res = crate::migrate::dest_initiate(&rqctx, vm, migrate_request) - .await - .map_err(<_ as Into>::into)?; - Some(res) - } else { - None + let ensure_options = crate::vm::EnsureOptions { + toml_config: server_context.static_config.vm.clone(), + use_reservoir: server_context.static_config.use_reservoir, + metrics_config: server_context.static_config.metrics.clone(), + oximeter_registry, + nexus_client, + vnc_server: server_context.vnc_server.clone(), + local_server_addr: rqctx.server.local_addr, }; - Ok(HttpResponseCreated(api::InstanceEnsureResponse { migrate })) + server_context + .vm + .ensure(&server_context.log, request, ensure_options) + .await + .map(HttpResponseCreated) + .map_err(|e| match e { + VmError::ResultChannelClosed => HttpError::for_internal_error( + "state driver unexpectedly dropped result channel".to_string(), + ), + VmError::WaitingToInitialize + | VmError::AlreadyInitialized + | VmError::RundownInProgress => HttpError::for_client_error( + Some(api::ErrorCode::AlreadyInitialized.to_string()), + http::StatusCode::CONFLICT, + "instance already initialized".to_string(), + ), + VmError::InitializationFailed(e) => HttpError::for_internal_error( + format!("VM initialization failed: {e}"), + ), + _ => HttpError::for_internal_error(format!( + "unexpected error from VM controller: {e}" + )), + }) } #[endpoint { @@ -693,40 +318,16 @@ async fn instance_spec_ensure( async fn instance_get_common( rqctx: &RequestContext>, -) -> Result<(api::Instance, VersionedInstanceSpec), HttpError> { +) -> Result { let ctx = rqctx.context(); - match &*ctx.services.vm.lock().await { - VmControllerState::NotCreated => Err(not_created_error()), - VmControllerState::Created(vm) => { - Ok(( - api::Instance { - properties: vm.properties().clone(), - state: vm.external_instance_state(), - disks: vec![], - // TODO: Fix this; we need a way to enumerate attached NICs. - // Possibly using the inventory of the instance? - // - // We *could* record whatever information about the NIC we want - // when they're requested (adding fields to the server), but that - // would make it difficult for Propolis to update any dynamic info - // (i.e., has the device faulted, etc). - nics: vec![], - }, - vm.instance_spec().await.clone(), - )) + ctx.vm.get().await.map_err(|e| match e { + VmError::NotCreated | VmError::WaitingToInitialize => { + not_created_error() } - VmControllerState::Destroyed { - last_instance, - last_instance_spec, - state_watcher, - .. - } => { - let watcher = state_watcher.borrow(); - let mut last_instance = last_instance.clone(); - last_instance.state = watcher.state; - Ok((*last_instance, *last_instance_spec.clone())) - } - } + _ => HttpError::for_internal_error(format!( + "unexpected error from VM controller: {e}" + )), + }) } #[endpoint { @@ -736,12 +337,7 @@ async fn instance_get_common( async fn instance_spec_get( rqctx: RequestContext>, ) -> Result, HttpError> { - let (instance, spec) = instance_get_common(&rqctx).await?; - Ok(HttpResponseOk(api::InstanceSpecGetResponse { - properties: instance.properties, - state: instance.state, - spec, - })) + Ok(HttpResponseOk(instance_get_common(&rqctx).await?)) } #[endpoint { @@ -751,8 +347,16 @@ async fn instance_spec_get( async fn instance_get( rqctx: RequestContext>, ) -> Result, HttpError> { - let (instance, _) = instance_get_common(&rqctx).await?; - Ok(HttpResponseOk(api::InstanceGetResponse { instance })) + instance_get_common(&rqctx).await.map(|full| { + HttpResponseOk(api::InstanceGetResponse { + instance: api::Instance { + properties: full.properties, + state: full.state, + disks: vec![], + nics: vec![], + }, + }) + }) } #[endpoint { @@ -765,19 +369,15 @@ async fn instance_state_monitor( ) -> Result, HttpError> { let ctx = rqctx.context(); let gen = request.into_inner().gen; - let mut state_watcher = { - // N.B. This lock must be dropped before entering the loop below. - let vm_state = ctx.services.vm.lock().await; - match &*vm_state { - VmControllerState::NotCreated => { - return Err(not_created_error()); - } - VmControllerState::Created(vm) => vm.state_watcher().clone(), - VmControllerState::Destroyed { state_watcher, .. } => { - state_watcher.clone() + let mut state_watcher = + ctx.vm.state_watcher().await.map_err(|e| match e { + VmError::NotCreated | VmError::WaitingToInitialize => { + not_created_error() } - } - }; + _ => HttpError::for_internal_error(format!( + "unexpected error from VM controller: {e}" + )), + })?; loop { let last = state_watcher.borrow().clone(); @@ -812,19 +412,29 @@ async fn instance_state_put( ) -> Result { let ctx = rqctx.context(); let requested_state = request.into_inner(); - let vm = ctx.vm().await?; + let vm = ctx.vm.active_vm().await.ok_or_else(not_created_error)?; let result = vm .put_state(requested_state) .map(|_| HttpResponseUpdatedNoContent {}) - .map_err(|e| e.into()); + .map_err(|e| match e { + VmError::NotCreated | VmError::WaitingToInitialize => { + not_created_error() + } + VmError::ForbiddenStateChange(reason) => HttpError::for_status( + Some(format!("instance state change not allowed: {}", reason)), + http::status::StatusCode::FORBIDDEN, + ), + _ => HttpError::for_internal_error(format!( + "unexpected error from VM controller: {e}" + )), + }); - drop(vm); if result.is_ok() { if let api::InstanceStateRequested::Reboot = requested_state { - let stats = MutexGuard::map( - ctx.services.oximeter_state.lock().await, - |state| &mut state.stats, - ); + let stats = + MutexGuard::map(vm.services().oximeter.lock().await, |state| { + &mut state.stats + }); if let Some(stats) = stats.as_ref() { stats.count_reset(); } @@ -844,8 +454,8 @@ async fn instance_serial_history_get( ) -> Result, HttpError> { let ctx = rqctx.context(); - let vm = ctx.vm().await?; - let serial = vm.com1().clone(); + let vm = ctx.vm.active_vm().await.ok_or_else(not_created_error)?; + let serial = vm.objects().lock_shared().await.com1().clone(); let query_params = query.into_inner(); let byte_offset = SerialHistoryOffset::try_from(&query_params)?; @@ -872,8 +482,8 @@ async fn instance_serial( websock: WebsocketConnection, ) -> dropshot::WebsocketChannelResult { let ctx = rqctx.context(); - let vm = ctx.vm().await?; - let serial = vm.com1().clone(); + let vm = ctx.vm.active_vm().await.ok_or_else(not_created_error)?; + let serial = vm.objects().lock_shared().await.com1().clone(); // Use the default buffering paramters for the websocket configuration // @@ -904,10 +514,8 @@ async fn instance_serial( } // Get serial task's handle and send it the websocket stream - ctx.services - .serial_task - .lock() - .await + let serial_task = vm.services().serial_task.lock().await; + serial_task .as_ref() .ok_or("Instance has no serial task")? .websocks_ch @@ -916,10 +524,10 @@ async fn instance_serial( .map_err(|e| format!("Serial socket hand-off failed: {}", e).into()) } -// This endpoint is meant to only be called during a migration from the destination -// instance to the source instance as part of the HTTP connection upgrade used to -// establish the migration link. We don't actually want this exported via OpenAPI -// clients. +// This endpoint is meant to only be called during a migration from the +// destination instance to the source instance as part of the HTTP connection +// upgrade used to establish the migration link. We don't actually want this +// exported via OpenAPI clients. #[channel { protocol = WEBSOCKETS, path = "/instance/migrate/{migration_id}/start", @@ -930,15 +538,10 @@ async fn instance_migrate_start( path_params: Path, websock: WebsocketConnection, ) -> dropshot::WebsocketChannelResult { + let ctx = rqctx.context(); let migration_id = path_params.into_inner().migration_id; - let conn = WebSocketStream::from_raw_socket( - websock.into_inner(), - Role::Server, - None, - ) - .await; - crate::migrate::source_start(rqctx, migration_id, conn).await?; - Ok(()) + let vm = ctx.vm.active_vm().await.ok_or_else(not_created_error)?; + Ok(vm.request_migration_out(migration_id, websock).await?) } #[endpoint { @@ -949,15 +552,18 @@ async fn instance_migrate_status( rqctx: RequestContext>, ) -> Result, HttpError> { let ctx = rqctx.context(); - match &*ctx.services.vm.lock().await { - VmControllerState::NotCreated => Err(not_created_error()), - VmControllerState::Created(vm) => { - Ok(HttpResponseOk(vm.migrate_status())) - } - VmControllerState::Destroyed { state_watcher, .. } => { - Ok(HttpResponseOk(state_watcher.borrow().migration.clone())) - } - } + ctx.vm + .state_watcher() + .await + .map(|rx| HttpResponseOk(rx.borrow().migration.clone())) + .map_err(|e| match e { + VmError::NotCreated | VmError::WaitingToInitialize => { + not_created_error() + } + _ => HttpError::for_internal_error(format!( + "unexpected error from VM controller: {e}" + )), + }) } /// Issues a snapshot request to a crucible backend. @@ -969,14 +575,16 @@ async fn instance_issue_crucible_snapshot_request( rqctx: RequestContext>, path_params: Path, ) -> Result, HttpError> { - let inst = rqctx.context().vm().await?; - let crucible_backends = inst.crucible_backends(); + let vm = + rqctx.context().vm.active_vm().await.ok_or_else(not_created_error)?; + let objects = vm.objects().lock_shared().await; let path_params = path_params.into_inner(); - let backend = crucible_backends.get(&path_params.id).ok_or_else(|| { - let s = format!("no disk with id {}!", path_params.id); - HttpError::for_not_found(Some(s.clone()), s) - })?; + let backend = + objects.crucible_backends().get(&path_params.id).ok_or_else(|| { + let s = format!("no disk with id {}!", path_params.id); + HttpError::for_not_found(Some(s.clone()), s) + })?; backend.snapshot(path_params.snapshot_id).await.map_err(|e| { HttpError::for_bad_request(Some(e.to_string()), e.to_string()) })?; @@ -994,14 +602,14 @@ async fn disk_volume_status( path_params: Path, ) -> Result, HttpError> { let path_params = path_params.into_inner(); - - let vm_controller = rqctx.context().vm().await?; - - let crucible_backends = vm_controller.crucible_backends(); - let backend = crucible_backends.get(&path_params.id).ok_or_else(|| { - let s = format!("No crucible backend for id {}", path_params.id); - HttpError::for_not_found(Some(s.clone()), s) - })?; + let vm = + rqctx.context().vm.active_vm().await.ok_or_else(not_created_error)?; + let objects = vm.objects().lock_shared().await; + let backend = + objects.crucible_backends().get(&path_params.id).ok_or_else(|| { + let s = format!("No crucible backend for id {}", path_params.id); + HttpError::for_not_found(Some(s.clone()), s) + })?; Ok(HttpResponseOk(api::VolumeStatus { active: backend.volume_is_active().await.map_err(|e| { @@ -1024,66 +632,29 @@ async fn instance_issue_crucible_vcr_request( let request = request.into_inner(); let new_vcr_json = request.vcr_json; let disk_name = request.name; - let log = rqctx.log.clone(); - - // Get the instance spec for storage backend from the disk name. We use - // the VCR stored there to send to crucible along with the new VCR we want - // to replace it. - let vm_controller = rqctx.context().vm().await?; - - // TODO(#205): Mutating a VM's configuration should be a first-class - // operation in the VM controller that synchronizes with ongoing migrations - // and other attempts to mutate the VM. For the time being, use the instance - // spec lock to exclude other concurrent attempts to reconfigure this - // backend. - let mut spec = vm_controller.instance_spec().await; - let VersionedInstanceSpec::V0(v0_spec) = &mut *spec; - - let (readonly, old_vcr_json) = { - let bes = &v0_spec.backends.storage_backends.get(&disk_name); - if let Some(StorageBackendV0::Crucible(bes)) = bes { - (bes.readonly, &bes.request_json) - } else { - let s = format!("Crucible backend for {:?} not found", disk_name); - return Err(HttpError::for_not_found(Some(s.clone()), s)); - } - }; - - // Get the crucible backend so we can call the replacement method on it. - let crucible_backends = vm_controller.crucible_backends(); - let backend = crucible_backends.get(&path_params.id).ok_or_else(|| { - let s = format!("No crucible backend for id {}", path_params.id); - HttpError::for_not_found(Some(s.clone()), s) - })?; - slog::info!( - log, - "{:?} {:?} vcr replace requested", - disk_name, - path_params.id, - ); - - // Try the replacement. - // Crucible does the heavy lifting here to verify that the old/new - // VCRs are different in just the correct way and will return error - // if there is any mismatch. - let replace_result = - backend.vcr_replace(old_vcr_json, &new_vcr_json).await.map_err( - |e| HttpError::for_bad_request(Some(e.to_string()), e.to_string()), - )?; - - // Our replacement request was accepted. We now need to update the - // spec stored in propolis so it matches what the downstairs now has. - let new_storage_backend: StorageBackendV0 = - StorageBackendV0::Crucible(CrucibleStorageBackend { - readonly, - request_json: new_vcr_json, - }); - v0_spec.backends.storage_backends.insert(disk_name, new_storage_backend); + let (tx, rx) = tokio::sync::oneshot::channel(); + let vm = + rqctx.context().vm.active_vm().await.ok_or_else(not_created_error)?; + + vm.reconfigure_crucible_volume(disk_name, path_params.id, new_vcr_json, tx) + .map_err(|e| match e { + VmError::ForbiddenStateChange(reason) => HttpError::for_status( + Some(format!("instance state change not allowed: {}", reason)), + http::status::StatusCode::FORBIDDEN, + ), + _ => HttpError::for_internal_error(format!( + "unexpected error from VM controller: {e}" + )), + })?; - slog::info!(log, "Replaced the VCR in backend of {:?}", path_params.id); + let result = rx.await.map_err(|_| { + HttpError::for_internal_error( + "VM worker task unexpectedly dropped result channel".to_string(), + ) + })?; - Ok(HttpResponseOk(replace_result)) + result.map(HttpResponseOk) } /// Issues an NMI to the instance. @@ -1094,8 +665,9 @@ async fn instance_issue_crucible_vcr_request( async fn instance_issue_nmi( rqctx: RequestContext>, ) -> Result, HttpError> { - let vm = rqctx.context().vm().await?; - vm.inject_nmi(); + let vm = + rqctx.context().vm.active_vm().await.ok_or_else(not_created_error)?; + let _ = vm.objects().lock_shared().await.machine().inject_nmi(); Ok(HttpResponseOk(())) } diff --git a/bin/propolis-server/src/lib/vcpu_tasks.rs b/bin/propolis-server/src/lib/vcpu_tasks.rs index cdbb20412..be2d37f35 100644 --- a/bin/propolis-server/src/lib/vcpu_tasks.rs +++ b/bin/propolis-server/src/lib/vcpu_tasks.rs @@ -30,7 +30,7 @@ pub struct VcpuTasks { } #[cfg_attr(test, mockall::automock)] -pub(crate) trait VcpuTaskController { +pub(crate) trait VcpuTaskController: Send + Sync + 'static { fn new_generation(&self); fn pause_all(&mut self); fn resume_all(&mut self); @@ -40,7 +40,7 @@ pub(crate) trait VcpuTaskController { impl VcpuTasks { pub(crate) fn new( machine: &propolis::Machine, - event_handler: Arc, + event_handler: Arc, log: slog::Logger, ) -> Result { let generation = Arc::new(AtomicUsize::new(0)); @@ -72,7 +72,7 @@ impl VcpuTasks { fn vcpu_loop( vcpu: &Vcpu, task: propolis::tasks::TaskHdl, - event_handler: Arc, + event_handler: Arc, generation: Arc, log: slog::Logger, ) { diff --git a/bin/propolis-server/src/lib/vm/active.rs b/bin/propolis-server/src/lib/vm/active.rs new file mode 100644 index 000000000..1274782f9 --- /dev/null +++ b/bin/propolis-server/src/lib/vm/active.rs @@ -0,0 +1,123 @@ +// This Source Code Form is subject to the terms of the Mozilla Public +// License, v. 2.0. If a copy of the MPL was not distributed with this +// file, You can obtain one at https://mozilla.org/MPL/2.0/. + +//! Implements a wrapper around an active VM. + +use std::sync::Arc; + +use propolis_api_types::{InstanceProperties, InstanceStateRequested}; +use slog::info; +use uuid::Uuid; + +use crate::vm::request_queue::ExternalRequest; + +use super::{ + objects::VmObjects, services::VmServices, CrucibleReplaceResultTx, + InstanceStateRx, VmError, +}; + +/// The components and services that make up an active Propolis VM. +pub(crate) struct ActiveVm { + /// The VM's logger. + pub(super) log: slog::Logger, + + /// The input queue that receives external requests to change the VM's + /// state. + pub(super) state_driver_queue: Arc, + + /// Receives external state updates from the state driver. + pub(super) external_state_rx: InstanceStateRx, + + /// The wrapped VM's properties. + pub(super) properties: InstanceProperties, + + /// A reference to the wrapped VM's components. Callers with a reference to + /// an `ActiveVm` can clone this to get a handle to those components. + pub(super) objects: Arc, + + /// Services that interact with VM users or the control plane outside the + /// Propolis API (e.g. the serial console, VNC, and metrics reporting). + pub(super) services: VmServices, + + /// The runtime on which this VM's state driver and any tasks spawned by + /// the VM's components will run. + pub(super) tokio_rt: tokio::runtime::Runtime, +} + +impl ActiveVm { + /// Yields a clonable reference to the active VM's components. + pub(crate) fn objects(&self) -> &Arc { + &self.objects + } + + /// Pushes a state change request to the VM's state change queue. + pub(crate) fn put_state( + &self, + requested: InstanceStateRequested, + ) -> Result<(), VmError> { + info!(self.log, "requested state via API"; + "state" => ?requested); + + self.state_driver_queue + .queue_external_request(match requested { + InstanceStateRequested::Run => ExternalRequest::Start, + InstanceStateRequested::Stop => ExternalRequest::Stop, + InstanceStateRequested::Reboot => ExternalRequest::Reboot, + }) + .map_err(Into::into) + } + + /// Pushes a request to migrate out of a VM to the VM's state change queue. + /// The migration protocol will communicate with the destination over the + /// provided websocket. + pub(crate) async fn request_migration_out( + &self, + migration_id: Uuid, + websock: dropshot::WebsocketConnection, + ) -> Result<(), VmError> { + Ok(self.state_driver_queue.queue_external_request( + ExternalRequest::MigrateAsSource { + migration_id, + websock: websock.into(), + }, + )?) + } + + /// Pushes a request to reconfigure a Crucible volume to the VM's state + /// change queue. + /// + /// # Arguments + /// + /// - `disk_name`: The name of the Crucible disk component (in the instance + /// spec) to modify. + /// - `backend_id`: The UUID to use to find the Crucible backend in the + /// VM's Crucible backend map. + /// - `new_vcr_json`: The new volume construction request to supply to the + /// selected backend. + /// - `result_tx`: The channel to which the state driver should send the + /// replacement result after it completes this operation. + pub(crate) fn reconfigure_crucible_volume( + &self, + disk_name: String, + backend_id: Uuid, + new_vcr_json: String, + result_tx: CrucibleReplaceResultTx, + ) -> Result<(), VmError> { + self.state_driver_queue + .queue_external_request( + ExternalRequest::ReconfigureCrucibleVolume { + disk_name, + backend_id, + new_vcr_json, + result_tx, + }, + ) + .map_err(Into::into) + } + + /// Yields a reference to this VM's services. + pub(crate) fn services(&self) -> &VmServices { + &self.services + } +} diff --git a/bin/propolis-server/src/lib/vm/ensure.rs b/bin/propolis-server/src/lib/vm/ensure.rs new file mode 100644 index 000000000..b3e3f1de8 --- /dev/null +++ b/bin/propolis-server/src/lib/vm/ensure.rs @@ -0,0 +1,364 @@ +// This Source Code Form is subject to the terms of the Mozilla Public +// License, v. 2.0. If a copy of the MPL was not distributed with this +// file, You can obtain one at https://mozilla.org/MPL/2.0/. + +//! Tools for handling instance ensure requests. +//! +//! To initialize a new VM, the server must (1) create a set of VM objects from +//! an instance spec, (2) set up VM services that use those objects, (3) use the +//! objects and services to drive the VM state machine to the `ActiveVm` state, +//! and (4) notify the original caller of the "instance ensure" API of the +//! completion of its request. If VM initialization fails, the actions required +//! to compensate and drive the state machine to `RundownComplete` depend on how +//! many steps were completed. +//! +//! When live migrating into an instance, the live migration task interleaves +//! initialization steps with the steps of the live migration protocol, and +//! needs to be able to unwind initialization correctly whenever the migration +//! protocol fails. +//! +//! The `VmEnsure` types in this module exist to hide the gory details of +//! initializing and unwinding from higher-level operations like the live +//! migration task. Each type represents a phase of the initialization process +//! and has a routine that consumes the current phase and moves to the next +//! phase. If a higher-level operation fails, it can call a failure handler on +//! its current phase to unwind the whole operation and drive the VM state +//! machine to the correct resting state. + +use std::sync::Arc; + +use propolis_api_types::{ + instance_spec::{v0::InstanceSpecV0, VersionedInstanceSpec}, + InstanceEnsureResponse, InstanceMigrateInitiateResponse, + InstanceSpecEnsureRequest, InstanceState, +}; +use slog::{debug, info}; + +use crate::{ + initializer::{ + build_instance, MachineInitializer, MachineInitializerState, + }, + vm::request_queue::InstanceAutoStart, +}; + +use super::{ + objects::{InputVmObjects, VmObjects}, + services::VmServices, + state_driver::InputQueue, + state_publisher::{ExternalStateUpdate, StatePublisher}, + EnsureOptions, InstanceEnsureResponseTx, VmError, +}; + +/// Holds state about an instance ensure request that has not yet produced any +/// VM objects or driven the VM state machine to the `ActiveVm` state. +pub(crate) struct VmEnsureNotStarted<'a> { + log: &'a slog::Logger, + vm: &'a Arc, + ensure_request: &'a InstanceSpecEnsureRequest, + ensure_options: &'a EnsureOptions, + ensure_response_tx: InstanceEnsureResponseTx, + state_publisher: &'a mut StatePublisher, +} + +impl<'a> VmEnsureNotStarted<'a> { + pub(super) fn new( + log: &'a slog::Logger, + vm: &'a Arc, + ensure_request: &'a InstanceSpecEnsureRequest, + ensure_options: &'a EnsureOptions, + ensure_response_tx: InstanceEnsureResponseTx, + state_publisher: &'a mut StatePublisher, + ) -> Self { + Self { + log, + vm, + ensure_request, + ensure_options, + ensure_response_tx, + state_publisher, + } + } + + pub(crate) fn instance_spec(&self) -> &InstanceSpecV0 { + let VersionedInstanceSpec::V0(v0) = &self.ensure_request.instance_spec; + v0 + } + + pub(crate) fn state_publisher(&mut self) -> &mut StatePublisher { + self.state_publisher + } + + /// Creates a set of VM objects using the instance spec stored in this + /// ensure request, but does not install them as an active VM. + pub(crate) async fn create_objects( + self, + ) -> anyhow::Result> { + debug!(self.log, "creating VM objects"); + + let input_queue = Arc::new(InputQueue::new( + self.log.new(slog::o!("component" => "request_queue")), + match &self.ensure_request.migrate { + Some(_) => InstanceAutoStart::Yes, + None => InstanceAutoStart::No, + }, + )); + + match self.initialize_vm_objects_from_spec(&input_queue).await { + Ok(objects) => { + // N.B. Once these `VmObjects` exist, it is no longer safe to + // call `vm_init_failed`. + let objects = Arc::new(VmObjects::new( + self.log.clone(), + self.vm.clone(), + objects, + )); + + Ok(VmEnsureObjectsCreated { + log: self.log, + vm: self.vm, + ensure_request: self.ensure_request, + ensure_options: self.ensure_options, + ensure_response_tx: self.ensure_response_tx, + state_publisher: self.state_publisher, + vm_objects: objects, + input_queue, + kernel_vm_paused: false, + }) + } + Err(e) => Err(self.fail(e).await), + } + } + + pub(crate) async fn fail(self, reason: anyhow::Error) -> anyhow::Error { + self.state_publisher + .update(ExternalStateUpdate::Instance(InstanceState::Failed)); + + self.vm.vm_init_failed().await; + let _ = self + .ensure_response_tx + .send(Err(VmError::InitializationFailed(reason.to_string()))); + + reason + } + + async fn initialize_vm_objects_from_spec( + &self, + event_queue: &Arc, + ) -> anyhow::Result { + let properties = &self.ensure_request.properties; + let spec = &self.ensure_request.instance_spec; + let options = self.ensure_options; + + info!(self.log, "initializing new VM"; + "spec" => #?spec, + "properties" => #?properties, + "use_reservoir" => options.use_reservoir, + "bootrom" => %options.toml_config.bootrom.display()); + + let vmm_log = self.log.new(slog::o!("component" => "vmm")); + + // Set up the 'shell' instance into which the rest of this routine will + // add components. + let VersionedInstanceSpec::V0(v0_spec) = &spec; + let machine = build_instance( + &properties.vm_name(), + v0_spec, + options.use_reservoir, + vmm_log, + )?; + + let mut init = MachineInitializer { + log: self.log.clone(), + machine: &machine, + devices: Default::default(), + block_backends: Default::default(), + crucible_backends: Default::default(), + spec: v0_spec, + properties, + toml_config: &options.toml_config, + producer_registry: options.oximeter_registry.clone(), + state: MachineInitializerState::default(), + }; + + init.initialize_rom(options.toml_config.bootrom.as_path())?; + let chipset = init.initialize_chipset( + &(event_queue.clone() + as Arc), + )?; + + init.initialize_rtc(&chipset)?; + init.initialize_hpet()?; + + let com1 = Arc::new(init.initialize_uart(&chipset)?); + let ps2ctrl = init.initialize_ps2(&chipset)?; + init.initialize_qemu_debug_port()?; + init.initialize_qemu_pvpanic(properties.into())?; + init.initialize_network_devices(&chipset)?; + + #[cfg(not(feature = "omicron-build"))] + init.initialize_test_devices(&options.toml_config.devices)?; + #[cfg(feature = "omicron-build")] + info!( + self.log, + "`omicron-build` feature enabled, ignoring any test devices" + ); + + #[cfg(feature = "falcon")] + { + init.initialize_softnpu_ports(&chipset)?; + init.initialize_9pfs(&chipset)?; + } + + init.initialize_storage_devices(&chipset, options.nexus_client.clone()) + .await?; + + let ramfb = init.initialize_fwcfg(v0_spec.devices.board.cpus)?; + init.initialize_cpus()?; + let vcpu_tasks = Box::new(crate::vcpu_tasks::VcpuTasks::new( + &machine, + event_queue.clone() + as Arc, + self.log.new(slog::o!("component" => "vcpu_tasks")), + )?); + + let MachineInitializer { + devices, + block_backends, + crucible_backends, + .. + } = init; + + Ok(InputVmObjects { + instance_spec: v0_spec.clone(), + vcpu_tasks, + machine, + devices, + block_backends, + crucible_backends, + com1, + framebuffer: Some(ramfb), + ps2ctrl, + }) + } +} + +/// Represents an instance ensure request that has proceeded far enough to +/// create a set of VM objects, but that has not yet installed those objects as +/// an `ActiveVm` or notified the requestor that its request is complete. +pub(crate) struct VmEnsureObjectsCreated<'a> { + log: &'a slog::Logger, + vm: &'a Arc, + ensure_request: &'a InstanceSpecEnsureRequest, + ensure_options: &'a EnsureOptions, + ensure_response_tx: InstanceEnsureResponseTx, + state_publisher: &'a mut StatePublisher, + vm_objects: Arc, + input_queue: Arc, + kernel_vm_paused: bool, +} + +impl<'a> VmEnsureObjectsCreated<'a> { + /// Prepares the VM's CPUs for an incoming live migration by activating them + /// (at the kernel VM level) and then pausing the kernel VM. This must be + /// done before importing any state into these objects. + /// + /// # Panics + /// + /// Panics if called more than once on the same set of objects. + pub(crate) async fn prepare_for_migration(&mut self) { + assert!(!self.kernel_vm_paused); + let guard = self.vm_objects.lock_exclusive().await; + guard.reset_vcpus(); + guard.pause_kernel_vm(); + self.kernel_vm_paused = true; + } + + /// Uses this struct's VM objects to create a set of VM services, then + /// installs an active VM into the parent VM state machine and notifies the + /// ensure requester that its request is complete. + pub(crate) async fn ensure_active(self) -> VmEnsureActive<'a> { + let vm_services = VmServices::new( + self.log, + self.vm, + &self.vm_objects, + &self.ensure_request.properties, + self.ensure_options, + ) + .await; + + self.vm + .make_active( + self.log, + self.input_queue.clone(), + &self.vm_objects, + vm_services, + ) + .await; + + // The response channel may be closed if the client who asked to ensure + // the VM timed out or disconnected. This is OK; now that the VM is + // active, a new client can recover by reading the current instance + // state and using the state change API to send commands to the state + // driver. + let _ = self.ensure_response_tx.send(Ok(InstanceEnsureResponse { + migrate: self.ensure_request.migrate.as_ref().map(|req| { + InstanceMigrateInitiateResponse { + migration_id: req.migration_id, + } + }), + })); + + VmEnsureActive { + vm: self.vm, + state_publisher: self.state_publisher, + vm_objects: self.vm_objects, + input_queue: self.input_queue, + kernel_vm_paused: self.kernel_vm_paused, + } + } +} + +/// Describes a set of VM objects that are fully initialized and referred to by +/// the `ActiveVm` in a VM state machine, but for which a state driver loop has +/// not started yet. +pub(crate) struct VmEnsureActive<'a> { + vm: &'a Arc, + state_publisher: &'a mut StatePublisher, + vm_objects: Arc, + input_queue: Arc, + kernel_vm_paused: bool, +} + +impl<'a> VmEnsureActive<'a> { + pub(crate) fn vm_objects(&self) -> &Arc { + &self.vm_objects + } + + pub(crate) fn state_publisher(&mut self) -> &mut StatePublisher { + self.state_publisher + } + + pub(crate) async fn fail(mut self) { + // If a caller asked to prepare the VM objects for migration in the + // previous phase, make sure that operation is undone before the VM + // objects are torn down. + if self.kernel_vm_paused { + let guard = self.vm_objects.lock_exclusive().await; + guard.resume_kernel_vm(); + self.kernel_vm_paused = false; + } + + self.state_publisher + .update(ExternalStateUpdate::Instance(InstanceState::Failed)); + + // Since there are extant VM objects, move to the Rundown state. The VM + // will move to RundownComplete when the objects are finally dropped. + self.vm.set_rundown().await; + } + + /// Yields the VM objects and input queue for this VM so that they can be + /// used to start a state driver loop. + pub(super) fn into_inner(self) -> (Arc, Arc) { + (self.vm_objects, self.input_queue) + } +} diff --git a/bin/propolis-server/src/lib/vm/guest_event.rs b/bin/propolis-server/src/lib/vm/guest_event.rs new file mode 100644 index 000000000..b34135a30 --- /dev/null +++ b/bin/propolis-server/src/lib/vm/guest_event.rs @@ -0,0 +1,67 @@ +// This Source Code Form is subject to the terms of the Mozilla Public +// License, v. 2.0. If a copy of the MPL was not distributed with this +// file, You can obtain one at https://mozilla.org/MPL/2.0/. + +//! Types and traits for handling guest-emitted events on the VM state driver. + +use std::{collections::VecDeque, time::Duration}; + +/// An event raised by some component in the instance (e.g. a vCPU or the +/// chipset) that the state worker must handle. +/// +/// The vCPU-sourced events carry a time element (duration since VM boot) as +/// emitted by the kernel vmm. This is used to deduplicate events when all +/// vCPUs running in-kernel are kicked out for the suspend state. +#[derive(Clone, Copy, Debug, Eq, PartialEq)] +pub(super) enum GuestEvent { + /// Fired when the bhyve VM enters its halt state. + VcpuSuspendHalt(Duration), + /// Fired when the bhyve VM enters its reset state. + VcpuSuspendReset(Duration), + /// Fired when the bhyve VM resets due to a triple fault. The first element + /// identifies the vCPU that sent this notification. + VcpuSuspendTripleFault(i32, Duration), + /// Chipset signaled halt condition + ChipsetHalt, + /// Chipset signaled reboot condition + ChipsetReset, +} + +#[derive(Debug, Default)] +pub(super) struct GuestEventQueue { + queue: VecDeque, +} + +/// A sink for events raised by a VM's vCPU tasks. +pub(crate) trait VcpuEventHandler: Send + Sync { + fn suspend_halt_event(&self, when: Duration); + fn suspend_reset_event(&self, when: Duration); + fn suspend_triple_fault_event(&self, vcpu_id: i32, when: Duration); + fn unhandled_vm_exit( + &self, + vcpu_id: i32, + exit: propolis::exits::VmExitKind, + ); + fn io_error_event(&self, vcpu_id: i32, error: std::io::Error); +} + +/// A sink for events raised by a VM's chipset. +pub(crate) trait ChipsetEventHandler: Send + Sync { + fn chipset_halt(&self); + fn chipset_reset(&self); +} + +impl GuestEventQueue { + pub(super) fn enqueue(&mut self, event: GuestEvent) -> bool { + if !self.queue.iter().any(|ev| *ev == event) { + self.queue.push_back(event); + true + } else { + false + } + } + + pub(super) fn pop_front(&mut self) -> Option { + self.queue.pop_front() + } +} diff --git a/bin/propolis-server/src/lib/vm/mod.rs b/bin/propolis-server/src/lib/vm/mod.rs index 8d47392db..9b51a51d0 100644 --- a/bin/propolis-server/src/lib/vm/mod.rs +++ b/bin/propolis-server/src/lib/vm/mod.rs @@ -2,1181 +2,589 @@ // License, v. 2.0. If a copy of the MPL was not distributed with this // file, You can obtain one at https://mozilla.org/MPL/2.0/. -//! Implements the VM controller: the public interface to a single Propolis -//! instance. +//! Implements the [`Vm`] type, which encapsulates a single Propolis virtual +//! machine instance and provides a public interface thereto to the Propolis +//! Dropshot server. //! -//! The VM controller serves two purposes. First, it collects all of the objects -//! describing a single Propolis VM (the Propolis `Instance` itself, the -//! instance's spec, direct references to components in the instance, etc.). -//! Second, it records requests and events that affect how a VM moves through -//! the stages of its lifecycle, i.e. how and when it boots, reboots, migrates, -//! and stops. +//! The VM state machine looks like this: //! -//! Each VM controller has a single "state driver" thread that processes -//! requests and events recorded by its controller and acts on the underlying -//! Propolis instance to move the VM into the appropriate states. Doing this -//! work on a single thread ensures that a VM can only undergo one state change -//! at a time, that there are no races to start/pause/resume/halt a VM's -//! components, and that there is a single source of truth as to a VM's current -//! state (and as to the steps that are required to move it to a different -//! state). Operations like live migration that require components to pause and -//! resume coordinate directly with the state driver thread. +//! ```text +//! [NoVm] +//! | +//! | +//! v +//! +---- WaitingForInit <----+ +//! | | | +//! | | | +//! | v | +//! | Active | +//! | | | +//! | | | +//! | v | +//! +-------> Rundown | +//! | | | +//! | | | +//! | v | +//! +---> RundownComplete ----+ +//! ``` //! -//! The VM controller's public API allows a Propolis Dropshot server to query a -//! VM's current state, to ask to change that state, and to obtain references to -//! objects in a VM as needed to handle other requests made of the server (e.g. -//! requests to connect to an instance's serial console or to take a disk -//! snapshot). The controller also implements traits that allow a VM's -//! components to raise events for the state driver to process (e.g. a request -//! from a VM's chipset to reboot or halt the VM). - -use crate::migrate; - -use futures::{future::BoxFuture, stream::FuturesUnordered, StreamExt}; -use std::{ - collections::{BTreeMap, VecDeque}, - fmt::Debug, - net::SocketAddr, - pin::Pin, - sync::{Arc, Condvar, Mutex, MutexGuard, Weak}, - task::{Context, Poll}, - thread::JoinHandle, - time::Duration, -}; +//! In the happy case where new VMs always start successfully, this state +//! machine transitions as follows: +//! +//! - New state machines start in [`VmState::NoVm`]. +//! - A request to create a new VM moves to [`VmState::WaitingForInit`]. +//! - Once all of the VM's components are created, the VM moves to +//! [`VmState::Active`]. +//! - When the VM stops, the VM moves to [`VmState::Rundown`]. +//! - When all references to the VM's components are dropped, the VM moves to +//! [`VmState::RundownComplete`]. A request to create a new VM will move back +//! to `WaitingForInit`. +//! +//! In any state except `NoVm`, the state machine holds enough state to describe +//! the most recent VM known to the state machine, whether it is being created +//! (`WaitingForInit`), running (`Active`), or being torn down (`Rundown` and +//! `RundownComplete`). +//! +//! In the `Active` state, the VM wrapper holds an [`active::ActiveVm`] and +//! allows API-layer callers to obtain references to it. These callers use these +//! references to ask to change a VM's state or change its configuration. An +//! active VM holds a reference to a [`objects::VmObjects`] structure that +//! bundles up all of the Propolis components (kernel VM, devices, and backends) +//! that make up an instance and a spec that describes that instance; API-layer +//! callers may use this structure to read the instance's properties and query +//! component state, but cannot mutate the VM's structure this way. +//! +//! Requests to change a VM's state or configuration (and events from a running +//! guest that might change a VM's state, like an in-guest shutdown or reboot +//! request or a triple fault) are placed in an [input +//! queue](state_driver::InputQueue) that is serviced by a single "state driver" +//! task. When an instance stops, this task moves the state machine to the +//! `Rundown` state, which renders new API-layer callers unable to clone new +//! references to the VM's `VmObjects`. When all outstanding references to the +//! objects are dropped, the VM moves to the `RundownComplete` state, obtains +//! the final instance state from the (joined) state driver task, and publishes +//! that state. At that point the VM may be reinitialized. +//! +//! The VM state machine delegates VM creation to the state driver task. This +//! task can fail to initialize a VM in two ways: +//! +//! 1. It may fail to create all of the VM's component objects (e.g. due to +//! bad configuration or resource exhaustion). +//! 2. It may successfully create all of the VM's component objects, but then +//! fail to populate their initial state via live migration from another +//! instance. +//! +//! In the former case, where no VM objects are ever created, the state driver +//! moves the state machine directly from `WaitingForInit` to `RundownComplete`. +//! In the latter case, the driver moves to `Rundown` and allows `VmObjects` +//! teardown to drive the state machine to `RundownComplete`. -use anyhow::Context as AnyhowContext; +use std::{collections::BTreeMap, net::SocketAddr, sync::Arc}; + +use active::ActiveVm; use oximeter::types::ProducerRegistry; -use propolis::{ - hw::{ps2::ctrl::PS2Ctrl, qemu::ramfb::RamFb, uart::LpcUart}, - vmm::Machine, -}; use propolis_api_types::{ - instance_spec::VersionedInstanceSpec, - InstanceMigrateStatusResponse as ApiMigrateStatusResponse, - InstanceMigrationStatus as ApiMigrationStatus, InstanceProperties, - InstanceState as ApiInstanceState, - InstanceStateMonitorResponse as ApiMonitoredState, - InstanceStateRequested as ApiInstanceStateRequested, - MigrationState as ApiMigrationState, + instance_spec::{v0::InstanceSpecV0, VersionedInstanceSpec}, + InstanceEnsureResponse, InstanceMigrateStatusResponse, + InstanceMigrationStatus, InstanceProperties, InstanceSpecEnsureRequest, + InstanceSpecGetResponse, InstanceState, InstanceStateMonitorResponse, + MigrationState, }; -use slog::{debug, error, info, Logger}; -use thiserror::Error; -use tokio::io::{AsyncRead, AsyncWrite}; -use tokio::sync::oneshot; -use tokio_tungstenite::WebSocketStream; -use uuid::Uuid; - -use crate::{ - initializer::{ - build_instance, MachineInitializer, MachineInitializerState, - }, - migrate::{MigrateError, MigrateRole}, - serial::Serial, - server::{BlockBackendMap, CrucibleBackendMap, DeviceMap, StaticConfig}, - vm::request_queue::ExternalRequest, -}; - -use self::request_queue::{ExternalRequestQueue, RequestDeniedReason}; -pub use nexus_client::Client as NexusClient; - +use rfb::server::VncServer; +use slog::info; +use state_driver::StateDriverOutput; +use state_publisher::StatePublisher; +use tokio::sync::{oneshot, watch, RwLock, RwLockReadGuard}; + +use crate::{server::MetricsEndpointConfig, vnc::PropolisVncServer}; + +mod active; +pub(crate) mod ensure; +pub(crate) mod guest_event; +pub(crate) mod objects; mod request_queue; +mod services; mod state_driver; - -/// Minimum thread count for the Tokio runtime driving the VMM tasks +pub(crate) mod state_publisher; + +/// Maps component names to lifecycle trait objects that allow +/// components to be started, paused, resumed, and halted. +pub(crate) type DeviceMap = + BTreeMap>; + +/// Maps component names to block backend trait objects. +pub(crate) type BlockBackendMap = + BTreeMap>; + +/// Maps component names to Crucible backend objects. +pub(crate) type CrucibleBackendMap = + BTreeMap>; + +/// Type alias for the sender side of the channel that receives +/// externally-visible instance state updates. +type InstanceStateTx = watch::Sender; + +/// Type alias for the receiver side of the channel that receives +/// externally-visible instance state updates. +type InstanceStateRx = watch::Receiver; + +/// Type alias for the results sent by the state driver in response to a request +/// to change a Crucible backend's configuration. +pub(crate) type CrucibleReplaceResult = + Result; + +/// Type alias for the sender side of a channel that receives Crucible backend +/// reconfiguration results. +pub(crate) type CrucibleReplaceResultTx = + oneshot::Sender; + +/// Type alias for the sender side of a channel that receives the results of +/// instance-ensure API calls. +type InstanceEnsureResponseTx = + oneshot::Sender>; + +/// The minimum number of threads to spawn in the Tokio runtime that runs the +/// state driver and any other VM-related tasks. const VMM_MIN_RT_THREADS: usize = 8; -const VMM_BASE_RT_THREADS: usize = 4; - -#[derive(Debug, Error)] -pub enum VmControllerError { - #[error("The requested operation requires an active instance")] - InstanceNotActive, - - #[error("The instance has a pending request to halt")] - InstanceHaltPending, - - #[error("Instance is already a migration source")] - AlreadyMigrationSource, - - #[error("Cannot request state {0:?} while migration is in progress")] - InvalidRequestForMigrationSource(ApiInstanceStateRequested), - - #[error("A migration into this instance is in progress")] - MigrationTargetInProgress, - - #[error("Another live migration into this instance already occurred")] - MigrationTargetPreviouslyCompleted, - - #[error("The most recent attempt to migrate into this instance failed")] - MigrationTargetFailed, - - #[error("Can't migrate into a running instance")] - TooLateToBeMigrationTarget, - - #[error("Failed to queue requested state change: {0}")] - StateChangeRequestDenied(#[from] request_queue::RequestDeniedReason), - - #[error("Migration protocol error: {0:?}")] - MigrationProtocolError(#[from] MigrateError), - - #[error("Failed to start vCPU workers")] - VcpuWorkerCreationFailed(#[from] super::vcpu_tasks::VcpuTaskError), - - #[error("Failed to create state worker: {0}")] - StateWorkerCreationFailed(std::io::Error), -} - -impl From for dropshot::HttpError { - fn from(vm_error: VmControllerError) -> Self { - use dropshot::HttpError; - match vm_error { - VmControllerError::AlreadyMigrationSource - | VmControllerError::InvalidRequestForMigrationSource(_) - | VmControllerError::MigrationTargetInProgress - | VmControllerError::MigrationTargetFailed - | VmControllerError::TooLateToBeMigrationTarget - | VmControllerError::StateChangeRequestDenied(_) - | VmControllerError::InstanceNotActive - | VmControllerError::InstanceHaltPending - | VmControllerError::MigrationTargetPreviouslyCompleted => { - HttpError::for_status( - Some(format!("Instance operation failed: {}", vm_error)), - http::status::StatusCode::FORBIDDEN, - ) - } - VmControllerError::MigrationProtocolError(_) - | VmControllerError::VcpuWorkerCreationFailed(_) - | VmControllerError::StateWorkerCreationFailed(_) => { - HttpError::for_internal_error(format!( - "Instance operation failed: {}", - vm_error - )) - } - } - } -} -/// A collection of objects that describe an instance and references to that -/// instance and its components. -pub(crate) struct VmObjects { - /// The underlying Propolis `Machine` this controller is managing. - machine: Option, - - /// The instance properties supplied when this controller was created. - properties: InstanceProperties, - - /// The instance spec used to create this controller's VM. - spec: tokio::sync::Mutex, +/// When creating a new VM, add the VM's vCPU count to this value, then spawn +/// that many threads on its Tokio runtime or [`VMM_MIN_RT_THREADS`], whichever +/// is greater. +const VMM_BASE_RT_THREADS: usize = 4; - /// Map of the emulated devices associated with the VM - devices: DeviceMap, +/// Errors generated by the VM controller and its subcomponents. +#[derive(Debug, thiserror::Error)] +pub(crate) enum VmError { + #[error("VM operation result channel unexpectedly closed")] + ResultChannelClosed, - /// Map of the instance's active block backends. - block_backends: BlockBackendMap, + #[error("VM not created")] + NotCreated, - /// Map of the instance's active Crucible backends. - crucible_backends: CrucibleBackendMap, + #[error("VM is currently initializing")] + WaitingToInitialize, - /// A wrapper around the instance's first COM port, suitable for providing a - /// connection to a guest's serial console. - com1: Arc>, + #[error("VM already initialized")] + AlreadyInitialized, - /// An optional reference to the guest's framebuffer. - framebuffer: Option>, + #[error("VM is currently shutting down")] + RundownInProgress, - /// A reference to the guest's PS/2 controller. - ps2ctrl: Arc, + #[error("VM initialization failed: {0}")] + InitializationFailed(String), - /// A notification receiver to which the state worker publishes the most - /// recent instance state information. - monitor_rx: tokio::sync::watch::Receiver, -} + #[error("Forbidden state change")] + ForbiddenStateChange(#[from] request_queue::RequestDeniedReason), -/// A message sent from a live migration destination task to update the -/// externally visible state of the migration attempt. -#[derive(Clone, Copy, Debug)] -pub enum MigrateTargetCommand { - /// Update the externally-visible migration state. - UpdateState(ApiMigrationState), + #[error("Failed to initialize VM's tokio runtime")] + TokioRuntimeInitializationFailed(#[source] std::io::Error), } -/// A message sent from a live migration driver to the state worker, asking it -/// to act on source instance components on the task's behalf. -#[derive(Clone, Copy, Debug)] -pub enum MigrateSourceCommand { - /// Update the externally-visible migration state. - UpdateState(ApiMigrationState), - - /// Pause the instance's devices and CPUs. - Pause, -} - -/// A message sent from the state worker to the live migration driver in -/// response to a previous command. -#[derive(Debug)] -pub enum MigrateSourceResponse { - /// A request to pause completed with the attached result. - Pause(Result<(), std::io::Error>), -} - -/// An event raised by a migration task that must be handled by the state -/// worker. -#[derive(Debug)] -enum MigrateTaskEvent { - /// The task completed with the associated result. - TaskExited(Result<(), MigrateError>), - - /// The task sent a command requesting work. - Command(T), -} +/// The top-level VM wrapper type. +pub(crate) struct Vm { + /// Lock wrapper for the VM state machine's contents. + /// + /// Routines that need to read VM properties or obtain a `VmObjects` handle + /// acquire this lock shared. + /// + /// Routines that drive the VM state machine acquire this lock exclusively. + inner: RwLock, -/// An event raised by some component in the instance (e.g. a vCPU or the -/// chipset) that the state worker must handle. -/// -/// The vCPU-sourced events carry a time element (duration since VM boot) as -/// emitted by the kernel vmm. This is used to deduplicate events when all -/// vCPUs running in-kernel are kicked out for the suspend state. -#[derive(Clone, Copy, Debug, Eq, PartialEq)] -enum GuestEvent { - /// VM entered halt state - VcpuSuspendHalt(Duration), - /// VM entered reboot state - VcpuSuspendReset(Duration), - /// vCPU encounted triple-fault - VcpuSuspendTripleFault(i32, Duration), - /// Chipset signaled halt condition - ChipsetHalt, - /// Chipset signaled reboot condition - ChipsetReset, + /// A logger for this VM. + log: slog::Logger, } -/// Shared instance state guarded by the controller's state mutex. This state is -/// accessed from the controller API and the VM's state worker. -#[derive(Debug)] -struct SharedVmStateInner { - external_request_queue: ExternalRequestQueue, - - /// The state worker's queue of unprocessed events from guest devices. - guest_event_queue: VecDeque, - - /// The expected ID of the next live migration this instance will - /// participate in (either in or out). If this is `Some`, external callers - /// who query migration state will observe that a live migration is in - /// progress even if the state driver has yet to pick up the live migration - /// tasks from its queue. - pending_migration_id: Option<(Uuid, MigrateRole)>, -} +/// Holds a VM state machine and state driver task handle. +struct VmInner { + /// The VM's current state. + state: VmState, -impl SharedVmStateInner { - fn new(parent_log: &Logger) -> Self { - let queue_log = - parent_log.new(slog::o!("component" => "external_request_queue")); - Self { - external_request_queue: ExternalRequestQueue::new(queue_log), - guest_event_queue: VecDeque::new(), - pending_migration_id: None, - } - } + /// A handle to the VM's current state driver task, if it has one. + driver: Option>, } -#[derive(Debug)] -pub(crate) struct SharedVmState { - inner: Mutex, - cv: Condvar, -} +/// Describes a past or future VM and its properties. +struct VmDescription { + /// Records the VM's last externally-visible state. + external_state_rx: InstanceStateRx, -/// A VM controller: a wrapper around a Propolis instance that supplies the -/// functions needed for the Propolis server to implement its own API. -pub struct VmController { - /// A collection of objects that don't change once an instance is ensured: - /// the instance itself, a description of it, and convenience references to - /// some of its members (used to avoid rummaging through the instance's - /// inventory). - vm_objects: VmObjects, - - /// A wrapper for the runtime state of this instance, managed by the state - /// worker thread. This also serves as a sink for hardware events (e.g. from - /// vCPUs and the chipset), so it is wrapped in an Arc so that it can be - /// shared with those events' sources. - worker_state: Arc, - - /// A handle to the state worker thread for this instance. - worker_thread: Mutex< - Option>>, - >, - - /// This controller's logger. - log: Logger, - - /// The Tokio runtime in which VMM-related processing is to be handled. - /// - /// This includes things such as device emulation, (block) backend - /// processing, and migration workloads. It is held in an [Option] only to - /// facilitate runtime shutdown when the [VmController] is dropped. - vmm_runtime: Option, + /// The VM's API-level instance properties. + properties: InstanceProperties, - /// Migration source state persisted across multiple migration attempts. - migration_src_state: Mutex, + /// The VM's last-known instance specification. + spec: InstanceSpecV0, - /// A weak reference to this controller, suitable for upgrading and passing - /// to tasks the controller spawns. - this: Weak, + /// The runtime on which the VM's state driver is running (or on which it + /// ran). + tokio_rt: Option, } -impl SharedVmState { - fn new(parent_log: &Logger) -> Self { - Self { - inner: Mutex::new(SharedVmStateInner::new(parent_log)), - cv: Condvar::new(), - } - } - - fn queue_external_request( - &self, - request: ExternalRequest, - ) -> Result<(), RequestDeniedReason> { - let mut inner = self.inner.lock().unwrap(); - let result = inner.external_request_queue.try_queue(request); - if result.is_ok() { - self.cv.notify_one(); - } - result - } +/// The states in the VM state machine. See the module comment for more details. +#[allow(clippy::large_enum_variant)] +enum VmState { + /// This state machine has never held a VM. + NoVm, - fn wait_for_next_event(&self) -> StateDriverEvent { - let guard = self.inner.lock().unwrap(); - let mut guard = self - .cv - .wait_while(guard, |i| { - i.external_request_queue.is_empty() - && i.guest_event_queue.is_empty() - }) - .unwrap(); - - if let Some(guest_event) = guard.guest_event_queue.pop_front() { - StateDriverEvent::Guest(guest_event) - } else { - StateDriverEvent::External( - guard.external_request_queue.pop_front().unwrap(), - ) - } - } + /// A new state driver is attempting to initialize objects for a VM with the + /// ecnlosed description. + WaitingForInit(VmDescription), - /// Add a guest event to the queue, so long as it does not appear to be a - /// duplicate of an existing event. - fn enqueue_guest_event(&self, event: GuestEvent) { - let mut inner = self.inner.lock().unwrap(); - if !inner.guest_event_queue.iter().any(|ev| *ev == event) { - // Only queue event if nothing else in the queue is a direct match - inner.guest_event_queue.push_back(event); - self.cv.notify_one(); - } - } + /// The VM is active, and callers can obtain a handle to its objects. + Active(active::ActiveVm), - pub fn suspend_halt_event(&self, when: Duration) { - self.enqueue_guest_event(GuestEvent::VcpuSuspendHalt(when)); - } + /// The previous VM is shutting down, but its objects have not been fully + /// destroyed yet. + Rundown(VmDescription), - pub fn suspend_reset_event(&self, when: Duration) { - self.enqueue_guest_event(GuestEvent::VcpuSuspendReset(when)); - } - - pub fn suspend_triple_fault_event(&self, vcpu_id: i32, when: Duration) { - self.enqueue_guest_event(GuestEvent::VcpuSuspendTripleFault( - vcpu_id, when, - )); - } - - pub fn unhandled_vm_exit( - &self, - vcpu_id: i32, - exit: propolis::exits::VmExitKind, - ) { - panic!("vCPU {}: Unhandled VM exit: {:?}", vcpu_id, exit); - } - - pub fn io_error_event(&self, vcpu_id: i32, error: std::io::Error) { - panic!("vCPU {}: Unhandled vCPU error: {}", vcpu_id, error); - } - - pub fn clear_pending_migration(&self) { - let mut inner = self.inner.lock().unwrap(); - inner.pending_migration_id = None; - } -} - -/// Functions called by a Propolis chipset to notify another component that an -/// event occurred. -pub trait ChipsetEventHandler: Send + Sync { - fn chipset_halt(&self); - fn chipset_reset(&self); + /// The previous VM and its objects have been cleaned up. + RundownComplete(VmDescription), } -impl ChipsetEventHandler for SharedVmState { - fn chipset_halt(&self) { - self.enqueue_guest_event(GuestEvent::ChipsetHalt); - } - - fn chipset_reset(&self) { - self.enqueue_guest_event(GuestEvent::ChipsetReset); +impl std::fmt::Display for VmState { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!( + f, + "{}", + match self { + Self::NoVm => "NoVm", + Self::WaitingForInit(_) => "WaitingForInit", + Self::Active(_) => "Active", + Self::Rundown(_) => "Rundown", + Self::RundownComplete(_) => "RundownComplete", + } + ) } } -impl VmController { - #[allow(clippy::too_many_arguments)] - pub fn new( - instance_spec: VersionedInstanceSpec, - properties: InstanceProperties, - &StaticConfig { vm: ref toml_config, use_reservoir, .. }: &StaticConfig, - producer_registry: Option, - nexus_client: Option, - log: Logger, - stop_ch: oneshot::Sender<()>, - ) -> anyhow::Result> { - let vmm_rt = Self::spawn_runtime(&properties)?; - - // All subsequent work should be run under our VMM runtime - let _rt_guard = vmm_rt.enter(); - - let bootrom = &toml_config.bootrom; - info!(log, "initializing new VM"; - "spec" => #?instance_spec, - "properties" => #?properties, - "use_reservoir" => use_reservoir, - "bootrom" => %bootrom.display()); - - let vmm_log = log.new(slog::o!("component" => "vmm")); - - // Set up the 'shell' instance into which the rest of this routine will - // add components. - let VersionedInstanceSpec::V0(v0_spec) = &instance_spec; - let machine = build_instance( - &properties.vm_name(), - v0_spec, - use_reservoir, - vmm_log, - )?; - - // Create the state monitor channel and the worker state struct that - // depends on it. The state struct can then be passed to device - // initialization as an event sink. - let (monitor_tx, monitor_rx) = - tokio::sync::watch::channel(ApiMonitoredState { - gen: 0, - state: ApiInstanceState::Creating, - migration: ApiMigrateStatusResponse { - migration_in: None, - migration_out: None, - }, - }); - - let worker_state = Arc::new(SharedVmState::new(&log)); - - // Create and initialize devices in the new instance. - let mut init = MachineInitializer { - log: log.clone(), - machine: &machine, - devices: DeviceMap::new(), - block_backends: BlockBackendMap::new(), - crucible_backends: CrucibleBackendMap::new(), - spec: v0_spec, - properties: &properties, - toml_config, - producer_registry, - state: MachineInitializerState::default(), - }; +/// Parameters to an instance ensure operation. +pub(super) struct EnsureOptions { + /// A reference to the VM configuration specified in the config TOML passed + /// to this propolis-server process. + pub(super) toml_config: Arc, - init.initialize_rom(bootrom.as_path())?; - let chipset = init.initialize_chipset( - &(worker_state.clone() as Arc), - )?; - init.initialize_rtc(&chipset)?; - init.initialize_hpet()?; - - let com1 = Arc::new(init.initialize_uart(&chipset)?); - let ps2ctrl = init.initialize_ps2(&chipset)?; - init.initialize_qemu_debug_port()?; - init.initialize_qemu_pvpanic((&properties).into())?; - init.initialize_network_devices(&chipset)?; - - #[cfg(not(feature = "omicron-build"))] - init.initialize_test_devices(&toml_config.devices)?; - #[cfg(feature = "omicron-build")] - info!( - log, - "`omicron-build` feature enabled, ignoring any test devices" - ); + /// True if VMs should allocate memory from the kernel VMM reservoir. + pub(super) use_reservoir: bool, - #[cfg(feature = "falcon")] - init.initialize_softnpu_ports(&chipset)?; - #[cfg(feature = "falcon")] - init.initialize_9pfs(&chipset)?; - init.initialize_storage_devices(&chipset, nexus_client)?; - let ramfb = init.initialize_fwcfg(v0_spec.devices.board.cpus)?; - init.initialize_cpus()?; - let vcpu_tasks = super::vcpu_tasks::VcpuTasks::new( - &machine, - worker_state.clone(), - log.new(slog::o!("component" => "vcpu_tasks")), - )?; - - let MachineInitializer { - devices, - block_backends, - crucible_backends, - .. - } = init; - - // The instance is fully set up; pass it to the new controller. - let shared_state_for_worker = worker_state.clone(); - let rt_hdl = vmm_rt.handle().clone(); - let controller = Arc::new_cyclic(|this| Self { - vm_objects: VmObjects { - machine: Some(machine), - properties, - spec: tokio::sync::Mutex::new(instance_spec), - devices, - block_backends, - crucible_backends, - com1, - framebuffer: Some(ramfb), - ps2ctrl, - monitor_rx, - }, - worker_state, - worker_thread: Mutex::new(None), - migration_src_state: Default::default(), - log: log.new(slog::o!("component" => "vm_controller")), - vmm_runtime: Some(vmm_rt), - this: this.clone(), - }); - - // Now that the controller exists, launch the state worker that will - // drive state transitions for this instance. When the VM halts, the - // worker will exit and drop its reference to the controller. - let ctrl_for_worker = controller.clone(); - let log_for_worker = - log.new(slog::o!("component" => "vm_state_worker")); - let worker_thread = std::thread::Builder::new() - .name("vm_state_worker".to_string()) - .spawn(move || { - let driver = state_driver::StateDriver::new( - rt_hdl, - ctrl_for_worker, - shared_state_for_worker, - vcpu_tasks, - log_for_worker, - monitor_tx, - ); - - let monitor_tx = driver.run_state_worker(); - - // Signal back to the server state once the worker has exited. - let _ = stop_ch.send(()); - monitor_tx - }) - .map_err(VmControllerError::StateWorkerCreationFailed)?; - - *controller.worker_thread.lock().unwrap() = Some(worker_thread); - Ok(controller) - } + /// Configuration used to serve Oximeter metrics from this server. + pub(super) metrics_config: Option, - pub fn properties(&self) -> &InstanceProperties { - &self.vm_objects.properties - } + /// An Oximeter producer registry to pass to components that will emit + /// Oximeter metrics. + pub(super) oximeter_registry: Option, - pub fn machine(&self) -> &Machine { - // Unwrap safety: The machine is created when the controller is created - // and removed only when the controller is dropped. - self.vm_objects - .machine - .as_ref() - .expect("VM controller always has a valid machine") - } + /// A Nexus client handle to pass to components that can make upcalls to + /// Nexus. + pub(super) nexus_client: Option, - pub(crate) fn migration_src_state( - &self, - ) -> MutexGuard<'_, migrate::source::PersistentState> { - self.migration_src_state.lock().unwrap() - } - - pub async fn instance_spec( - &self, - ) -> tokio::sync::MutexGuard<'_, VersionedInstanceSpec> { - self.vm_objects.spec.lock().await - } + /// A reference to the process's VNC server, used to connect the server to + /// a new VM's framebuffer. + pub(super) vnc_server: Arc>, - pub fn com1(&self) -> &Arc> { - &self.vm_objects.com1 - } - - pub fn framebuffer(&self) -> Option<&Arc> { - self.vm_objects.framebuffer.as_ref() - } + /// The address of this Propolis process, used by the live migration + /// protocol to transfer serial console connections. + pub(super) local_server_addr: SocketAddr, +} - pub fn ps2ctrl(&self) -> &Arc { - &self.vm_objects.ps2ctrl +impl Vm { + /// Creates a new VM. + pub fn new(log: &slog::Logger) -> Arc { + let log = log.new(slog::o!("component" => "vm_wrapper")); + let inner = VmInner { state: VmState::NoVm, driver: None }; + Arc::new(Self { inner: RwLock::new(inner), log }) } - pub fn crucible_backends( + /// If the VM is `Active`, yields a shared lock guard with a reference to + /// the relevant `ActiveVm`. Returns `None` if there is no active VM. + pub(super) async fn active_vm( &self, - ) -> &BTreeMap> { - &self.vm_objects.crucible_backends - } - - pub fn log(&self) -> &Logger { - &self.log - } - pub fn rt_hdl(&self) -> &tokio::runtime::Handle { - self.vmm_runtime - .as_ref() - .expect("vmm_runtime is populated until VmController is dropped") - .handle() - } - - pub fn external_instance_state(&self) -> ApiInstanceState { - self.vm_objects.monitor_rx.borrow().state - } + ) -> Option> { + RwLockReadGuard::try_map(self.inner.read().await, |inner| { + if let VmState::Active(vm) = &inner.state { + Some(vm) + } else { + None + } + }) + .ok() + } + + /// Returns the state, properties, and instance spec for the instance most + /// recently wrapped by this `Vm`. + pub(super) async fn get(&self) -> Result { + let guard = self.inner.read().await; + match &guard.state { + // If no VM has ever been created, there's nothing to get. + VmState::NoVm => Err(VmError::NotCreated), + + // If the VM is active, pull the required data out of its objects. + VmState::Active(vm) => { + let spec = + vm.objects().lock_shared().await.instance_spec().clone(); + let state = vm.external_state_rx.borrow().clone(); + Ok(InstanceSpecGetResponse { + properties: vm.properties.clone(), + spec: VersionedInstanceSpec::V0(spec), + state: state.state, + }) + } - pub fn inject_nmi(&self) { - if let Some(machine) = &self.vm_objects.machine { - match machine.inject_nmi() { - Ok(_) => { - info!(self.log, "Sending NMI to instance"); - } - Err(e) => { - error!(self.log, "Could not send NMI to instance: {}", e); - } - }; + // If the VM is not active yet, or there is only a + // previously-run-down VM, return the state saved in the state + // machine. + VmState::WaitingForInit(vm) + | VmState::Rundown(vm) + | VmState::RundownComplete(vm) => Ok(InstanceSpecGetResponse { + properties: vm.properties.clone(), + state: vm.external_state_rx.borrow().state, + spec: VersionedInstanceSpec::V0(vm.spec.clone()), + }), } } - pub fn state_watcher( + /// Yields a handle to the most recent instance state receiver wrapped by + /// this `Vm`. + pub(super) async fn state_watcher( &self, - ) -> &tokio::sync::watch::Receiver { - &self.vm_objects.monitor_rx + ) -> Result { + let guard = self.inner.read().await; + match &guard.state { + VmState::NoVm => Err(VmError::NotCreated), + VmState::Active(vm) => Ok(vm.external_state_rx.clone()), + VmState::WaitingForInit(vm) + | VmState::Rundown(vm) + | VmState::RundownComplete(vm) => Ok(vm.external_state_rx.clone()), + } } - /// Asks to queue a request to start a source migration task for this VM. - /// The migration will have the supplied `migration_id` and will obtain its - /// connection to the target by calling `upgrade_fn` to obtain a future that - /// yields the necessary connection. + /// Moves this VM from the `WaitingForInit` state to the `Active` state, + /// creating an `ActiveVm` with the supplied input queue, VM objects, and VM + /// services. /// - /// This routine fails if the VM was not marked as a migration source or if - /// it has another pending request that precludes migration. Note that this - /// routine does not fail if the future returned from `upgrade_fn` fails to - /// produce a connection to the destination. + /// # Panics /// - /// On success, clients may query the instance's migration status to - /// determine how the migration has progressed. - pub fn request_migration_from< - T: AsyncRead + AsyncWrite + Unpin + Send + 'static, - >( - &self, - migration_id: Uuid, - conn: WebSocketStream, - protocol: crate::migrate::protocol::Protocol, - ) -> Result<(), VmControllerError> { - let mut inner = self.worker_state.inner.lock().unwrap(); - - // Check that the request can be enqueued before setting up the - // migration task. - if !inner.external_request_queue.migrate_as_source_will_enqueue()? { - return Ok(()); - } - - let migration_request = - self.launch_source_migration_task(migration_id, conn, protocol); - - // Unwrap is safe because the queue state was checked under the lock. - inner.external_request_queue.try_queue(migration_request).unwrap(); - self.worker_state.cv.notify_one(); - Ok(()) - } - - /// Launches a task that will execute a live migration out of this VM. - /// Returns a state change request message to queue to the state driver, - /// which will coordinate with this task to run the migration. - fn launch_source_migration_task< - T: AsyncRead + AsyncWrite + Unpin + Send + 'static, - >( - &self, - migration_id: Uuid, - conn: WebSocketStream, - protocol: crate::migrate::protocol::Protocol, - ) -> ExternalRequest { - let log_for_task = - self.log.new(slog::o!("component" => "migrate_source_task")); - let ctrl_for_task = self.this.upgrade().unwrap(); - let (start_tx, start_rx) = tokio::sync::oneshot::channel(); - let (command_tx, command_rx) = tokio::sync::mpsc::channel(1); - let (response_tx, response_rx) = tokio::sync::mpsc::channel(1); - - // The migration process uses async operations when communicating with - // the migration target. Run that work on the async runtime. - info!(self.log, "Launching migration source task"); - let task = self.rt_hdl().spawn(async move { - info!(log_for_task, "Waiting to be told to start"); - start_rx.await.unwrap(); - - info!(log_for_task, "Starting migration procedure"); - if let Err(e) = crate::migrate::source::migrate( - ctrl_for_task, - command_tx, - response_rx, - conn, - protocol, - ) - .await - { - error!(log_for_task, "Migration task failed: {}", e); - return Err(e); + /// Panics if the VM is not in the `WaitingForInit` state. + async fn make_active( + self: &Arc, + log: &slog::Logger, + state_driver_queue: Arc, + objects: &Arc, + services: services::VmServices, + ) { + info!(self.log, "installing active VM"); + let mut guard = self.inner.write().await; + let old = std::mem::replace(&mut guard.state, VmState::NoVm); + match old { + VmState::WaitingForInit(vm) => { + guard.state = VmState::Active(ActiveVm { + log: log.clone(), + state_driver_queue, + external_state_rx: vm.external_state_rx, + properties: vm.properties, + objects: objects.clone(), + services, + tokio_rt: vm.tokio_rt.expect("WaitingForInit has runtime"), + }); } - - Ok(()) - }); - - ExternalRequest::MigrateAsSource { - migration_id, - task, - start_tx, - command_rx, - response_tx, + state => unreachable!( + "only a starting VM's state worker calls make_active \ + (current state: {state})" + ), } } - /// Asks to queue a request to start a destination migration task for this - /// VM. The migration will have the supplied `migration_id` and will obtain - /// its connection to the source by calling `upgrade_fn` to obtain a future - /// that yields the necessary connection. + /// Moves this VM from the `WaitingForInit` state to the `RundownComplete` + /// state in response to an instance initialization failure. /// - /// This routine fails if the VM has already begun to run or if a previous - /// migration in was attempted (regardless of its outcome). Note that this - /// routine does not fail if the future returned from `upgrade_fn` - /// subsequently fails to produce a connection to the destination (though - /// the migration attempt will then fail). + /// The caller must ensure there are no active `VmObjects` that refer to + /// this VM. /// - /// On success, clients may query the instance's migration status to - /// determine how the migration has progressed. - pub fn request_migration_into< - T: AsyncRead + AsyncWrite + Unpin + Send + 'static, - >( - &self, - migration_id: Uuid, - conn: WebSocketStream, - local_addr: SocketAddr, - protocol: crate::migrate::protocol::Protocol, - ) -> Result<(), VmControllerError> { - let mut inner = self.worker_state.inner.lock().unwrap(); - if !inner.external_request_queue.migrate_as_target_will_enqueue()? { - return Ok(()); - } - - // Check that the request can be enqueued before setting up the - // migration task. - let migration_request = self.launch_target_migration_task( - migration_id, - conn, - local_addr, - protocol, - ); - - // Unwrap is safe because the queue state was checked under the lock. - inner.external_request_queue.try_queue(migration_request).unwrap(); - self.worker_state.cv.notify_one(); - Ok(()) - } - - /// Launches a task that will execute a live migration into this VM. - /// Returns a state change request message to queue to the state driver, - /// which will coordinate with this task to run the migration. - fn launch_target_migration_task< - T: AsyncRead + AsyncWrite + Unpin + Send + 'static, - >( - &self, - migration_id: Uuid, - conn: WebSocketStream, - local_addr: SocketAddr, - protocol: crate::migrate::protocol::Protocol, - ) -> ExternalRequest { - let log_for_task = - self.log.new(slog::o!("component" => "migrate_source_task")); - let ctrl_for_task = self.this.upgrade().unwrap(); - let (start_tx, start_rx) = tokio::sync::oneshot::channel(); - let (command_tx, command_rx) = tokio::sync::mpsc::channel(1); - - // The migration process uses async operations when communicating with - // the migration target. Run that work on the async runtime. - info!(self.log, "Launching migration target task"); - let task = self.rt_hdl().spawn(async move { - info!(log_for_task, "Waiting to be told to start"); - start_rx.await.unwrap(); - - info!(log_for_task, "Starting migration procedure"); - if let Err(e) = crate::migrate::destination::migrate( - ctrl_for_task, - command_tx, - conn, - local_addr, - protocol, - ) - .await - { - error!(log_for_task, "Migration task failed: {}", e); - return Err(e); - } - - Ok(()) - }); - - ExternalRequest::MigrateAsTarget { - migration_id, - task, - start_tx, - command_rx, - } - } - - /// Handles a request to change the wrapped instance's state. - pub fn put_state( - &self, - requested: ApiInstanceStateRequested, - ) -> Result<(), VmControllerError> { - info!(self.log(), "Requested state {:?} via API", requested); - - self.worker_state - .queue_external_request(match requested { - ApiInstanceStateRequested::Run => ExternalRequest::Start, - ApiInstanceStateRequested::Stop => ExternalRequest::Stop, - ApiInstanceStateRequested::Reboot => ExternalRequest::Reboot, - }) - .map_err(Into::into) - } - - pub fn migrate_status(&self) -> ApiMigrateStatusResponse { - let mut published = - self.vm_objects.monitor_rx.borrow().migration.clone(); - - // There's a window between the point where a request to migrate returns - // and the point where the state worker actually picks up the migration - // and publishes its state. To ensure that migrations are visible as - // soon as they're queued, pick up the queued migration (if there is - // one) and insert it into the output in the appropriate position. The - // state driver will consume the pending migration before actually - // executing it. - let inner = self.worker_state.inner.lock().unwrap(); - if let Some((id, role)) = inner.pending_migration_id { - match role { - MigrateRole::Destination => { - published.migration_in = Some(ApiMigrationStatus { - id, - state: ApiMigrationState::Sync, - }); - } - MigrateRole::Source => { - published.migration_out = Some(ApiMigrationStatus { - id, - state: ApiMigrationState::Sync, - }); - } + /// # Panics + /// + /// Panics if the VM is not in the `WaitingForInit` state. + async fn vm_init_failed(&self) { + let mut guard = self.inner.write().await; + let old = std::mem::replace(&mut guard.state, VmState::NoVm); + match old { + VmState::WaitingForInit(vm) => { + guard.state = VmState::RundownComplete(vm) } + state => unreachable!( + "start failures should only occur before an active VM is \ + installed (current state: {state})" + ), } - - published } - pub(crate) fn for_each_device( - &self, - mut func: impl FnMut(&str, &Arc), - ) { - for (name, dev) in self.vm_objects.devices.iter() { - func(name, dev); - } - } - - pub(crate) fn for_each_device_fallible( - &self, - mut func: F, - ) -> std::result::Result<(), E> - where - F: FnMut( - &str, - &Arc, - ) -> std::result::Result<(), E>, - { - for (name, dev) in self.vm_objects.devices.iter() { - func(name, dev)?; - } - Ok(()) - } - - pub(crate) fn device_by_name( - &self, - name: &String, - ) -> Option> { - self.vm_objects.devices.get(name).cloned() - } - - /// Spawn a Tokio runtime in which to run the VMM-related (device emulation, - /// block backends, etc) tasks for an instance. - pub(crate) fn spawn_runtime( - properties: &InstanceProperties, - ) -> anyhow::Result { - // For now, just base the runtime size on vCPU count - let thread_count = usize::max( - VMM_MIN_RT_THREADS, - VMM_BASE_RT_THREADS + properties.vcpus as usize, - ); - - tokio::runtime::Builder::new_multi_thread() - .thread_name("tokio-rt-vmm") - .worker_threads(thread_count) - .enable_all() - .build() - .context("spawning tokio runtime for VMM") - } -} - -impl Drop for VmController { - fn drop(&mut self) { - info!(self.log, "Dropping VM controller"); - let machine = self - .vm_objects - .machine - .take() - .expect("VM controller should have an instance at drop"); - - // Destroy the underlying kernel VMM resource - let hdl = machine.destroy(); - let _ = hdl.destroy(); - - // Detach block backends so they can do any final clean-up - debug!(self.log, "Detaching block backends"); - for backend in self.vm_objects.block_backends.values() { - let _ = backend.attachment().detach(); - } - - // A fully-initialized controller is kept alive in part by its worker - // thread, which owns the sender side of the controller's state-change - // notification channel. Since the controller is being dropped, the - // worker is gone, so reclaim the sender from it and use it to publish - // that the controller is being destroyed. - if let Some(thread) = self.worker_thread.lock().unwrap().take() { - let api_state = thread.join().unwrap(); - let old_state = api_state.borrow().clone(); - - // Preserve the instance's state if it failed so that clients can - // distinguish gracefully-stopped instances from failed instances. - if matches!(old_state.state, ApiInstanceState::Failed) { - return; - } + /// Moves this VM from the `Active` state to the `Rundown` state. + /// + /// This routine should only be called by the state driver. + /// + /// # Panics + /// + /// Panics if the VM is not in the `Active` state. + async fn set_rundown(&self) { + info!(self.log, "setting VM rundown"); + let services = { + let mut guard = self.inner.write().await; + let old = std::mem::replace(&mut guard.state, VmState::NoVm); + let vm = match old { + VmState::Active(vm) => vm, + state => panic!( + "VM should be active before being run down (current state: \ + {state})" + ), + }; - let gen = old_state.gen + 1; - let _ = api_state.send(ApiMonitoredState { - gen, - state: ApiInstanceState::Destroyed, - ..old_state + let spec = vm.objects().lock_shared().await.instance_spec().clone(); + let ActiveVm { external_state_rx, properties, tokio_rt, .. } = vm; + guard.state = VmState::Rundown(VmDescription { + external_state_rx, + properties, + spec, + tokio_rt: Some(tokio_rt), }); - } + vm.services + }; - // Tokio will be upset if the VMM runtime is implicitly shutdown (via - // drop) in blocking context. We avoid such troubles by doing an - // explicit background shutdown. - let rt = self.vmm_runtime.take().expect("vmm_runtime is populated"); - rt.shutdown_background(); + services.stop(&self.log).await; } -} - -/// An event that a VM's state driver must process. -#[derive(Debug)] -enum StateDriverEvent { - /// An event that was raised from within the guest. - Guest(GuestEvent), - /// An event that was raised by an external entity (e.g. an API call to the - /// server). - External(ExternalRequest), -} - -/// Commands issued by the state driver back to its VM controller. These are -/// abstracted into a trait to allow them to be mocked out for testing without -/// having to supply mock implementations of the rest of the VM controller's -/// functionality. -#[cfg_attr(test, mockall::automock)] -trait StateDriverVmController { - /// Pause VM at the kernel VMM level, ensuring that in-kernel-emulated - /// devices and vCPUs are brought to a consistent state. + /// Moves this VM from the `Rundown` state to the `RundownComplete` state. /// - /// When the VM is paused, attempts to run its vCPUs (via `VM_RUN` ioctl) - /// will fail. A corresponding `resume_vm()` call must be made prior to - /// allowing vCPU tasks to run. - fn pause_vm(&self); - - /// Resume a previously-paused VM at the kernel VMM level. This will resume - /// any timers driving in-kernel-emulated devices, and allow the vCPU to run - /// again. - fn resume_vm(&self); - - /// Sends a reset request to each device in the instance, then sends a - /// reset command to the instance's bhyve VM. - fn reset_devices_and_machine(&self); - - /// Sends each device (and backend) a start request. - fn start_devices(&self) -> anyhow::Result<()>; - - /// Sends each device a pause request, then waits for all these requests to - /// complete. - fn pause_devices(&self); - - /// Sends each device a resume request. - fn resume_devices(&self); - - /// Sends each device (and backend) a halt request. - fn halt_devices(&self); - - /// Resets the state of each vCPU in the instance to its on-reboot state. - fn reset_vcpu_state(&self); -} - -impl StateDriverVmController for VmController { - fn pause_vm(&self) { - info!(self.log, "Pausing kernel VMM resources"); - self.machine().hdl.pause().expect("VM_PAUSE should succeed") - } - - fn resume_vm(&self) { - info!(self.log, "Resuming kernel VMM resources"); - self.machine().hdl.resume().expect("VM_RESUME should succeed") - } + /// This routine should only be called when dropping VM objects. + /// + /// # Panics + /// + /// Panics if the VM is not in the `Rundown` state. + async fn complete_rundown(&self) { + info!(self.log, "completing VM rundown"); + let mut guard = self.inner.write().await; + let old = std::mem::replace(&mut guard.state, VmState::NoVm); + let rt = match old { + VmState::Rundown(mut vm) => { + let rt = vm.tokio_rt.take().expect("rundown VM has a runtime"); + guard.state = VmState::RundownComplete(vm); + rt + } + state => unreachable!( + "VM rundown completed from invalid prior state {state}" + ), + }; - fn reset_devices_and_machine(&self) { - let _rtguard = self.rt_hdl().enter(); - self.for_each_device(|name, dev| { - info!(self.log, "Sending reset request to {}", name); - dev.reset(); - }); + let StateDriverOutput { mut state_publisher, final_state } = guard + .driver + .take() + .expect("driver must exist in rundown") + .await + .expect("state driver shouldn't panic"); - self.machine().reinitialize().unwrap(); - } + state_publisher.update(state_publisher::ExternalStateUpdate::Instance( + final_state, + )); - fn start_devices(&self) -> anyhow::Result<()> { - let _rtguard = self.rt_hdl().enter(); - self.for_each_device_fallible(|name, dev| { - info!(self.log, "Sending startup complete to {}", name); - let res = dev.start(); - if let Err(e) = &res { - error!(self.log, "Startup failed for {}: {:?}", name, e); - } - res - })?; - for (name, backend) in self.vm_objects.block_backends.iter() { - debug!(self.log, "Starting block backend {}", name); - let res = backend.start(); - if let Err(e) = &res { - error!(self.log, "Startup failed for {}: {:?}", name, e); - return res; - } - } - Ok(()) + // Shut down the runtime without blocking to wait for tasks to complete + // (since blocking is illegal in an async context). + // + // This must happen after the state driver task has successfully joined + // (otherwise it might be canceled and will fail to yield the VM's final + // state). + rt.shutdown_background(); } - fn pause_devices(&self) { - let _rtguard = self.rt_hdl().enter(); - self.for_each_device(|name, dev| { - info!(self.log, "Sending pause request to {}", name); - dev.pause(); - }); - - // Create a Future that returns the name of the device that has finished - // pausing: this allows keeping track of which devices have and haven't - // completed pausing yet. - struct NamedFuture { - name: String, - future: BoxFuture<'static, ()>, - } - - impl std::future::Future for NamedFuture { - type Output = String; - - fn poll( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll { - let mut_self = self.get_mut(); - match Pin::new(&mut mut_self.future).poll(cx) { - Poll::Pending => Poll::Pending, - Poll::Ready(()) => Poll::Ready(mut_self.name.clone()), - } - } - } + /// Attempts to move this VM to the `Active` state by setting up a state + /// driver task and directing it to initialize a new VM. + pub(crate) async fn ensure( + self: &Arc, + log: &slog::Logger, + ensure_request: InstanceSpecEnsureRequest, + options: EnsureOptions, + ) -> Result { + let log_for_driver = + log.new(slog::o!("component" => "vm_state_driver")); + + // This routine will create a state driver task that actually + // initializes the VM. The external instance-ensure API shouldn't return + // until that task has disposed of the initialization request. Create a + // channel to allow the state driver task to send back an ensure result + // at the appropriate moment. + let (ensure_reply_tx, ensure_rx) = oneshot::channel(); + + // The external state receiver needs to exist as soon as this routine + // returns, so create the appropriate channel here. The sender side of + // the channel will move to the state driver task. + let (external_publisher, external_rx) = StatePublisher::new( + &log_for_driver, + InstanceStateMonitorResponse { + gen: 1, + state: if ensure_request.migrate.is_some() { + InstanceState::Migrating + } else { + InstanceState::Creating + }, + migration: InstanceMigrateStatusResponse { + migration_in: ensure_request.migrate.as_ref().map(|req| { + InstanceMigrationStatus { + id: req.migration_id, + state: MigrationState::Sync, + } + }), + migration_out: None, + }, + }, + ); - info!(self.log, "Waiting for devices to pause"); - self.rt_hdl().block_on(async { - let mut stream: FuturesUnordered<_> = self - .vm_objects - .devices - .iter() - .map(|(name, dev)| { - info!(self.log, "Got paused future from dev {}", name); - NamedFuture { name: name.to_string(), future: dev.paused() } - }) - .collect(); - - loop { - match stream.next().await { - Some(name) => { - info!(self.log, "dev {} completed pause", name); - } - - None => { - // done - info!(self.log, "all devices paused"); - break; - } + // Take the lock for writing, since in the common case this call will be + // creating a new VM and there's no easy way to upgrade from a reader + // lock to a writer lock. + { + let mut guard = self.inner.write().await; + match guard.state { + VmState::WaitingForInit(_) => { + return Err(VmError::WaitingToInitialize); } - } - }); - } + VmState::Active(_) => return Err(VmError::AlreadyInitialized), + VmState::Rundown(_) => return Err(VmError::RundownInProgress), + _ => {} + }; - fn resume_devices(&self) { - let _rtguard = self.rt_hdl().enter(); - self.for_each_device(|name, dev| { - info!(self.log, "Sending resume request to {}", name); - dev.resume(); - }); - } + let VersionedInstanceSpec::V0(v0_spec) = + ensure_request.instance_spec.clone(); + + let thread_count = usize::max( + VMM_MIN_RT_THREADS, + VMM_BASE_RT_THREADS + v0_spec.devices.board.cpus as usize, + ); + + let tokio_rt = tokio::runtime::Builder::new_multi_thread() + .thread_name("tokio-rt-vmm") + .worker_threads(thread_count) + .enable_all() + .build() + .map_err(VmError::TokioRuntimeInitializationFailed)?; + + let properties = ensure_request.properties.clone(); + let vm_for_driver = self.clone(); + guard.driver = Some(tokio_rt.spawn(async move { + state_driver::run_state_driver( + log_for_driver, + vm_for_driver, + external_publisher, + ensure_request, + ensure_reply_tx, + options, + ) + .await + })); - fn halt_devices(&self) { - let _rtguard = self.rt_hdl().enter(); - self.for_each_device(|name, dev| { - info!(self.log, "Sending halt request to {}", name); - dev.halt(); - }); - for (name, backend) in self.vm_objects.block_backends.iter() { - debug!(self.log, "Stopping and detaching block backend {}", name); - backend.stop(); - if let Err(err) = backend.detach() { - error!( - self.log, - "Error while detaching block backend {name}: {err:?}", - ); - } + guard.state = VmState::WaitingForInit(VmDescription { + external_state_rx: external_rx.clone(), + properties, + spec: v0_spec, + tokio_rt: Some(tokio_rt), + }); } - } - fn reset_vcpu_state(&self) { - for vcpu in self.machine().vcpus.iter() { - info!(self.log, "Resetting vCPU {}", vcpu.id); - vcpu.activate().unwrap(); - vcpu.reboot_state().unwrap(); - if vcpu.is_bsp() { - info!(self.log, "Resetting BSP vCPU {}", vcpu.id); - vcpu.set_run_state(propolis::bhyve_api::VRS_RUN, None).unwrap(); - vcpu.set_reg( - propolis::bhyve_api::vm_reg_name::VM_REG_GUEST_RIP, - 0xfff0, - ) - .unwrap(); - } - } + // Wait for the state driver task to dispose of this request. + ensure_rx.await.map_err(|_| VmError::ResultChannelClosed)? } } diff --git a/bin/propolis-server/src/lib/vm/objects.rs b/bin/propolis-server/src/lib/vm/objects.rs new file mode 100644 index 000000000..8fff5bbb9 --- /dev/null +++ b/bin/propolis-server/src/lib/vm/objects.rs @@ -0,0 +1,475 @@ +// This Source Code Form is subject to the terms of the Mozilla Public +// License, v. 2.0. If a copy of the MPL was not distributed with this +// file, You can obtain one at https://mozilla.org/MPL/2.0/. + +//! A collection of all of the components that make up a Propolis VM instance. + +use std::{ + ops::{Deref, DerefMut}, + pin::Pin, + sync::Arc, + task::{Context, Poll}, +}; + +use futures::{future::BoxFuture, stream::FuturesUnordered, StreamExt}; +use propolis::{ + hw::{ps2::ctrl::PS2Ctrl, qemu::ramfb::RamFb, uart::LpcUart}, + vmm::VmmHdl, + Machine, +}; +use propolis_api_types::instance_spec::v0::InstanceSpecV0; +use slog::{error, info}; +use tokio::sync::{RwLock, RwLockReadGuard, RwLockWriteGuard}; + +use crate::{serial::Serial, vcpu_tasks::VcpuTaskController}; + +use super::{ + state_driver::VmStartReason, BlockBackendMap, CrucibleBackendMap, DeviceMap, +}; + +/// A collection of components that make up a Propolis VM instance. +pub(crate) struct VmObjects { + /// A reference to the VM state machine that created these objects. Used to + /// complete rundown when the objects are dropped. + parent: Arc, + + /// Synchronizes access to the VM's objects. + /// + /// API-layer callers that want to enumerate a VM's devices or read its spec + /// acquire this lock shared. The state driver acquires this lock exclusive + /// to mutate the VM. + inner: RwLock, +} + +/// A collection of objects that should eventually be wrapped in a lock and +/// stored in a `VmObjects` structure. See [`VmObjectsLocked`]. +pub(super) struct InputVmObjects { + pub instance_spec: InstanceSpecV0, + pub vcpu_tasks: Box, + pub machine: Machine, + pub devices: DeviceMap, + pub block_backends: BlockBackendMap, + pub crucible_backends: CrucibleBackendMap, + pub com1: Arc>, + pub framebuffer: Option>, + pub ps2ctrl: Arc, +} + +/// The collection of objects and state that make up a Propolis instance. +pub(crate) struct VmObjectsLocked { + /// The objects' associated logger. + log: slog::Logger, + + /// The instance spec that describes this collection of objects. + instance_spec: InstanceSpecV0, + + /// The set of tasks that run this VM's vCPUs. + vcpu_tasks: Box, + + /// The Propolis kernel VMM for this instance. + machine: Machine, + + /// Maps from component names to the trait objects that implement lifecycle + /// operations (e.g. pause and resume) for eligible components. + devices: DeviceMap, + + /// Maps from component names to trait objects that implement the block + /// storage backend trait. + block_backends: BlockBackendMap, + + /// Maps from component names to Crucible backend objects. + crucible_backends: CrucibleBackendMap, + + /// A handle to the serial console connection to the VM's first COM port. + com1: Arc>, + + /// A handle to the VM's framebuffer. + framebuffer: Option>, + + /// A handle to the VM's PS/2 controller. + ps2ctrl: Arc, +} + +impl VmObjects { + /// Creates a new VM object container. + pub(super) fn new( + log: slog::Logger, + parent: Arc, + input: InputVmObjects, + ) -> Self { + let inner = VmObjectsLocked::new(&log, input); + Self { parent, inner: tokio::sync::RwLock::new(inner) } + } + + /// Yields a shared lock guard referring to the underlying object + /// collection. + pub(crate) async fn lock_shared(&self) -> VmObjectsShared { + VmObjectsShared(self.inner.read().await) + } + + /// Yields an exclusive lock guard referring to the underlying object + /// collection. + pub(crate) async fn lock_exclusive(&self) -> VmObjectsExclusive { + VmObjectsExclusive(self.inner.write().await) + } +} + +impl VmObjectsLocked { + /// Associates a collection of VM objects with a logger. + fn new(log: &slog::Logger, input: InputVmObjects) -> Self { + Self { + log: log.clone(), + instance_spec: input.instance_spec, + vcpu_tasks: input.vcpu_tasks, + machine: input.machine, + devices: input.devices, + block_backends: input.block_backends, + crucible_backends: input.crucible_backends, + com1: input.com1, + framebuffer: input.framebuffer, + ps2ctrl: input.ps2ctrl, + } + } + + /// Yields the VM's current instance spec. + pub(crate) fn instance_spec(&self) -> &InstanceSpecV0 { + &self.instance_spec + } + + /// Yields a mutable reference to the VM's current instance spec. + pub(crate) fn instance_spec_mut(&mut self) -> &mut InstanceSpecV0 { + &mut self.instance_spec + } + + /// Yields the VM's current Propolis VM aggregation. + pub(crate) fn machine(&self) -> &Machine { + &self.machine + } + + /// Yields the VM's current kernel VMM handle. + pub(crate) fn vmm_hdl(&self) -> &Arc { + &self.machine.hdl + } + + /// Yields an accessor to the VM's memory context, or None if guest memory + /// is not currently accessible. + pub(crate) fn access_mem( + &self, + ) -> Option> { + self.machine.acc_mem.access() + } + + /// Obtains a handle to the lifecycle trait object for the component with + /// the supplied `name`. + pub(crate) fn device_by_name( + &self, + name: &str, + ) -> Option> { + self.devices.get(name).cloned() + } + + /// Yields the VM's current Crucible backend map. + pub(crate) fn crucible_backends(&self) -> &CrucibleBackendMap { + &self.crucible_backends + } + + /// Yields a clonable reference to the serial console for this VM's first + /// COM port. + pub(crate) fn com1(&self) -> &Arc> { + &self.com1 + } + + /// Yields a clonable reference to this VM's framebuffer. + pub(crate) fn framebuffer(&self) -> &Option> { + &self.framebuffer + } + + /// Yields a clonable reference to this VM's PS/2 controller. + pub(crate) fn ps2ctrl(&self) -> &Arc { + &self.ps2ctrl + } + + /// Iterates over all of the lifecycle trait objects in this VM and calls + /// `func` on each one. + pub(crate) fn for_each_device( + &self, + mut func: impl FnMut(&str, &Arc), + ) { + for (name, dev) in self.devices.iter() { + func(name, dev); + } + } + + /// Iterates over all of the lifecycle objects in this VM and calls `func` + /// on each one. If any invocation of `func` fails, this routine returns + /// immediately and yields the relevant error. + pub(crate) fn for_each_device_fallible( + &self, + mut func: impl FnMut( + &str, + &Arc, + ) -> std::result::Result<(), E>, + ) -> std::result::Result<(), E> { + for (name, dev) in self.devices.iter() { + func(name, dev)?; + } + + Ok(()) + } + + /// Pauses the VM at the kernel VMM level, ensuring that in-kernel-emulated + /// devices and vCPUs are brought to a consistent state. + /// + /// When the VM is paused, attempts to run its vCPUs (via `VM_RUN` ioctl) + /// will fail. A corresponding `resume_vm()` call must be made prior to + /// allowing vCPU tasks to run. + pub(super) fn pause_kernel_vm(&self) { + info!(self.log, "pausing kernel VMM resources"); + self.machine.hdl.pause().expect("VM_PAUSE should succeed"); + } + + /// Resumes the VM at the kernel VMM level. + pub(super) fn resume_kernel_vm(&self) { + info!(self.log, "resuming kernel VMM resources"); + self.machine.hdl.resume().expect("VM_RESUME should succeed"); + } + + /// Reinitializes the VM by resetting all of its devices and its kernel VMM. + pub(super) fn reset_devices_and_machine(&self) { + self.for_each_device(|name, dev| { + info!(self.log, "sending reset request to {}", name); + dev.reset(); + }); + + self.machine.reinitialize().unwrap(); + } + + /// Starts a VM's devices and allows all of its vCPU tasks to run. + /// + /// This function may be called either after initializing a new VM from + /// scratch or after an inbound live migration. In the latter case, this + /// routine assumes that the caller initialized and activated the VM's vCPUs + /// prior to importing state from the migration source. + pub(super) async fn start( + &mut self, + reason: VmStartReason, + ) -> anyhow::Result<()> { + match reason { + VmStartReason::ExplicitRequest => { + self.reset_vcpus(); + } + VmStartReason::MigratedIn => { + self.resume_kernel_vm(); + } + } + + let result = self.start_devices().await; + if result.is_ok() { + self.vcpu_tasks.resume_all(); + } + + result + } + + /// Pauses this VM's devices and its kernel VMM. + pub(crate) async fn pause(&mut self) { + self.vcpu_tasks.pause_all(); + self.pause_devices().await; + self.pause_kernel_vm(); + } + + /// Resumes this VM's devices and its kernel VMM. + pub(crate) fn resume(&mut self) { + self.resume_kernel_vm(); + self.resume_devices(); + self.vcpu_tasks.resume_all(); + } + + /// Stops the VM's vCPU tasks and devices. + pub(super) async fn halt(&mut self) { + self.vcpu_tasks.exit_all(); + self.halt_devices().await; + } + + /// Resets the VM's kernel vCPU state. + pub(super) fn reset_vcpus(&self) { + self.vcpu_tasks.new_generation(); + self.reset_vcpu_state(); + } + + /// Hard-resets a VM by pausing, resetting, and resuming all its devices and + /// vCPUs. + pub(super) async fn reboot(&mut self) { + // Reboot is implemented as a pause -> reset -> resume transition. + // + // First, pause the vCPUs and all devices so no partially-completed + // work is present. + self.vcpu_tasks.pause_all(); + self.pause_devices().await; + + // Reset all entities and the VM's bhyve state, then reset the + // vCPUs. The vCPU reset must come after the bhyve reset. + self.reset_devices_and_machine(); + self.reset_vcpus(); + + // Resume devices so they're ready to do more work, then resume + // vCPUs. + self.resume_devices(); + self.vcpu_tasks.resume_all(); + } + + /// Starts all of a VM's devices and allows its block backends to process + /// requests from their devices. + async fn start_devices(&self) -> anyhow::Result<()> { + self.for_each_device_fallible(|name, dev| { + info!(self.log, "sending startup complete to {}", name); + let res = dev.start(); + if let Err(e) = &res { + error!(self.log, "startup failed for {}: {:?}", name, e); + } + res + })?; + + for (name, backend) in self.block_backends.iter() { + info!(self.log, "starting block backend {}", name); + let res = backend.start().await; + if let Err(e) = &res { + error!(self.log, "startup failed for {}: {:?}", name, e); + return res; + } + } + + Ok(()) + } + + /// Pauses all of a VM's devices. + async fn pause_devices(&self) { + self.for_each_device(|name, dev| { + info!(self.log, "sending pause request to {}", name); + dev.pause(); + }); + + struct NamedFuture { + name: String, + future: BoxFuture<'static, ()>, + } + + impl std::future::Future for NamedFuture { + type Output = String; + + fn poll( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll { + let mut_self = self.get_mut(); + Pin::new(&mut mut_self.future) + .poll(cx) + .map(|_| mut_self.name.clone()) + } + } + + info!(self.log, "waiting for devices to pause"); + let mut stream: FuturesUnordered<_> = self + .devices + .iter() + .map(|(name, dev)| { + info!(self.log, "got paused future from dev {}", name); + NamedFuture { name: name.clone(), future: dev.paused() } + }) + .collect(); + + while let Some(name) = stream.next().await { + info!(self.log, "dev {} completed pause", name); + } + + info!(self.log, "all devices paused"); + } + + /// Resumes all of a VM's devices. + fn resume_devices(&self) { + self.for_each_device(|name, dev| { + info!(self.log, "sending resume request to {}", name); + dev.resume(); + }) + } + + /// Stops all of a VM's devices and detaches its block backends from their + /// devices. + async fn halt_devices(&self) { + self.for_each_device(|name, dev| { + info!(self.log, "sending halt request to {}", name); + dev.halt(); + }); + + for (name, backend) in self.block_backends.iter() { + info!(self.log, "stopping and detaching block backend {}", name); + backend.stop().await; + if let Err(err) = backend.detach() { + error!(self.log, "error detaching block backend"; + "name" => name, + "error" => ?err); + } + } + } + + /// Resets a VM's kernel vCPU objects to their initial states. + fn reset_vcpu_state(&self) { + for vcpu in self.machine.vcpus.iter() { + info!(self.log, "resetting vCPU {}", vcpu.id); + vcpu.activate().unwrap(); + vcpu.reboot_state().unwrap(); + if vcpu.is_bsp() { + info!(self.log, "Resetting BSP vCPU {}", vcpu.id); + vcpu.set_run_state(propolis::bhyve_api::VRS_RUN, None).unwrap(); + vcpu.set_reg( + propolis::bhyve_api::vm_reg_name::VM_REG_GUEST_RIP, + 0xfff0, + ) + .unwrap(); + } + } + } +} + +impl Drop for VmObjects { + fn drop(&mut self) { + // Signal to these objects' owning VM that rundown has completed and a + // new VM can be created. + // + // It is always safe to complete rundown at this point because the state + // driver ensures that if it creates VM objects, then it will not drop + // them without first moving the VM to the Rundown state. + let parent = self.parent.clone(); + tokio::spawn(async move { + parent.complete_rundown().await; + }); + } +} + +/// A shared lock on the contents of a [`VmObjects`]. +pub(crate) struct VmObjectsShared<'o>(RwLockReadGuard<'o, VmObjectsLocked>); + +/// An exclusive lock on the contents of a [`VmObjects`]. +pub(crate) struct VmObjectsExclusive<'o>(RwLockWriteGuard<'o, VmObjectsLocked>); + +impl Deref for VmObjectsShared<'_> { + type Target = VmObjectsLocked; + + fn deref(&self) -> &Self::Target { + &self.0 + } +} + +impl Deref for VmObjectsExclusive<'_> { + type Target = VmObjectsLocked; + + fn deref(&self) -> &Self::Target { + &self.0 + } +} + +impl DerefMut for VmObjectsExclusive<'_> { + fn deref_mut(&mut self) -> &mut Self::Target { + &mut self.0 + } +} diff --git a/bin/propolis-server/src/lib/vm/request_queue.rs b/bin/propolis-server/src/lib/vm/request_queue.rs index 9d23faa26..56c630258 100644 --- a/bin/propolis-server/src/lib/vm/request_queue.rs +++ b/bin/propolis-server/src/lib/vm/request_queue.rs @@ -27,54 +27,40 @@ use slog::{debug, info, Logger}; use thiserror::Error; use uuid::Uuid; -use crate::migrate::MigrateError; +/// Wraps a [`dropshot::WebsocketConnection`] for inclusion in an +/// [`ExternalRequest`]. +// +// This newtype allows this module's tests (which want to verify queuing +// dispositions and don't care about request contents) to construct a +// `MigrateAsSource` request without having to conjure up a real websocket +// connection. +pub(crate) struct WebsocketConnection(Option); + +impl From for WebsocketConnection { + fn from(value: dropshot::WebsocketConnection) -> Self { + Self(Some(value)) + } +} -use super::{ - MigrateSourceCommand, MigrateSourceResponse, MigrateTargetCommand, -}; +impl WebsocketConnection { + /// Yields the wrapped [`dropshot::WebsocketConnection`]. + pub(crate) fn into_inner(self) -> dropshot::WebsocketConnection { + // Unwrapping is safe here because the only way an external consumer can + // get an instance of this wrapper is to use the From impl, which always + // wraps a `Some`. + self.0.unwrap() + } +} /// An external request made of a VM controller via the server API. Handled by /// the controller's state driver thread. -#[derive(Debug)] pub enum ExternalRequest { - /// Initializes the VM through live migration by running a - /// migration-destination task. - MigrateAsTarget { - /// The ID of the live migration to use when initializing. - migration_id: Uuid, - - /// A handle to the task that will execute the migration procedure. - task: tokio::task::JoinHandle>, - - /// The sender side of a one-shot channel that, when signaled, tells the - /// migration task to start its work. - start_tx: tokio::sync::oneshot::Sender<()>, - - /// A channel that receives commands from the migration task. - command_rx: tokio::sync::mpsc::Receiver, - }, - - /// Resets all the VM's devices and CPUs, then starts the VM. + /// Asks the state worker to start a brand-new VM (i.e. not one initialized + /// by live migration, which implicitly starts the VM). Start, /// Asks the state worker to start a migration-source task. - MigrateAsSource { - /// The ID of the live migration for which this VM will be the source. - migration_id: Uuid, - - /// A handle to the task that will execute the migration procedure. - task: tokio::task::JoinHandle>, - - /// The sender side of a one-shot channel that, when signaled, tells the - /// migration task to start its work. - start_tx: tokio::sync::oneshot::Sender<()>, - - /// A channel that receives commands from the migration task. - command_rx: tokio::sync::mpsc::Receiver, - - /// A channel used to send responses to migration commands. - response_tx: tokio::sync::mpsc::Sender, - }, + MigrateAsSource { migration_id: Uuid, websock: WebsocketConnection }, /// Resets the guest by pausing all devices, resetting them to their /// cold-boot states, and resuming the devices. Note that this is not a @@ -84,6 +70,48 @@ pub enum ExternalRequest { /// Halts the VM. Note that this is not a graceful shutdown and does not /// coordinate with guest software. Stop, + + /// Attempts to update the volume construction request for the supplied + /// Crucible volume. + /// + /// TODO: Due to https://github.com/oxidecomputer/crucible/issues/871, this + /// is only allowed once the VM is started and the volume has activated, but + /// it should be allowed even before the VM has started. + ReconfigureCrucibleVolume { + /// The name of the Crucible backend component in the instance spec. + disk_name: String, + + /// The ID of the Crucible backend in the VM's Crucible backend map. + backend_id: Uuid, + + /// The new volume construction request to supply to the Crucible + /// upstairs. + new_vcr_json: String, + + /// The sink for the result of this operation. + result_tx: super::CrucibleReplaceResultTx, + }, +} + +impl std::fmt::Debug for ExternalRequest { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + Self::Start => write!(f, "Start"), + Self::MigrateAsSource { migration_id, .. } => f + .debug_struct("MigrateAsSource") + .field("migration_id", migration_id) + .finish(), + Self::Reboot => write!(f, "Reboot"), + Self::Stop => write!(f, "Stop"), + Self::ReconfigureCrucibleVolume { + disk_name, backend_id, .. + } => f + .debug_struct("ReconfigureCrucibleVolume") + .field("disk_name", disk_name) + .field("backend_id", backend_id) + .finish(), + } + } } /// A set of reasons why a request to queue an external state transition can @@ -93,9 +121,6 @@ pub enum RequestDeniedReason { #[error("Operation requires an active instance")] InstanceNotActive, - #[error("Already migrating into this instance")] - MigrationTargetInProgress, - #[error("Instance is currently starting")] StartInProgress, @@ -147,13 +172,14 @@ enum RequestDisposition { /// The current disposition for each kind of incoming request. #[derive(Copy, Clone, Debug)] struct AllowedRequests { - migrate_as_target: RequestDisposition, start: RequestDisposition, migrate_as_source: RequestDisposition, reboot: RequestDisposition, + mutate: RequestDisposition, stop: RequestDisposition, } +/// A queue for external requests to change an instance's state. #[derive(Debug)] pub struct ExternalRequestQueue { queue: VecDeque, @@ -161,20 +187,37 @@ pub struct ExternalRequestQueue { log: Logger, } +/// Indicates whether this queue's creator will start the relevant instance +/// without waiting for a Start request from the queue. +pub enum InstanceAutoStart { + Yes, + No, +} + impl ExternalRequestQueue { /// Creates a new queue that logs to the supplied logger. - pub fn new(log: Logger) -> Self { + pub fn new(log: Logger, auto_start: InstanceAutoStart) -> Self { + // If the queue is being created for an instance that will start + // automatically (e.g. due to a migration in), set the request + // disposition for future start requests to Ignore for idempotency. + let start = match auto_start { + InstanceAutoStart::Yes => RequestDisposition::Ignore, + InstanceAutoStart::No => RequestDisposition::Enqueue, + }; + Self { queue: VecDeque::new(), allowed: AllowedRequests { - migrate_as_target: RequestDisposition::Enqueue, - start: RequestDisposition::Enqueue, + start, migrate_as_source: RequestDisposition::Deny( RequestDeniedReason::InstanceNotActive, ), reboot: RequestDisposition::Deny( RequestDeniedReason::InstanceNotActive, ), + mutate: RequestDisposition::Deny( + RequestDeniedReason::InstanceNotActive, + ), stop: RequestDisposition::Enqueue, }, log, @@ -187,25 +230,26 @@ impl ExternalRequestQueue { } /// Indicates whether the queue is empty. + #[cfg(test)] pub fn is_empty(&self) -> bool { self.queue.is_empty() } - /// Asks to place the supplied request on the queue. If the requests is + /// Asks to place the supplied request on the queue. If the request is /// enqueued, updates the dispositions to use for future requests. pub fn try_queue( &mut self, request: ExternalRequest, ) -> Result<(), RequestDeniedReason> { let disposition = match request { - ExternalRequest::MigrateAsTarget { .. } => { - self.allowed.migrate_as_target - } ExternalRequest::Start => self.allowed.start, ExternalRequest::MigrateAsSource { .. } => { self.allowed.migrate_as_source } ExternalRequest::Reboot => self.allowed.reboot, + ExternalRequest::ReconfigureCrucibleVolume { .. } => { + self.allowed.mutate + } // Requests to stop always succeed. Note that a request to stop a VM // that hasn't started should still be queued to the state worker so @@ -237,51 +281,6 @@ impl ExternalRequestQueue { .get_new_dispositions(DispositionChangeReason::StateChange(state)); } - /// Indicates whether the queue would allow a request to migrate into this - /// instance. This can be used to avoid setting up migration tasks for - /// requests that will ultimately be denied. - /// - /// # Return value - /// - /// - `Ok(true)` if the request will be queued. - /// - `Ok(false)` if the request is allowed for idempotency reasons but will - /// not be queued. - /// - `Err` if the request is forbidden. - pub fn migrate_as_target_will_enqueue( - &self, - ) -> Result { - match self.allowed.migrate_as_target { - RequestDisposition::Enqueue => Ok(true), - RequestDisposition::Ignore => Ok(false), - RequestDisposition::Deny(reason) => Err(reason), - } - } - - /// Indicates whether the queue would allow a request to migrate out of this - /// instance. This can be used to avoid setting up migration tasks for - /// requests that will ultimately be denied. - /// - /// # Return value - /// - /// - `Ok(true)` if the request will be queued. - /// - `Ok(false)` if the request is allowed for idempotency reasons but will - /// not be queued. - /// - `Err` if the request is forbidden. - pub fn migrate_as_source_will_enqueue( - &self, - ) -> Result { - assert!(!matches!( - self.allowed.migrate_as_source, - RequestDisposition::Ignore - )); - - match self.allowed.migrate_as_source { - RequestDisposition::Enqueue => Ok(true), - RequestDisposition::Ignore => unreachable!(), - RequestDisposition::Deny(reason) => Err(reason), - } - } - /// Computes a new set of queue dispositions given the current state of the /// queue and the event that is changing those dispositions. fn get_new_dispositions( @@ -295,54 +294,26 @@ impl ExternalRequestQueue { use RequestDeniedReason as DenyReason; use RequestDisposition as Disposition; match reason { - // Starting the instance, whether via migration or cold boot, - // forecloses on further attempts to migrate in. For idempotency, - // further requests to start are allowed when an instance-starting - // transition is enqueued. - ChangeReason::ApiRequest(ExternalRequest::MigrateAsTarget { - .. - }) - | ChangeReason::ApiRequest(ExternalRequest::Start) => { - let (migrate_as_target_disposition, deny_reason) = match reason - { - // If this is a request to migrate in, make sure future - // requests to migrate in are handled idempotently. - ChangeReason::ApiRequest( - ExternalRequest::MigrateAsTarget { .. }, - ) => ( - Disposition::Ignore, - DenyReason::MigrationTargetInProgress, - ), - ChangeReason::ApiRequest(ExternalRequest::Start) => ( - Disposition::Deny(DenyReason::StartInProgress), - DenyReason::StartInProgress, - ), - _ => unreachable!(), - }; - + ChangeReason::ApiRequest(ExternalRequest::Start) => { + let reason = DenyReason::StartInProgress; AllowedRequests { - migrate_as_target: migrate_as_target_disposition, start: Disposition::Ignore, - migrate_as_source: Disposition::Deny(deny_reason), - reboot: Disposition::Deny(deny_reason), + migrate_as_source: Disposition::Deny(reason), + reboot: Disposition::Deny(reason), + mutate: Disposition::Deny(reason), stop: self.allowed.stop, } } ChangeReason::ApiRequest(ExternalRequest::MigrateAsSource { .. }) => { - assert!(matches!(self.allowed.start, Disposition::Ignore)); - - // Requests to migrate into the instance should not be enqueued - // from this point, but whether they're dropped or ignored - // depends on how the instance was originally initialized. - assert!(!matches!( - self.allowed.migrate_as_target, - Disposition::Enqueue - )); + assert!( + matches!(self.allowed.start, Disposition::Ignore), + "{:?}", + self.allowed + ); AllowedRequests { - migrate_as_target: self.allowed.migrate_as_target, start: self.allowed.start, migrate_as_source: Disposition::Deny( DenyReason::AlreadyMigrationSource, @@ -350,6 +321,9 @@ impl ExternalRequestQueue { reboot: Disposition::Deny( DenyReason::InvalidRequestForMigrationSource, ), + mutate: Disposition::Deny( + DenyReason::InvalidRequestForMigrationSource, + ), stop: self.allowed.stop, } } @@ -357,39 +331,41 @@ impl ExternalRequestQueue { // Requests to reboot prevent additional reboot requests from being // queued, but do not affect other operations. ChangeReason::ApiRequest(ExternalRequest::Reboot) => { - assert!(matches!(self.allowed.start, Disposition::Ignore)); - assert!(!matches!( - self.allowed.migrate_as_target, - Disposition::Enqueue - )); - + assert!( + matches!(self.allowed.start, Disposition::Ignore), + "{:?}", + self.allowed + ); AllowedRequests { reboot: Disposition::Ignore, ..self.allowed } } // Requests to stop the instance block other requests from being // queued. Additional requests to stop are ignored for idempotency. ChangeReason::ApiRequest(ExternalRequest::Stop) => { + let reason = DenyReason::HaltPending; AllowedRequests { - migrate_as_target: Disposition::Deny( - DenyReason::HaltPending, - ), - start: Disposition::Deny(DenyReason::HaltPending), - migrate_as_source: Disposition::Deny( - DenyReason::HaltPending, - ), - reboot: Disposition::Deny(DenyReason::HaltPending), + start: Disposition::Deny(reason), + migrate_as_source: Disposition::Deny(reason), + reboot: Disposition::Deny(reason), + mutate: Disposition::Deny(reason), stop: Disposition::Ignore, } } + // Requests to mutate VM configuration don't move the VM state + // machine and don't change any request dispositions. + ChangeReason::ApiRequest( + ExternalRequest::ReconfigureCrucibleVolume { .. }, + ) => self.allowed, + // When an instance begins running, requests to migrate out of it or // to reboot it become valid. ChangeReason::StateChange(InstanceStateChange::StartedRunning) => { AllowedRequests { - migrate_as_target: self.allowed.migrate_as_target, start: self.allowed.start, migrate_as_source: Disposition::Enqueue, reboot: Disposition::Enqueue, + mutate: Disposition::Enqueue, stop: self.allowed.stop, } } @@ -414,28 +390,22 @@ impl ExternalRequestQueue { // previous dispositions for migrate and reboot requests may not be // "deny". ChangeReason::StateChange(InstanceStateChange::Stopped) => { + let reason = DenyReason::InstanceNotActive; AllowedRequests { - migrate_as_target: Disposition::Deny( - DenyReason::InstanceNotActive, - ), - start: Disposition::Deny(DenyReason::InstanceNotActive), - migrate_as_source: Disposition::Deny( - DenyReason::InstanceNotActive, - ), - reboot: Disposition::Deny(DenyReason::InstanceNotActive), + start: Disposition::Deny(reason), + migrate_as_source: Disposition::Deny(reason), + reboot: Disposition::Deny(reason), + mutate: Disposition::Deny(reason), stop: Disposition::Ignore, } } ChangeReason::StateChange(InstanceStateChange::Failed) => { + let reason = DenyReason::InstanceFailed; AllowedRequests { - migrate_as_target: Disposition::Deny( - DenyReason::InstanceFailed, - ), - start: Disposition::Deny(DenyReason::InstanceFailed), - migrate_as_source: Disposition::Deny( - DenyReason::InstanceFailed, - ), - reboot: Disposition::Deny(DenyReason::InstanceFailed), + start: Disposition::Deny(reason), + migrate_as_source: Disposition::Deny(reason), + reboot: Disposition::Deny(reason), + mutate: Disposition::Deny(reason), stop: self.allowed.stop, } } @@ -443,6 +413,46 @@ impl ExternalRequestQueue { } } +// It's possible for an external request queue to be dropped with outstanding +// requests if an event from the guest shuts down the VM before the queue can be +// drained. If this happens, notify anyone waiting on a specific request on the +// queue that the VM is gone. +impl Drop for ExternalRequestQueue { + fn drop(&mut self) { + for req in self.queue.drain(..) { + match req { + // Crucible VCR change requestors wait for their requests to be + // retired. + ExternalRequest::ReconfigureCrucibleVolume { + result_tx, + .. + } => { + let _ = + result_tx.send(Err(dropshot::HttpError::for_status( + Some( + "VM destroyed before request could be handled" + .to_string(), + ), + http::StatusCode::GONE, + ))); + } + + // Requests to start, reboot, and stop are handled + // asynchronously (calls to change the instance's state return + // as soon as they're queued). + ExternalRequest::Start + | ExternalRequest::Reboot + | ExternalRequest::Stop => {} + + // Dropping a request to migrate out drops the embedded + // connection to the migration target, thus notifying it that + // the source is gone. + ExternalRequest::MigrateAsSource { .. } => {} + } + } + } +} + #[cfg(test)] mod test { use super::*; @@ -453,77 +463,52 @@ mod test { slog::Logger::root(slog::Discard, slog::o!()) } - fn make_migrate_as_target_request() -> ExternalRequest { - let task = tokio::task::spawn(async { Ok(()) }); - let (start_tx, _) = tokio::sync::oneshot::channel(); - let (_, command_rx) = tokio::sync::mpsc::channel(1); - ExternalRequest::MigrateAsTarget { + fn make_migrate_as_source_request() -> ExternalRequest { + ExternalRequest::MigrateAsSource { migration_id: Uuid::new_v4(), - task, - start_tx, - command_rx, + websock: WebsocketConnection(None), } } - fn make_migrate_as_source_request() -> ExternalRequest { - let task = tokio::task::spawn(async { Ok(()) }); - let (start_tx, _) = tokio::sync::oneshot::channel(); - let (_, command_rx) = tokio::sync::mpsc::channel(1); - let (response_tx, _) = tokio::sync::mpsc::channel(1); - ExternalRequest::MigrateAsSource { - migration_id: Uuid::new_v4(), - task, - start_tx, - command_rx, - response_tx, + fn make_reconfigure_crucible_request() -> ExternalRequest { + let (tx, _rx) = tokio::sync::oneshot::channel(); + ExternalRequest::ReconfigureCrucibleVolume { + disk_name: "".to_string(), + backend_id: Uuid::new_v4(), + new_vcr_json: "".to_string(), + result_tx: tx, } } #[tokio::test] - async fn migrate_as_target_is_idempotent() { - let mut queue = ExternalRequestQueue::new(test_logger()); + async fn start_requests_become_idempotent_after_first_request() { + let mut queue = + ExternalRequestQueue::new(test_logger(), InstanceAutoStart::No); - // Requests to migrate as a target should queue normally at first. - assert!(queue.migrate_as_target_will_enqueue().unwrap()); + // The first request to start should succeed. + assert!(queue.try_queue(ExternalRequest::Start).is_ok()); - // After queuing such a request, subsequent requests should be allowed - // without enqueuing anything. - assert!(queue.try_queue(make_migrate_as_target_request()).is_ok()); - assert!(!queue.migrate_as_target_will_enqueue().unwrap()); + // The second one should too, but only for idempotency: the queue should + // then have only one start request on it. + assert!(queue.try_queue(ExternalRequest::Start).is_ok()); + assert!(matches!(queue.pop_front(), Some(ExternalRequest::Start))); + assert!(queue.pop_front().is_none()); - // Pop the request and tell the queue the instance is running. - assert!(matches!( - queue.pop_front(), - Some(ExternalRequest::MigrateAsTarget { .. }) - )); + // Start requests continue to be ignored even after the instance starts + // to run. queue.notify_instance_state_change(InstanceStateChange::StartedRunning); - - // Because the instance was started via migration in, future requests - // to migrate in should be allowed. - assert!(queue.try_queue(make_migrate_as_target_request()).is_ok()); - assert!(!queue.migrate_as_target_will_enqueue().unwrap()); - } - - #[tokio::test] - async fn migrate_as_target_is_forbidden_after_cold_boot() { - let mut queue = ExternalRequestQueue::new(test_logger()); assert!(queue.try_queue(ExternalRequest::Start).is_ok()); - queue.notify_instance_state_change(InstanceStateChange::StartedRunning); - - assert!(queue.migrate_as_target_will_enqueue().is_err()); - assert!(queue.try_queue(make_migrate_as_target_request()).is_err()); + assert!(queue.pop_front().is_none()); } #[tokio::test] async fn migrate_as_source_is_not_idempotent() { // Simulate a running instance. - let mut queue = ExternalRequestQueue::new(test_logger()); - assert!(queue.try_queue(ExternalRequest::Start).is_ok()); - assert!(matches!(queue.pop_front(), Some(ExternalRequest::Start))); + let mut queue = + ExternalRequestQueue::new(test_logger(), InstanceAutoStart::Yes); queue.notify_instance_state_change(InstanceStateChange::StartedRunning); // Requests to migrate out should be allowed. - assert!(queue.migrate_as_source_will_enqueue().unwrap()); assert!(queue.try_queue(make_migrate_as_source_request()).is_ok()); // Once the request is queued, other requests to migrate out are @@ -534,7 +519,6 @@ mod test { // is assumed), but requests to migrate out are issued by the target // Propolis (which does not assume idempotency and issues only one // request per migration attempt). - assert!(queue.migrate_as_source_will_enqueue().is_err()); assert!(queue.try_queue(make_migrate_as_source_request()).is_err()); // If migration fails, the instance resumes running, and then another @@ -544,22 +528,19 @@ mod test { Some(ExternalRequest::MigrateAsSource { .. }) )); queue.notify_instance_state_change(InstanceStateChange::StartedRunning); - assert!(queue.migrate_as_source_will_enqueue().unwrap()); assert!(queue.try_queue(make_migrate_as_source_request()).is_ok()); // A successful migration stops the instance, which forecloses on future // requests to migrate out. queue.pop_front(); queue.notify_instance_state_change(InstanceStateChange::Stopped); - assert!(queue.migrate_as_source_will_enqueue().is_err()); assert!(queue.try_queue(make_migrate_as_source_request()).is_err()); } #[tokio::test] async fn stop_requests_enqueue_after_vm_failure() { - let mut queue = ExternalRequestQueue::new(test_logger()); - assert!(queue.try_queue(ExternalRequest::Start).is_ok()); - assert!(matches!(queue.pop_front(), Some(ExternalRequest::Start))); + let mut queue = + ExternalRequestQueue::new(test_logger(), InstanceAutoStart::No); queue.notify_instance_state_change(InstanceStateChange::Failed); assert!(queue.try_queue(ExternalRequest::Stop).is_ok()); @@ -568,9 +549,8 @@ mod test { #[tokio::test] async fn reboot_requests_are_idempotent_except_when_stopping() { - let mut queue = ExternalRequestQueue::new(test_logger()); - assert!(queue.try_queue(ExternalRequest::Start).is_ok()); - assert!(matches!(queue.pop_front(), Some(ExternalRequest::Start))); + let mut queue = + ExternalRequestQueue::new(test_logger(), InstanceAutoStart::Yes); queue.notify_instance_state_change(InstanceStateChange::StartedRunning); // Once the instance is started, reboot requests should be allowed, but @@ -601,4 +581,46 @@ mod test { queue.notify_instance_state_change(InstanceStateChange::Rebooted); assert!(queue.try_queue(ExternalRequest::Reboot).is_err()); } + + #[tokio::test] + async fn mutation_requires_running_and_not_migrating_out() { + let mut queue = + ExternalRequestQueue::new(test_logger(), InstanceAutoStart::No); + + // Mutating a VM before it has started is not allowed. + assert!(queue.try_queue(make_reconfigure_crucible_request()).is_err()); + + // Merely dequeuing the start request doesn't allow mutation; the VM + // actually has to be running. + assert!(queue.try_queue(ExternalRequest::Start).is_ok()); + assert!(matches!(queue.pop_front(), Some(ExternalRequest::Start))); + assert!(queue.try_queue(make_reconfigure_crucible_request()).is_err()); + queue.notify_instance_state_change(InstanceStateChange::StartedRunning); + assert!(queue.try_queue(make_reconfigure_crucible_request()).is_ok()); + assert!(matches!( + queue.pop_front(), + Some(ExternalRequest::ReconfigureCrucibleVolume { .. }) + )); + + // Successfully requesting migration out should block new mutation + // requests (they should wait for the migration to resolve and then go + // to the target). + assert!(queue.try_queue(make_migrate_as_source_request()).is_ok()); + assert!(queue.try_queue(make_reconfigure_crucible_request()).is_err()); + + // But if the VM resumes (due to a failed migration out) these requests + // should succeed again. + assert!(queue.pop_front().is_some()); + queue.notify_instance_state_change(InstanceStateChange::StartedRunning); + assert!(queue.try_queue(make_reconfigure_crucible_request()).is_ok()); + } + + #[tokio::test] + async fn mutation_disallowed_after_stop() { + let mut queue = + ExternalRequestQueue::new(test_logger(), InstanceAutoStart::Yes); + queue.notify_instance_state_change(InstanceStateChange::StartedRunning); + queue.notify_instance_state_change(InstanceStateChange::Stopped); + assert!(queue.try_queue(make_reconfigure_crucible_request()).is_err()); + } } diff --git a/bin/propolis-server/src/lib/vm/services.rs b/bin/propolis-server/src/lib/vm/services.rs new file mode 100644 index 000000000..095fa5050 --- /dev/null +++ b/bin/propolis-server/src/lib/vm/services.rs @@ -0,0 +1,207 @@ +// This Source Code Form is subject to the terms of the Mozilla Public +// License, v. 2.0. If a copy of the MPL was not distributed with this +// file, You can obtain one at https://mozilla.org/MPL/2.0/. + +//! Services visible to consumers outside this Propolis that depend on +//! functionality supplied by an extant VM. + +use std::sync::Arc; + +use oximeter::types::ProducerRegistry; +use propolis_api_types::InstanceProperties; +use rfb::server::VncServer; +use slog::{error, info, Logger}; + +use crate::{ + serial::SerialTaskControlMessage, server::MetricsEndpointConfig, + stats::virtual_machine::VirtualMachine, vnc::PropolisVncServer, +}; + +use super::objects::{VmObjects, VmObjectsShared}; + +/// Information used to serve Oximeter metrics. +#[derive(Default)] +pub(crate) struct OximeterState { + /// The Oximeter server to which Oximeter clients connect to query for + /// metrics. + server: Option, + + /// The statistics object used by the API layer to record its metrics. + pub stats: Option, +} + +/// A collection of services visible to consumers outside this Propolis that +/// depend on the functionality supplied by an extant VM. +pub(crate) struct VmServices { + /// A VM's serial console handler task. + pub serial_task: tokio::sync::Mutex>, + + /// A VM's Oximeter server. + pub oximeter: tokio::sync::Mutex, + + /// A reference to the VM's host process's VNC server. + pub vnc_server: Arc>, +} + +impl VmServices { + /// Starts a new set of VM services using the supplied VM objects and server + /// configuration. + pub(super) async fn new( + log: &slog::Logger, + vm: &Arc, + vm_objects: &VmObjects, + vm_properties: &InstanceProperties, + ensure_options: &super::EnsureOptions, + ) -> Self { + let oximeter_state = if let Some(cfg) = &ensure_options.metrics_config { + let registry = ensure_options.oximeter_registry.as_ref().expect( + "should have a producer registry if metrics are configured", + ); + + register_oximeter_producer(log, cfg, registry, vm_properties).await + } else { + OximeterState::default() + }; + + let vm_objects = vm_objects.lock_shared().await; + let vnc_server = ensure_options.vnc_server.clone(); + if let Some(ramfb) = vm_objects.framebuffer() { + vnc_server + .server + .initialize( + crate::vnc::RamFb::new(ramfb.get_framebuffer_spec()), + vm_objects.ps2ctrl().clone(), + vm.clone(), + ) + .await; + + let notifier_server_ref = vnc_server.clone(); + let rt = tokio::runtime::Handle::current(); + ramfb.set_notifier(Box::new(move |config, is_valid| { + let vnc = notifier_server_ref.clone(); + rt.block_on(vnc.server.update(config, is_valid, &vnc)); + })); + } + + let serial_task = start_serial_task(log, &vm_objects).await; + + Self { + serial_task: tokio::sync::Mutex::new(Some(serial_task)), + oximeter: tokio::sync::Mutex::new(oximeter_state), + vnc_server, + } + } + + /// Directs all the services in this service block to stop. + pub(super) async fn stop(&self, log: &Logger) { + self.vnc_server.stop().await; + + if let Some(serial_task) = self.serial_task.lock().await.take() { + let _ = serial_task + .control_ch + .send(SerialTaskControlMessage::Stopping) + .await; + let _ = serial_task.task.await; + } + + let mut oximeter_state = self.oximeter.lock().await; + if let Some(server) = oximeter_state.server.take() { + if let Err(e) = server.close().await { + error!(log, "failed to close oximeter producer server"; + "error" => ?e); + } + } + + let _ = oximeter_state.stats.take(); + } +} + +/// Creates an Oximeter producer and registers it with Oximeter, which will call +/// back into the server to gather the producer's metrics. +async fn register_oximeter_producer( + log: &slog::Logger, + cfg: &MetricsEndpointConfig, + registry: &ProducerRegistry, + vm_properties: &InstanceProperties, +) -> OximeterState { + let mut oximeter_state = OximeterState::default(); + let virtual_machine = VirtualMachine::from(vm_properties); + + // Create the server itself. + // + // The server manages all details of the registration with Nexus, so we + // don't need our own task for that or way to shut it down. + oximeter_state.server = match crate::stats::start_oximeter_server( + virtual_machine.instance_id, + cfg, + log, + registry, + ) { + Ok(server) => { + info!(log, "created metric producer server"); + Some(server) + } + Err(err) => { + error!( + log, + "failed to construct metric producer server, \ + no metrics will be available for this instance."; + "error" => ?err, + ); + None + } + }; + + // Assign our own metrics production for this VM instance to the + // registry, letting the server actually return them to oximeter when + // polled. + oximeter_state.stats = match crate::stats::register_server_metrics( + registry, + virtual_machine, + log, + ) + .await + { + Ok(stats) => Some(stats), + Err(e) => { + error!( + log, + "failed to register our server metrics with \ + the ProducerRegistry, no server stats will \ + be produced"; + "error" => ?e, + ); + + None + } + }; + + oximeter_state +} + +/// Launches a serial console handler task. +async fn start_serial_task<'vm>( + log: &slog::Logger, + vm_objects: &VmObjectsShared<'vm>, +) -> crate::serial::SerialTask { + let (websocks_ch, websocks_recv) = tokio::sync::mpsc::channel(1); + let (control_ch, control_recv) = tokio::sync::mpsc::channel(1); + + let serial = vm_objects.com1().clone(); + serial.set_task_control_sender(control_ch.clone()).await; + let err_log = log.new(slog::o!("component" => "serial task")); + let task = tokio::spawn(async move { + if let Err(e) = crate::serial::instance_serial_task( + websocks_recv, + control_recv, + serial, + err_log.clone(), + ) + .await + { + error!(err_log, "Failure in serial task: {}", e); + } + }); + + crate::serial::SerialTask { task, control_ch, websocks_ch } +} diff --git a/bin/propolis-server/src/lib/vm/state_driver.rs b/bin/propolis-server/src/lib/vm/state_driver.rs index 4a2832f64..a5fa9cdae 100644 --- a/bin/propolis-server/src/lib/vm/state_driver.rs +++ b/bin/propolis-server/src/lib/vm/state_driver.rs @@ -2,324 +2,442 @@ // License, v. 2.0. If a copy of the MPL was not distributed with this // file, You can obtain one at https://mozilla.org/MPL/2.0/. -use std::sync::Arc; +//! A task to handle requests to change a VM's state or configuration. -use crate::migrate::{MigrateError, MigrateRole}; -use crate::vcpu_tasks::VcpuTaskController; - -use super::{ - request_queue, ExternalRequest, GuestEvent, MigrateSourceCommand, - MigrateSourceResponse, MigrateTargetCommand, MigrateTaskEvent, - SharedVmState, StateDriverEvent, +use std::{ + sync::{Arc, Mutex}, + time::Duration, }; +use anyhow::Context; use propolis_api_types::{ - InstanceMigrateStatusResponse as ApiMigrateStatusResponse, - InstanceMigrationStatus as ApiMigrationStatus, - InstanceState as ApiInstanceState, - InstanceStateMonitorResponse as ApiMonitoredState, - MigrationState as ApiMigrationState, + instance_spec::{ + components::backends::CrucibleStorageBackend, v0::StorageBackendV0, + }, + InstanceSpecEnsureRequest, InstanceState, MigrationState, }; -use slog::{error, info, Logger}; +use slog::{error, info}; +use tokio::sync::Notify; use uuid::Uuid; -#[usdt::provider(provider = "propolis")] -mod probes { - fn state_driver_pause() {} - fn state_driver_resume() {} -} +use crate::{ + migrate::{ + destination::DestinationProtocol, source::SourceProtocol, MigrateRole, + }, + vm::state_publisher::ExternalStateUpdate, +}; -/// Tells the state driver whether or not to continue running after responding -/// to an event. +use super::{ + ensure::{VmEnsureActive, VmEnsureNotStarted}, + guest_event::{self, GuestEvent}, + objects::VmObjects, + request_queue::{self, ExternalRequest, InstanceAutoStart}, + state_publisher::{MigrationStateUpdate, StatePublisher}, + InstanceEnsureResponseTx, +}; + +/// Tells the state driver what to do after handling an event. #[derive(Debug, PartialEq, Eq)] enum HandleEventOutcome { Continue, - Exit, + Exit { final_state: InstanceState }, } /// A reason for starting a VM. #[derive(Debug, PartialEq, Eq)] -enum VmStartReason { +pub(super) enum VmStartReason { MigratedIn, ExplicitRequest, } -/// A wrapper around all the data needed to describe the status of a live -/// migration. -struct PublishedMigrationState { - state: ApiMigrationState, - id: Uuid, - role: MigrateRole, +/// A kind of event the state driver can handle. +#[derive(Debug)] +enum InputQueueEvent { + ExternalRequest(ExternalRequest), + GuestEvent(GuestEvent), } -impl PublishedMigrationState { - /// Updates an `old` migration status response to contain information about - /// the migration described by `self`. - fn apply_to( - self, - old: ApiMigrateStatusResponse, - ) -> ApiMigrateStatusResponse { - let new = ApiMigrationStatus { id: self.id, state: self.state }; - match self.role { - MigrateRole::Destination => ApiMigrateStatusResponse { - migration_in: Some(new), - migration_out: old.migration_out, - }, - MigrateRole::Source => ApiMigrateStatusResponse { - migration_in: old.migration_in, - migration_out: Some(new), - }, - } - } -} +/// The lock-guarded parts of a state driver's input queue. +struct InputQueueInner { + /// State change requests from the external API. + external_requests: request_queue::ExternalRequestQueue, -enum PublishedState { - Instance(ApiInstanceState), - Migration(PublishedMigrationState), - Complete(ApiInstanceState, PublishedMigrationState), + /// State change requests from the VM's components. These take precedence + /// over external state change requests. + guest_events: super::guest_event::GuestEventQueue, } -pub(super) struct StateDriver< - V: super::StateDriverVmController, - C: VcpuTaskController, -> { - /// A handle to the host server's tokio runtime, useful for spawning tasks - /// that need to interact with async code (e.g. spinning up migration - /// tasks). - runtime_hdl: tokio::runtime::Handle, - - /// A reference to the command sink to which this driver should send its - /// requests to send messages to devices or update other VM controller - /// state. - controller: Arc, - - /// A reference to the state this driver shares with its VM controller. - shared_state: Arc, - - /// The controller for this instance's vCPU tasks. - vcpu_tasks: C, - - /// The state worker's logger. - log: Logger, - - /// The generation number to use when publishing externally-visible state - /// updates. - state_gen: u64, +impl InputQueueInner { + fn new(log: slog::Logger, auto_start: InstanceAutoStart) -> Self { + Self { + external_requests: request_queue::ExternalRequestQueue::new( + log, auto_start, + ), + guest_events: super::guest_event::GuestEventQueue::default(), + } + } +} - /// Whether the worker's VM's devices are paused. - paused: bool, +/// A queue for external state change requests and guest-driven state changes. +pub(super) struct InputQueue { + /// Contains the input queue's sub-queues, one for external state change + /// requests and one for events emitted by the VM. + inner: Mutex, - /// The sender side of the monitor that reflects the instance's current - /// externally-visible state (including migration state). - api_state_tx: tokio::sync::watch::Sender, + /// Notifies the state driver that a new event is present on the queue. + /// + /// Notifiers must use [`Notify::notify_one`] when signaling this `Notify` + /// to guarantee the state driver does not miss incoming messages. See the + /// comments in [`InputQueue::wait_for_next_event`]. + notify: Notify, } -impl StateDriver -where - V: super::StateDriverVmController, - C: VcpuTaskController, -{ - /// Constructs a new state driver context. +impl InputQueue { + /// Creates a new state driver input queue. pub(super) fn new( - runtime_hdl: tokio::runtime::Handle, - controller: Arc, - shared_controller_state: Arc, - vcpu_tasks: C, - log: Logger, - api_state_tx: tokio::sync::watch::Sender, + log: slog::Logger, + auto_start: InstanceAutoStart, ) -> Self { Self { - runtime_hdl, - controller, - shared_state: shared_controller_state, - vcpu_tasks, - log, - state_gen: 0, - paused: false, - api_state_tx, + inner: Mutex::new(InputQueueInner::new(log, auto_start)), + notify: Notify::new(), } } - /// Yields the current externally-visible instance state. - fn get_instance_state(&self) -> ApiInstanceState { - self.api_state_tx.borrow().state - } - - /// Retrieves the most recently published migration state from the external - /// migration state channel. + /// Waits for an event to arrive on the input queue and returns it for + /// processing. /// - /// This function does not return the borrowed monitor, so the state may - /// change again as soon as this function returns. - fn get_migration_status(&self) -> ApiMigrateStatusResponse { - self.api_state_tx.borrow().migration.clone() - } + /// External requests and guest events are stored in separate queues. If + /// both queues have events when this routine is called, the guest event + /// queue takes precedence. + /// + /// # Synchronization + /// + /// This routine assumes that it is only ever called by one task (the state + /// driver). If multiple threads call this routine simultaneously, they may + /// miss wakeups and not return when new events are pushed to the queue or + /// cause a panic (see below). + async fn wait_for_next_event(&self) -> InputQueueEvent { + loop { + { + let mut guard = self.inner.lock().unwrap(); + if let Some(guest_event) = guard.guest_events.pop_front() { + return InputQueueEvent::GuestEvent(guest_event); + } else if let Some(req) = guard.external_requests.pop_front() { + return InputQueueEvent::ExternalRequest(req); + } + } - /// Sets the published instance and/or migration state and increases the - /// state generation number. - fn set_published_state(&mut self, state: PublishedState) { - let (instance_state, migration_state) = match state { - PublishedState::Instance(i) => (Some(i), None), - PublishedState::Migration(m) => (None, Some(m)), - PublishedState::Complete(i, m) => (Some(i), Some(m)), - }; + // It's safe not to use `Notified::enable` here because (1) only one + // thread (the state driver) can call `wait_for_next_event` on a + // given input queue, and (2) all the methods of signaling the queue + // use `notify_one`, which buffers a permit if no one is waiting + // when the signal arrives. This means that if a notification is + // sent after the lock is dropped but before `notified()` is called + // here, the ensuing wait will be satisfied immediately. + self.notify.notified().await; + } + } - let ApiMonitoredState { - state: old_state, - migration: old_migration, - .. - } = self.api_state_tx.borrow().clone(); + /// Notifies the external request queue that the instance's state has + /// changed so that it can change the dispositions for new state change + /// requests. + fn notify_instance_state_change( + &self, + state: request_queue::InstanceStateChange, + ) { + let mut guard = self.inner.lock().unwrap(); + guard.external_requests.notify_instance_state_change(state); + } - let state = instance_state.unwrap_or(old_state); - let migration = if let Some(migration_state) = migration_state { - migration_state.apply_to(old_migration) - } else { - old_migration - }; + /// Submits an external state change request to the queue. + pub(super) fn queue_external_request( + &self, + request: ExternalRequest, + ) -> Result<(), request_queue::RequestDeniedReason> { + let mut inner = self.inner.lock().unwrap(); + let result = inner.external_requests.try_queue(request); + if result.is_ok() { + self.notify.notify_one(); + } + result + } +} - info!(self.log, "publishing new instance state"; - "gen" => self.state_gen, - "state" => ?state, - "migration" => ?migration); +impl guest_event::VcpuEventHandler for InputQueue { + fn suspend_halt_event(&self, when: Duration) { + let mut guard = self.inner.lock().unwrap(); + if guard + .guest_events + .enqueue(guest_event::GuestEvent::VcpuSuspendHalt(when)) + { + self.notify.notify_one(); + } + } - self.state_gen += 1; - let _ = self.api_state_tx.send(ApiMonitoredState { - gen: self.state_gen, - state, - migration, - }); + fn suspend_reset_event(&self, when: Duration) { + let mut guard = self.inner.lock().unwrap(); + if guard + .guest_events + .enqueue(guest_event::GuestEvent::VcpuSuspendReset(when)) + { + self.notify.notify_one(); + } } - /// Publishes the supplied externally-visible instance state to the external - /// instance state channel. - fn set_instance_state(&mut self, state: ApiInstanceState) { - self.set_published_state(PublishedState::Instance(state)); + fn suspend_triple_fault_event(&self, vcpu_id: i32, when: Duration) { + let mut guard = self.inner.lock().unwrap(); + if guard.guest_events.enqueue( + guest_event::GuestEvent::VcpuSuspendTripleFault(vcpu_id, when), + ) { + self.notify.notify_one(); + } } - /// Publishes the supplied externally-visible migration status to the - /// instance state channel. - fn set_migration_state( - &mut self, - role: MigrateRole, - migration_id: Uuid, - state: ApiMigrationState, + fn unhandled_vm_exit( + &self, + vcpu_id: i32, + exit: propolis::exits::VmExitKind, ) { - self.set_published_state(PublishedState::Migration( - PublishedMigrationState { state, id: migration_id, role }, - )); + panic!("vCPU {}: Unhandled VM exit: {:?}", vcpu_id, exit); } - /// Publishes that an instance is migrating and sets its migration state in - /// a single transaction, then consumes the pending migration information - /// from the shared VM state block. - fn publish_migration_start( - &mut self, - migration_id: Uuid, - role: MigrateRole, - ) { - // Order matters here. The 'pending migration' field exists so that - // migration status is available through the external API as soon as an - // external request to migrate returns, even if the migration hasn't yet - // been picked up off the queue. To ensure the migration is continuously - // visible, publish the "actual" migration before consuming the pending - // one. - self.set_published_state(PublishedState::Complete( - ApiInstanceState::Migrating, - PublishedMigrationState { - state: ApiMigrationState::Sync, - id: migration_id, - role, - }, - )); + fn io_error_event(&self, vcpu_id: i32, error: std::io::Error) { + panic!("vCPU {}: Unhandled vCPU error: {}", vcpu_id, error); + } +} - self.shared_state.clear_pending_migration(); +impl guest_event::ChipsetEventHandler for InputQueue { + fn chipset_halt(&self) { + let mut guard = self.inner.lock().unwrap(); + if guard.guest_events.enqueue(guest_event::GuestEvent::ChipsetHalt) { + self.notify.notify_one(); + } } - /// Manages an instance's lifecycle once it has moved to the Running state. - pub(super) fn run_state_worker( - mut self, - ) -> tokio::sync::watch::Sender { - info!(self.log, "State worker launched"); + fn chipset_reset(&self) { + let mut guard = self.inner.lock().unwrap(); + if guard.guest_events.enqueue(guest_event::GuestEvent::ChipsetReset) { + self.notify.notify_one(); + } + } +} - loop { - let event = self.shared_state.wait_for_next_event(); - info!(self.log, "State worker handling event"; "event" => ?event); +/// The context for a VM state driver task's main loop. +struct StateDriver { + /// The state driver's associated logger. + log: slog::Logger, - let outcome = self.handle_event(event); - info!(self.log, "State worker handled event"; "outcome" => ?outcome); - if matches!(outcome, HandleEventOutcome::Exit) { - break; - } + /// The VM objects this driver is managing. + objects: Arc, + + /// The input queue this driver gets events from. + input_queue: Arc, + + /// The channel to which this driver publishes external instance state + /// changes. + external_state: StatePublisher, + + /// True if the VM is paused. + paused: bool, + + /// State persisted from previous attempts to migrate out of this VM. + migration_src_state: crate::migrate::source::PersistentState, +} + +/// The values returned by a state driver task when it exits. +pub(super) struct StateDriverOutput { + /// The channel this driver used to publish external instance state changes. + pub state_publisher: StatePublisher, + + /// The terminal state of this instance. When the instance completes + /// rundown, the parent VM publishes this state to the associated channel. + pub final_state: InstanceState, +} + +/// Creates a new set of VM objects in response to an `ensure_request` directed +/// to the supplied `vm`. +pub(super) async fn run_state_driver( + log: slog::Logger, + vm: Arc, + mut state_publisher: StatePublisher, + ensure_request: InstanceSpecEnsureRequest, + ensure_result_tx: InstanceEnsureResponseTx, + ensure_options: super::EnsureOptions, +) -> StateDriverOutput { + let activated_vm = match create_and_activate_vm( + &log, + &vm, + &mut state_publisher, + &ensure_request, + ensure_result_tx, + &ensure_options, + ) + .await + { + Ok(activated) => activated, + Err(e) => { + error!(log, "failed to activate new VM"; "error" => #%e); + return StateDriverOutput { + state_publisher, + final_state: InstanceState::Failed, + }; } + }; + + let (objects, input_queue) = activated_vm.into_inner(); + let state_driver = StateDriver { + log, + objects, + input_queue, + external_state: state_publisher, + paused: false, + migration_src_state: Default::default(), + }; + + // Run the VM until it exits, then set rundown on the parent VM so that no + // new external callers can access its objects or services. + let output = state_driver.run(ensure_request.migrate.is_some()).await; + vm.set_rundown().await; + output +} + +/// Processes the supplied `ensure_request` to create a set of VM objects that +/// can be moved into a new `StateDriver`. +async fn create_and_activate_vm<'a>( + log: &'a slog::Logger, + vm: &'a Arc, + state_publisher: &'a mut StatePublisher, + ensure_request: &'a InstanceSpecEnsureRequest, + ensure_result_tx: InstanceEnsureResponseTx, + ensure_options: &'a super::EnsureOptions, +) -> anyhow::Result> { + let ensure = VmEnsureNotStarted::new( + log, + vm, + ensure_request, + ensure_options, + ensure_result_tx, + state_publisher, + ); + + if let Some(migrate_request) = ensure_request.migrate.as_ref() { + let migration = match crate::migrate::destination::initiate( + log, + migrate_request, + ensure_options.local_server_addr, + ) + .await + { + Ok(mig) => mig, + Err(e) => { + return Err(ensure + .fail(e.into()) + .await + .context("creating migration protocol handler")); + } + }; - info!(self.log, "State worker exiting"); + // Delegate the rest of the activation process to the migration + // protocol. If the migration fails, the callee is responsible for + // dispatching failure messages to any API clients who are awaiting + // the results of their instance ensure calls. + Ok(migration + .run(ensure) + .await + .context("running live migration protocol")?) + } else { + let created = ensure + .create_objects() + .await + .context("creating VM objects for new instance")?; - self.api_state_tx + Ok(created.ensure_active().await) } +} - fn handle_event(&mut self, event: StateDriverEvent) -> HandleEventOutcome { - let next_action = match event { - StateDriverEvent::Guest(guest_event) => { - return self.handle_guest_event(guest_event); +impl StateDriver { + pub(super) async fn run(mut self, migrated_in: bool) -> StateDriverOutput { + info!(self.log, "state driver launched"); + + let final_state = if migrated_in { + if self.start_vm(VmStartReason::MigratedIn).await.is_ok() { + self.run_loop().await + } else { + InstanceState::Failed } - StateDriverEvent::External(external_event) => external_event, + } else { + self.run_loop().await }; - match next_action { - ExternalRequest::MigrateAsTarget { - migration_id, - task, - start_tx, - command_rx, - } => { - self.migrate_as_target( - migration_id, - task, - start_tx, - command_rx, - ); - HandleEventOutcome::Continue - } - ExternalRequest::Start => { - self.start_vm(VmStartReason::ExplicitRequest); - HandleEventOutcome::Continue - } - ExternalRequest::Reboot => { - self.do_reboot(); - HandleEventOutcome::Continue + StateDriverOutput { state_publisher: self.external_state, final_state } + } + + async fn run_loop(&mut self) -> InstanceState { + info!(self.log, "state driver entered main loop"); + loop { + let event = self.input_queue.wait_for_next_event().await; + info!(self.log, "state driver handling event"; "event" => ?event); + + let outcome = match event { + InputQueueEvent::ExternalRequest(req) => { + self.handle_external_request(req).await + } + InputQueueEvent::GuestEvent(event) => { + self.handle_guest_event(event).await + } + }; + + info!(self.log, "state driver handled event"; "outcome" => ?outcome); + match outcome { + HandleEventOutcome::Continue => {} + HandleEventOutcome::Exit { final_state } => { + info!(self.log, "state driver exiting"; + "final_state" => ?final_state); + + return final_state; + } } - ExternalRequest::MigrateAsSource { - migration_id, - task, - start_tx, - command_rx, - response_tx, - } => { - self.migrate_as_source( - migration_id, - task, - start_tx, - command_rx, - response_tx, - ); - HandleEventOutcome::Continue + } + } + + async fn start_vm( + &mut self, + start_reason: VmStartReason, + ) -> anyhow::Result<()> { + info!(self.log, "starting instance"; "reason" => ?start_reason); + + let start_result = + self.objects.lock_exclusive().await.start(start_reason).await; + match &start_result { + Ok(()) => { + self.publish_steady_state(InstanceState::Running); } - ExternalRequest::Stop => { - self.do_halt(); - HandleEventOutcome::Exit + Err(e) => { + error!(&self.log, "failed to start devices"; + "error" => ?e); + self.publish_steady_state(InstanceState::Failed); } } + + start_result } - fn handle_guest_event(&mut self, event: GuestEvent) -> HandleEventOutcome { + async fn handle_guest_event( + &mut self, + event: GuestEvent, + ) -> HandleEventOutcome { match event { GuestEvent::VcpuSuspendHalt(_when) => { info!(self.log, "Halting due to VM suspend event",); - self.do_halt(); - HandleEventOutcome::Exit + self.do_halt().await; + HandleEventOutcome::Exit { + final_state: InstanceState::Destroyed, + } } GuestEvent::VcpuSuspendReset(_when) => { info!(self.log, "Resetting due to VM suspend event"); - self.do_reboot(); + self.do_reboot().await; HandleEventOutcome::Continue } GuestEvent::VcpuSuspendTripleFault(vcpu_id, _when) => { @@ -327,1058 +445,269 @@ where self.log, "Resetting due to triple fault on vCPU {}", vcpu_id ); - self.do_reboot(); + self.do_reboot().await; HandleEventOutcome::Continue } GuestEvent::ChipsetHalt => { info!(self.log, "Halting due to chipset-driven halt"); - self.do_halt(); - HandleEventOutcome::Exit + self.do_halt().await; + HandleEventOutcome::Exit { + final_state: InstanceState::Destroyed, + } } GuestEvent::ChipsetReset => { info!(self.log, "Resetting due to chipset-driven reset"); - self.do_reboot(); + self.do_reboot().await; HandleEventOutcome::Continue } } } - fn start_vm(&mut self, start_reason: VmStartReason) { - info!(self.log, "Starting instance"; "reason" => ?start_reason); - - // Only move to the Starting state if this VM is starting by explicit - // request (as opposed to the implicit start that happens after a - // migration in). In this case, no one has initialized vCPU state yet, - // so explicitly initialize it here. - // - // In the migration-in case, remain in the Migrating state until the - // VM is actually running. Note that this is contractual behavior--sled - // agent relies on this to represent that a migrating instance is - // continuously running through a successful migration. - match start_reason { - VmStartReason::ExplicitRequest => { - self.set_instance_state(ApiInstanceState::Starting); - self.reset_vcpus(); + async fn handle_external_request( + &mut self, + request: ExternalRequest, + ) -> HandleEventOutcome { + match request { + ExternalRequest::Start => { + match self.start_vm(VmStartReason::ExplicitRequest).await { + Ok(_) => HandleEventOutcome::Continue, + Err(_) => HandleEventOutcome::Exit { + final_state: InstanceState::Failed, + }, + } } - VmStartReason::MigratedIn => { - assert_eq!( - self.get_instance_state(), - ApiInstanceState::Migrating - ); - // Signal the kernel VMM to resume devices which are handled by - // the in-kernel emulation. They were kept paused for - // consistency while migration state was loaded. - self.controller.resume_vm(); + ExternalRequest::MigrateAsSource { migration_id, websock } => { + self.migrate_as_source(migration_id, websock.into_inner()) + .await; + + // The callee either queues its own stop request (on a + // successful migration out) or resumes the VM (on a failed + // migration out). Either way, the main loop can just proceed to + // process the queue as normal. + HandleEventOutcome::Continue } - } - - match self.controller.start_devices() { - Ok(()) => { - self.vcpu_tasks.resume_all(); - self.publish_steady_state(ApiInstanceState::Running); + ExternalRequest::Reboot => { + self.do_reboot().await; + HandleEventOutcome::Continue } - Err(e) => { - error!(&self.log, "Failed to start devices: {:?}", e); - self.publish_steady_state(ApiInstanceState::Failed); + ExternalRequest::Stop => { + self.do_halt().await; + HandleEventOutcome::Exit { + final_state: InstanceState::Destroyed, + } + } + ExternalRequest::ReconfigureCrucibleVolume { + disk_name, + backend_id, + new_vcr_json, + result_tx, + } => { + let _ = result_tx.send( + self.reconfigure_crucible_volume( + disk_name, + &backend_id, + new_vcr_json, + ) + .await, + ); + HandleEventOutcome::Continue } } } - fn do_reboot(&mut self) { - info!(self.log, "Resetting instance"); - - self.set_instance_state(ApiInstanceState::Rebooting); - - // Reboot is implemented as a pause -> reset -> resume transition. - // - // First, pause the vCPUs and all devices so no partially-completed - // work is present. - self.vcpu_tasks.pause_all(); - self.controller.pause_devices(); - - // Reset all the entities and the VM's bhyve state, then reset the - // vCPUs. The vCPU reset must come after the bhyve reset. - self.controller.reset_devices_and_machine(); - self.reset_vcpus(); - - // Resume devices so they're ready to do more work, then resume vCPUs. - self.controller.resume_devices(); - self.vcpu_tasks.resume_all(); - - // Notify the request queue that this reboot request was processed. - // This does not use the `publish_steady_state` path because the queue - // treats an instance's initial transition to "Running" as a one-time - // event that's different from a return to the running state from a - // transient intermediate state. - self.notify_request_queue(request_queue::InstanceStateChange::Rebooted); - self.set_instance_state(ApiInstanceState::Running); - } + async fn do_reboot(&mut self) { + info!(self.log, "resetting instance"); - fn do_halt(&mut self) { - info!(self.log, "Stopping instance"); - self.set_instance_state(ApiInstanceState::Stopping); + self.external_state + .update(ExternalStateUpdate::Instance(InstanceState::Rebooting)); - // Entities expect to be paused before being halted. Note that the VM - // may be paused already if it is being torn down after a successful - // migration out. - if !self.paused { - self.pause(); - } + self.objects.lock_exclusive().await.reboot().await; - self.vcpu_tasks.exit_all(); - self.controller.halt_devices(); - self.publish_steady_state(ApiInstanceState::Stopped); + // Notify other consumers that the instance successfully rebooted and is + // now back to Running. + self.input_queue.notify_instance_state_change( + request_queue::InstanceStateChange::Rebooted, + ); + self.external_state + .update(ExternalStateUpdate::Instance(InstanceState::Running)); } - fn migrate_as_target( - &mut self, - migration_id: Uuid, - mut task: tokio::task::JoinHandle>, - start_tx: tokio::sync::oneshot::Sender<()>, - mut command_rx: tokio::sync::mpsc::Receiver, - ) { - self.publish_migration_start(migration_id, MigrateRole::Destination); - - // Ensure the VM's vCPUs are activated properly so that they can enter - // the guest after migration. Do this before allowing the migration task - // to start so that reset doesn't overwrite any state written by - // migration. - self.reset_vcpus(); - - // Place the VM in a paused state so we can load emulated device state - // in a consistent manner - self.controller.pause_vm(); - - start_tx.send(()).unwrap(); - loop { - let action = self.runtime_hdl.block_on(async { - Self::next_migrate_task_event( - &mut task, - &mut command_rx, - &self.log, - ) - .await - }); - - match action { - MigrateTaskEvent::TaskExited(res) => { - if res.is_ok() { - // Clients that observe that migration has finished - // need to observe that the instance is running before - // they are guaranteed to be able to do anything else - // that requires a running instance. - assert!(matches!( - self.get_migration_status() - .migration_in - .unwrap() - .state, - ApiMigrationState::Finish - )); - - self.start_vm(VmStartReason::MigratedIn); - } else { - assert!(matches!( - self.get_migration_status() - .migration_in - .unwrap() - .state, - ApiMigrationState::Error - )); - - // Resume the kernel VM so that if this state driver is - // asked to halt, the pause resulting therefrom won't - // observe that the VM is already paused. - self.controller.resume_vm(); - self.publish_steady_state(ApiInstanceState::Failed); - }; - - break; - } - MigrateTaskEvent::Command( - MigrateTargetCommand::UpdateState(state), - ) => { - self.set_migration_state( - MigrateRole::Destination, - migration_id, - state, - ); - } - } - } - } + async fn do_halt(&mut self) { + info!(self.log, "stopping instance"); + self.external_state + .update(ExternalStateUpdate::Instance(InstanceState::Stopping)); - fn migrate_as_source( - &mut self, - migration_id: Uuid, - mut task: tokio::task::JoinHandle>, - start_tx: tokio::sync::oneshot::Sender<()>, - mut command_rx: tokio::sync::mpsc::Receiver, - response_tx: tokio::sync::mpsc::Sender, - ) { - self.publish_migration_start(migration_id, MigrateRole::Source); - start_tx.send(()).unwrap(); + { + let mut guard = self.objects.lock_exclusive().await; - // Wait either for the migration task to exit or for it to ask the - // worker to pause or resume the instance's devices. - loop { - let action = self.runtime_hdl.block_on(async { - Self::next_migrate_task_event( - &mut task, - &mut command_rx, - &self.log, - ) - .await - }); - - match action { - // If the task exited, bubble its result back up to the main - // state worker loop to decide on the instance's next state. - // - // If migration failed while devices were paused, this instance - // is allowed to resume, so resume its components here. - MigrateTaskEvent::TaskExited(res) => { - if res.is_ok() { - assert!(matches!( - self.get_migration_status() - .migration_out - .unwrap() - .state, - ApiMigrationState::Finish - )); - - self.shared_state - .queue_external_request(ExternalRequest::Stop) - .expect("can always queue a request to stop"); - } else { - assert!(matches!( - self.get_migration_status() - .migration_out - .unwrap() - .state, - ApiMigrationState::Error - )); - - if self.paused { - self.resume(); - self.publish_steady_state( - ApiInstanceState::Running, - ); - } - } - - break; - } - MigrateTaskEvent::Command(cmd) => match cmd { - MigrateSourceCommand::UpdateState(state) => { - self.set_migration_state( - MigrateRole::Source, - migration_id, - state, - ); - } - MigrateSourceCommand::Pause => { - self.pause(); - response_tx - .blocking_send(MigrateSourceResponse::Pause(Ok(()))) - .unwrap(); - } - }, + // Entities expect to be paused before being halted. Note that the VM + // may be paused already if it is being torn down after a successful + // migration out. + if !self.paused { + guard.pause().await; + self.paused = true; } - } - } - async fn next_migrate_task_event( - task: &mut tokio::task::JoinHandle>, - command_rx: &mut tokio::sync::mpsc::Receiver, - log: &Logger, - ) -> MigrateTaskEvent { - if let Some(cmd) = command_rx.recv().await { - return MigrateTaskEvent::Command(cmd); + guard.halt().await; } - // The sender side of the command channel is dropped, which means the - // migration task is exiting. Wait for it to finish and snag its result. - match task.await { - Ok(res) => { - info!(log, "Migration source task exited: {:?}", res); - MigrateTaskEvent::TaskExited(res) - } - Err(join_err) => { - if join_err.is_cancelled() { - panic!("Migration task canceled"); - } else { - panic!( - "Migration task panicked: {:?}", - join_err.into_panic() - ); - } - } - } - } - - fn pause(&mut self) { - assert!(!self.paused); - probes::state_driver_pause!(|| ()); - self.vcpu_tasks.pause_all(); - self.controller.pause_devices(); - self.controller.pause_vm(); - self.paused = true; + self.publish_steady_state(InstanceState::Stopped); } - fn resume(&mut self) { - assert!(self.paused); - probes::state_driver_resume!(|| ()); - self.controller.resume_vm(); - self.controller.resume_devices(); - self.vcpu_tasks.resume_all(); - self.paused = false; - } - - fn reset_vcpus(&self) { - self.vcpu_tasks.new_generation(); - self.controller.reset_vcpu_state(); - } - - fn publish_steady_state(&mut self, state: ApiInstanceState) { + fn publish_steady_state(&mut self, state: InstanceState) { let change = match state { - ApiInstanceState::Running => { + InstanceState::Running => { request_queue::InstanceStateChange::StartedRunning } - ApiInstanceState::Stopped => { + InstanceState::Stopped => { request_queue::InstanceStateChange::Stopped } - ApiInstanceState::Failed => { - request_queue::InstanceStateChange::Failed - } + InstanceState::Failed => request_queue::InstanceStateChange::Failed, _ => panic!( "Called publish_steady_state on non-terminal state {:?}", state ), }; - self.notify_request_queue(change); - self.set_instance_state(state); + self.input_queue.notify_instance_state_change(change); + self.external_state.update(ExternalStateUpdate::Instance(state)); } - fn notify_request_queue( - &self, - queue_change: request_queue::InstanceStateChange, - ) { - self.shared_state - .inner - .lock() - .unwrap() - .external_request_queue - .notify_instance_state_change(queue_change); - } -} - -#[cfg(test)] -mod test { - use anyhow::bail; - use mockall::Sequence; - - use super::*; - use crate::vcpu_tasks::MockVcpuTaskController; - use crate::vm::MockStateDriverVmController; - - struct TestStateDriver { - driver: - StateDriver, - state_rx: tokio::sync::watch::Receiver, - } - - impl TestStateDriver { - fn api_state(&self) -> ApiInstanceState { - self.state_rx.borrow().state - } - } - - struct TestObjects { - vm_ctrl: MockStateDriverVmController, - vcpu_ctrl: MockVcpuTaskController, - shared_state: Arc, - } - - fn make_state_driver(objects: TestObjects) -> TestStateDriver { - let logger = slog::Logger::root(slog::Discard, slog::o!()); - let (state_tx, state_rx) = - tokio::sync::watch::channel(ApiMonitoredState { - gen: 0, - state: ApiInstanceState::Creating, - migration: ApiMigrateStatusResponse { - migration_in: None, - migration_out: None, - }, - }); - - TestStateDriver { - driver: StateDriver::new( - tokio::runtime::Handle::current(), - Arc::new(objects.vm_ctrl), - objects.shared_state.clone(), - objects.vcpu_ctrl, - logger, - state_tx, - ), - state_rx, - } - } - - /// Generates default mocks for the VM controller and vCPU task controller - /// that accept unlimited requests to read state. - fn make_default_mocks() -> TestObjects { - let logger = slog::Logger::root(slog::Discard, slog::o!()); - let vm_ctrl = MockStateDriverVmController::new(); - let vcpu_ctrl = MockVcpuTaskController::new(); - TestObjects { - vm_ctrl, - vcpu_ctrl, - shared_state: Arc::new(SharedVmState::new(&logger)), - } - } - - fn add_reboot_expectations( - vm_ctrl: &mut MockStateDriverVmController, - vcpu_ctrl: &mut MockVcpuTaskController, + async fn migrate_as_source( + &mut self, + migration_id: Uuid, + websock: dropshot::WebsocketConnection, ) { - // The reboot process requires careful ordering of steps to make sure - // the VM's vCPUs are put into the correct state when the machine starts - // up. - let mut seq = Sequence::new(); - - // First, reboot has to pause everything. It doesn't actually matter - // whether vCPUs or devices pause first, but there's no way to specify - // that these events must be sequenced before other expectations but - // have no ordering with respect to each other. - vcpu_ctrl - .expect_pause_all() - .times(1) - .in_sequence(&mut seq) - .returning(|| ()); - vm_ctrl - .expect_pause_devices() - .times(1) - .in_sequence(&mut seq) - .returning(|| ()); - - // The devices and--importantly--the bhyve VM itself must be reset - // before resetting any vCPU state (so that bhyve will accept the ioctls - // sent to the vCPUs during the reset process). - vm_ctrl - .expect_reset_devices_and_machine() - .times(1) - .in_sequence(&mut seq) - .returning(|| ()); - vcpu_ctrl - .expect_new_generation() - .times(1) - .in_sequence(&mut seq) - .returning(|| ()); - vm_ctrl - .expect_reset_vcpu_state() - .times(1) - .in_sequence(&mut seq) - .returning(|| ()); - - // Entities and vCPUs can technically be resumed in either order, but - // resuming devices first allows them to be ready when the vCPUs start - // creating work for them to do. - vm_ctrl - .expect_resume_devices() - .times(1) - .in_sequence(&mut seq) - .returning(|| ()); - vcpu_ctrl - .expect_resume_all() - .times(1) - .in_sequence(&mut seq) - .returning(|| ()); - } - - #[tokio::test] - async fn guest_triple_fault_reboots() { - let mut test_objects = make_default_mocks(); - - add_reboot_expectations( - &mut test_objects.vm_ctrl, - &mut test_objects.vcpu_ctrl, - ); - let mut driver = make_state_driver(test_objects); - driver.driver.handle_event(StateDriverEvent::Guest( - GuestEvent::VcpuSuspendTripleFault( - 0, - std::time::Duration::default(), - ), - )); - - assert!(matches!(driver.api_state(), ApiInstanceState::Running)); - } - - #[tokio::test] - async fn guest_chipset_reset_reboots() { - let mut test_objects = make_default_mocks(); - - add_reboot_expectations( - &mut test_objects.vm_ctrl, - &mut test_objects.vcpu_ctrl, - ); - let mut driver = make_state_driver(test_objects); - driver - .driver - .handle_event(StateDriverEvent::Guest(GuestEvent::ChipsetReset)); - - assert!(matches!(driver.api_state(), ApiInstanceState::Running)); - } - - #[tokio::test] - async fn start_from_cold_boot() { - let mut test_objects = make_default_mocks(); - let vm_ctrl = &mut test_objects.vm_ctrl; - let vcpu_ctrl = &mut test_objects.vcpu_ctrl; - let mut seq = Sequence::new(); - vcpu_ctrl - .expect_new_generation() - .times(1) - .in_sequence(&mut seq) - .returning(|| ()); - vm_ctrl - .expect_reset_vcpu_state() - .times(1) - .in_sequence(&mut seq) - .returning(|| ()); - vm_ctrl - .expect_start_devices() - .times(1) - .in_sequence(&mut seq) - .returning(|| Ok(())); - vcpu_ctrl - .expect_resume_all() - .times(1) - .in_sequence(&mut seq) - .returning(|| ()); - - let mut driver = make_state_driver(test_objects); - driver - .driver - .handle_event(StateDriverEvent::External(ExternalRequest::Start)); - - assert!(matches!(driver.api_state(), ApiInstanceState::Running)); - } - - #[tokio::test] - async fn device_start_failure_causes_instance_failure() { - let mut test_objects = make_default_mocks(); - let vm_ctrl = &mut test_objects.vm_ctrl; - let vcpu_ctrl = &mut test_objects.vcpu_ctrl; - let mut seq = Sequence::new(); - vcpu_ctrl - .expect_new_generation() - .times(1) - .in_sequence(&mut seq) - .returning(|| ()); - vm_ctrl - .expect_reset_vcpu_state() - .times(1) - .in_sequence(&mut seq) - .returning(|| ()); - vm_ctrl - .expect_start_devices() - .times(1) - .in_sequence(&mut seq) - .returning(|| bail!("injected failure into start_devices!")); - - let mut driver = make_state_driver(test_objects); - - // Failure allows the instance to be preserved for debugging. - assert_eq!( - driver.driver.handle_event(StateDriverEvent::External( - ExternalRequest::Start - )), - HandleEventOutcome::Continue - ); - - assert!(matches!(driver.api_state(), ApiInstanceState::Failed)); - } - - #[tokio::test] - async fn devices_pause_before_halting() { - let mut test_objects = make_default_mocks(); - let vm_ctrl = &mut test_objects.vm_ctrl; - let vcpu_ctrl = &mut test_objects.vcpu_ctrl; - let mut seq = Sequence::new(); - vcpu_ctrl - .expect_pause_all() - .times(1) - .in_sequence(&mut seq) - .returning(|| ()); - vm_ctrl - .expect_pause_devices() - .times(1) - .in_sequence(&mut seq) - .returning(|| ()); - vm_ctrl - .expect_pause_vm() - .times(1) - .in_sequence(&mut seq) - .returning(|| ()); - vcpu_ctrl - .expect_exit_all() - .times(1) - .in_sequence(&mut seq) - .returning(|| ()); - vm_ctrl - .expect_halt_devices() - .times(1) - .in_sequence(&mut seq) - .returning(|| ()); - - let mut driver = make_state_driver(test_objects); - driver - .driver - .handle_event(StateDriverEvent::External(ExternalRequest::Stop)); - - assert!(matches!(driver.api_state(), ApiInstanceState::Stopped)); - } - - #[tokio::test] - async fn devices_pause_once_when_halting_after_migration_out() { - let migration_id = Uuid::new_v4(); - let (start_tx, start_rx) = tokio::sync::oneshot::channel(); - let (task_exit_tx, task_exit_rx) = tokio::sync::oneshot::channel(); - let (command_tx, command_rx) = tokio::sync::mpsc::channel(1); - let (response_tx, mut response_rx) = tokio::sync::mpsc::channel(1); - let migrate_task = tokio::spawn(async move { - start_rx.await.unwrap(); - task_exit_rx.await.unwrap() - }); + let conn = tokio_tungstenite::WebSocketStream::from_raw_socket( + websock.into_inner(), + tokio_tungstenite::tungstenite::protocol::Role::Server, + None, + ) + .await; + + let migration = match crate::migrate::source::initiate( + &self.log, + migration_id, + conn, + &self.objects, + &self.migration_src_state, + ) + .await + { + Ok(migration) => migration, + Err(_) => { + self.external_state.update(ExternalStateUpdate::Migration( + MigrationStateUpdate { + id: migration_id, + state: MigrationState::Error, + role: MigrateRole::Source, + }, + )); + + return; + } + }; - let mut test_objects = make_default_mocks(); - let vm_ctrl = &mut test_objects.vm_ctrl; - let vcpu_ctrl = &mut test_objects.vcpu_ctrl; - - // This test will simulate a migration out (with a pause command), then - // order the state driver to halt. This should produce exactly one set - // of pause commands and one set of halt commands with no resume - // commands. - vm_ctrl.expect_pause_devices().times(1).returning(|| ()); - vcpu_ctrl.expect_pause_all().times(1).returning(|| ()); - vcpu_ctrl.expect_exit_all().times(1).returning(|| ()); - vm_ctrl.expect_halt_devices().times(1).returning(|| ()); - vm_ctrl.expect_resume_devices().never(); - vcpu_ctrl.expect_resume_all().never(); - vm_ctrl.expect_pause_vm().times(1).returning(|| ()); - vm_ctrl.expect_resume_vm().never(); - - let mut driver = make_state_driver(test_objects); - - // The state driver expects to run on an OS thread outside the async - // runtime so that it can call `block_on` to wait for messages from the + // Publish that migration is in progress before actually launching the // migration task. - let hdl = std::thread::spawn(move || { - driver.driver.handle_event(StateDriverEvent::External( - ExternalRequest::MigrateAsSource { - migration_id, - task: migrate_task, - start_tx, - command_rx, - response_tx, - }, - )); - - // Return the driver (which has the mocks attached) when the thread - // is joined so the test can continue using it. - driver - }); - - // Simulate a pause and the successful completion of migration. - command_tx.send(MigrateSourceCommand::Pause).await.unwrap(); - let resp = response_rx.recv().await.unwrap(); - assert!(matches!(resp, MigrateSourceResponse::Pause(Ok(())))); - command_tx - .send(MigrateSourceCommand::UpdateState(ApiMigrationState::Finish)) - .await - .unwrap(); - - drop(command_tx); - task_exit_tx.send(Ok(())).unwrap(); - - // Wait for the call to `handle_event` to return before tearing anything - // else down. - driver = tokio::task::spawn_blocking(move || hdl.join().unwrap()) - .await - .unwrap(); - - // The migration should appear to have finished. The state driver will - // queue a "stop" command to itself in this case, but because the driver - // is not directly processing the queue here, the test has to issue this - // call itself. - assert_eq!( - driver.driver.get_migration_status().migration_out.unwrap(), - ApiMigrationStatus { + self.external_state.update(ExternalStateUpdate::Complete( + InstanceState::Migrating, + MigrationStateUpdate { + state: MigrationState::Sync, id: migration_id, - state: ApiMigrationState::Finish - } - ); - - driver - .driver - .handle_event(StateDriverEvent::External(ExternalRequest::Stop)); - - assert!(matches!(driver.api_state(), ApiInstanceState::Stopped)); - } - - #[tokio::test] - async fn paused_vm_resumes_after_failed_migration_out() { - let migration_id = Uuid::new_v4(); - let (start_tx, start_rx) = tokio::sync::oneshot::channel(); - let (task_exit_tx, task_exit_rx) = tokio::sync::oneshot::channel(); - let (command_tx, command_rx) = tokio::sync::mpsc::channel(1); - let (response_tx, mut response_rx) = tokio::sync::mpsc::channel(1); - let migrate_task = tokio::spawn(async move { - start_rx.await.unwrap(); - task_exit_rx.await.unwrap() - }); - - let mut test_objects = make_default_mocks(); - let vm_ctrl = &mut test_objects.vm_ctrl; - let vcpu_ctrl = &mut test_objects.vcpu_ctrl; - - // This test will simulate a migration out up through pausing the - // source, then fail migration. This should pause and resume all the - // devices and the vCPUs. - vm_ctrl.expect_pause_devices().times(1).returning(|| ()); - vm_ctrl.expect_resume_devices().times(1).returning(|| ()); - vcpu_ctrl.expect_pause_all().times(1).returning(|| ()); - vcpu_ctrl.expect_resume_all().times(1).returning(|| ()); - - // VMM will be paused once prior to exporting state, and then resumed - // afterwards when the migration fails. - let mut pause_seq = Sequence::new(); - vm_ctrl - .expect_pause_vm() - .times(1) - .in_sequence(&mut pause_seq) - .returning(|| ()); - vm_ctrl - .expect_resume_vm() - .times(1) - .in_sequence(&mut pause_seq) - .returning(|| ()); - - let mut driver = make_state_driver(test_objects); - let hdl = std::thread::spawn(move || { - let outcome = driver.driver.handle_event( - StateDriverEvent::External(ExternalRequest::MigrateAsSource { - migration_id, - task: migrate_task, - start_tx, - command_rx, - response_tx, - }), - ); - - (driver, outcome) - }); - - // Simulate a successful pause. - command_tx.send(MigrateSourceCommand::Pause).await.unwrap(); - let resp = response_rx.recv().await.unwrap(); - assert!(matches!(resp, MigrateSourceResponse::Pause(Ok(())))); + role: MigrateRole::Source, + }, + )); - // Simulate failure. The migration protocol must both update the state - // to Error and make the task return `Err`. - command_tx - .send(MigrateSourceCommand::UpdateState(ApiMigrationState::Error)) + match migration + .run( + &self.objects, + &mut self.external_state, + &mut self.migration_src_state, + ) .await - .unwrap(); - drop(command_tx); - task_exit_tx.send(Err(MigrateError::UnexpectedMessage)).unwrap(); - - // Wait for the call to `handle_event` to return. - let (driver, outcome) = - tokio::task::spawn_blocking(move || hdl.join().unwrap()) - .await - .unwrap(); - - // The VM should be running and the state driver should continue - // operating normally. - assert!(matches!(driver.api_state(), ApiInstanceState::Running)); - assert_eq!(outcome, HandleEventOutcome::Continue); - assert_eq!( - driver.driver.get_migration_status().migration_out.unwrap(), - ApiMigrationStatus { - id: migration_id, - state: ApiMigrationState::Error + { + Ok(()) => { + info!(self.log, "migration out succeeded, queuing stop"); + // On a successful migration out, the protocol promises to leave + // the VM objects in a paused state, so don't pause them again. + self.paused = true; + self.input_queue + .queue_external_request(ExternalRequest::Stop) + .expect("can always queue a request to stop"); } - ); - } - - #[tokio::test] - async fn vm_starts_after_migration_in() { - let migration_id = Uuid::new_v4(); - let (start_tx, start_rx) = tokio::sync::oneshot::channel(); - let (task_exit_tx, task_exit_rx) = tokio::sync::oneshot::channel(); - let (command_tx, command_rx) = tokio::sync::mpsc::channel(1); - let migrate_task = tokio::spawn(async move { - start_rx.await.unwrap(); - task_exit_rx.await.unwrap() - }); - - let mut test_objects = make_default_mocks(); - let vm_ctrl = &mut test_objects.vm_ctrl; - let vcpu_ctrl = &mut test_objects.vcpu_ctrl; - - vcpu_ctrl.expect_new_generation().times(1).returning(|| ()); - vm_ctrl.expect_reset_vcpu_state().times(1).returning(|| ()); - vm_ctrl.expect_start_devices().times(1).returning(|| Ok(())); - vcpu_ctrl.expect_resume_all().times(1).returning(|| ()); - - let mut pause_seq = Sequence::new(); - vm_ctrl - .expect_pause_vm() - .times(1) - .in_sequence(&mut pause_seq) - .returning(|| ()); - vm_ctrl - .expect_resume_vm() - .times(1) - .in_sequence(&mut pause_seq) - .returning(|| ()); - - let mut driver = make_state_driver(test_objects); - - // The state driver expects to run on an OS thread outside the async - // runtime so that it can call `block_on` to wait for messages from the - // migration task. - let hdl = std::thread::spawn(move || { - driver.driver.handle_event(StateDriverEvent::External( - ExternalRequest::MigrateAsTarget { - migration_id, - task: migrate_task, - start_tx, - command_rx, - }, - )); - - driver - }); - - // Explicitly drop the command channel to signal to the driver that - // the migration task is completing. - command_tx - .send(MigrateTargetCommand::UpdateState(ApiMigrationState::Finish)) - .await - .unwrap(); - drop(command_tx); - task_exit_tx.send(Ok(())).unwrap(); - - // Wait for the call to `handle_event` to return before tearing anything - // else down. - let driver = tokio::task::spawn_blocking(move || hdl.join().unwrap()) - .await - .unwrap(); + Err(e) => { + info!(self.log, "migration out failed, resuming"; + "error" => ?e); - assert_eq!( - driver.driver.get_migration_status().migration_in.unwrap(), - ApiMigrationStatus { - id: migration_id, - state: ApiMigrationState::Finish + self.publish_steady_state(InstanceState::Running); } - ); - assert!(matches!(driver.api_state(), ApiInstanceState::Running)); + } } - #[tokio::test] - async fn failed_migration_in_fails_instance() { - let migration_id = Uuid::new_v4(); - let (start_tx, start_rx) = tokio::sync::oneshot::channel(); - let (task_exit_tx, task_exit_rx) = tokio::sync::oneshot::channel(); - let (command_tx, command_rx) = tokio::sync::mpsc::channel(1); - let migrate_task = tokio::spawn(async move { - start_rx.await.unwrap(); - task_exit_rx.await.unwrap() - }); - - let mut test_objects = make_default_mocks(); - let vm_ctrl = &mut test_objects.vm_ctrl; - let vcpu_ctrl = &mut test_objects.vcpu_ctrl; - - vcpu_ctrl.expect_new_generation().times(1).returning(|| ()); - vm_ctrl.expect_reset_vcpu_state().times(1).returning(|| ()); - vm_ctrl.expect_pause_vm().times(1).returning(|| ()); - vm_ctrl.expect_resume_vm().times(1).returning(|| ()); - let mut driver = make_state_driver(test_objects); + async fn reconfigure_crucible_volume( + &self, + disk_name: String, + backend_id: &Uuid, + new_vcr_json: String, + ) -> super::CrucibleReplaceResult { + info!(self.log, "request to replace Crucible VCR"; + "disk_name" => %disk_name, + "backend_id" => %backend_id); + + fn spec_element_not_found(disk_name: &str) -> dropshot::HttpError { + let msg = format!("Crucible backend for {:?} not found", disk_name); + dropshot::HttpError::for_not_found(Some(msg.clone()), msg) + } - // The state driver expects to run on an OS thread outside the async - // runtime so that it can call `block_on` to wait for messages from the - // migration task. - let hdl = std::thread::spawn(move || { - let outcome = driver.driver.handle_event( - StateDriverEvent::External(ExternalRequest::MigrateAsTarget { - migration_id, - task: migrate_task, - start_tx, - command_rx, - }), - ); - - (driver, outcome) - }); + let mut objects = self.objects.lock_exclusive().await; + let (readonly, old_vcr_json) = { + let StorageBackendV0::Crucible(bes) = objects + .instance_spec() + .backends + .storage_backends + .get(&disk_name) + .ok_or_else(|| spec_element_not_found(&disk_name))? + else { + return Err(spec_element_not_found(&disk_name)); + }; + + (bes.readonly, &bes.request_json) + }; - // The migration task is required to update the migration state to - // "Error" before exiting when migration fails. - command_tx - .send(MigrateTargetCommand::UpdateState(ApiMigrationState::Error)) - .await - .unwrap(); - drop(command_tx); - task_exit_tx.send(Err(MigrateError::UnexpectedMessage)).unwrap(); - - // Wait for the call to `handle_event` to return. - let (driver, outcome) = - tokio::task::spawn_blocking(move || hdl.join().unwrap()) - .await - .unwrap(); - - // The migration should appear to have failed, but the VM should be - // preserved for debugging. - assert_eq!(outcome, HandleEventOutcome::Continue); - assert!(matches!(driver.api_state(), ApiInstanceState::Failed)); - assert_eq!( - driver.driver.get_migration_status().migration_in.unwrap(), - ApiMigrationStatus { - id: migration_id, - state: ApiMigrationState::Error - } - ); - } + let replace_result = { + let backend = objects + .crucible_backends() + .get(backend_id) + .ok_or_else(|| { + let msg = + format!("No crucible backend for id {backend_id}"); + dropshot::HttpError::for_not_found(Some(msg.clone()), msg) + })?; + + backend.vcr_replace(old_vcr_json, &new_vcr_json).await.map_err( + |e| { + dropshot::HttpError::for_bad_request( + Some(e.to_string()), + e.to_string(), + ) + }, + ) + }?; - #[tokio::test] - async fn failed_vm_start_after_migration_in_fails_instance() { - let migration_id = Uuid::new_v4(); - let (start_tx, start_rx) = tokio::sync::oneshot::channel(); - let (task_exit_tx, task_exit_rx) = tokio::sync::oneshot::channel(); - let (command_tx, command_rx) = tokio::sync::mpsc::channel(1); - let migrate_task = tokio::spawn(async move { - start_rx.await.unwrap(); - task_exit_rx.await.unwrap() + let new_bes = StorageBackendV0::Crucible(CrucibleStorageBackend { + readonly, + request_json: new_vcr_json, }); - let mut test_objects = make_default_mocks(); - let vm_ctrl = &mut test_objects.vm_ctrl; - let vcpu_ctrl = &mut test_objects.vcpu_ctrl; - - vcpu_ctrl.expect_new_generation().times(1).returning(|| ()); - vm_ctrl.expect_reset_vcpu_state().times(1).returning(|| ()); - - let mut pause_seq = Sequence::new(); - vm_ctrl - .expect_pause_vm() - .times(1) - .in_sequence(&mut pause_seq) - .returning(|| ()); - vm_ctrl - .expect_resume_vm() - .times(1) - .in_sequence(&mut pause_seq) - .returning(|| ()); - - vm_ctrl - .expect_start_devices() - .times(1) - .returning(|| bail!("injected failure into start_devices!")); - - let mut driver = make_state_driver(test_objects); - - // The state driver expects to run on an OS thread outside the async - // runtime so that it can call `block_on` to wait for messages from the - // migration task. - let hdl = std::thread::spawn(move || { - let outcome = driver.driver.handle_event( - StateDriverEvent::External(ExternalRequest::MigrateAsTarget { - migration_id, - task: migrate_task, - start_tx, - command_rx, - }), - ); - - (driver, outcome) - }); + objects + .instance_spec_mut() + .backends + .storage_backends + .insert(disk_name, new_bes); - // Explicitly drop the command channel to signal to the driver that - // the migration task is completing. - command_tx - .send(MigrateTargetCommand::UpdateState(ApiMigrationState::Finish)) - .await - .unwrap(); - drop(command_tx); - task_exit_tx.send(Ok(())).unwrap(); - - // Wait for the call to `handle_event` to return. - let (driver, outcome) = - tokio::task::spawn_blocking(move || hdl.join().unwrap()) - .await - .unwrap(); - - // The instance should have failed, but should also be preserved for - // debugging. - assert_eq!(outcome, HandleEventOutcome::Continue); - assert!(matches!(driver.api_state(), ApiInstanceState::Failed)); - - // The migration has still succeeded in this case. - assert_eq!( - driver.driver.get_migration_status().migration_in.unwrap(), - ApiMigrationStatus { - id: migration_id, - state: ApiMigrationState::Finish - } - ); - } + info!(self.log, "replaced Crucible VCR"; "backend_id" => %backend_id); - #[tokio::test] - async fn start_vm_after_migration_in_does_not_publish_starting_state() { - let mut test_objects = make_default_mocks(); - let vm_ctrl = &mut test_objects.vm_ctrl; - let vcpu_ctrl = &mut test_objects.vcpu_ctrl; - - // A call to start a VM after a successful migration should start vCPUs - // and devices without resetting anything. - vcpu_ctrl.expect_resume_all().times(1).returning(|| ()); - vm_ctrl.expect_start_devices().times(1).returning(|| Ok(())); - - // As noted below, the instance state is being magicked directly into a - // `Migrating` state, rather than executing the logic which would - // typically carry it there. As such, `pause_vm()` will not be called - // as part of setup. Since instance start _is_ being tested here, the - // `resume_vm()` call is expected. - vm_ctrl.expect_pause_vm().never(); - vm_ctrl.expect_resume_vm().times(1).returning(|| ()); - - // Skip the rigmarole of standing up a fake migration. Instead, just - // push the driver into the state it would have after a successful - // migration to appease the assertions in `start_vm`. - // - // Faking an entire migration, as in the previous tests, requires the - // state driver to run on its own worker thread. This is fine for tests - // that only want to examine state after the driver has finished an - // operation, but this test wants to test side effects of a specific - // part of the state driver's actions, which are tough to synchronize - // with when the driver is running on another thread. - let mut driver = make_state_driver(test_objects); - driver.driver.set_instance_state(ApiInstanceState::Migrating); - - // The driver starts in the Migrating state and should go directly to - // the Running state without passing through Starting. Because there's - // no way to guarantee that the test will see all intermediate states - // that `start_vm` publishes, instead assert that the final state of - // Running is correct and that the state generation only went up by 1 - // (implying that there were no intervening transitions). - let migrating_gen = driver.driver.api_state_tx.borrow().gen; - driver.driver.start_vm(VmStartReason::MigratedIn); - let new_state = driver.driver.api_state_tx.borrow().clone(); - assert!(matches!(new_state.state, ApiInstanceState::Running)); - assert_eq!(new_state.gen, migrating_gen + 1); + Ok(replace_result) } } diff --git a/bin/propolis-server/src/lib/vm/state_publisher.rs b/bin/propolis-server/src/lib/vm/state_publisher.rs new file mode 100644 index 000000000..63a4f93a1 --- /dev/null +++ b/bin/propolis-server/src/lib/vm/state_publisher.rs @@ -0,0 +1,114 @@ +// This Source Code Form is subject to the terms of the Mozilla Public +// License, v. 2.0. If a copy of the MPL was not distributed with this +// file, You can obtain one at https://mozilla.org/MPL/2.0/. + +//! Helper types for publishing instance states as made visible through the +//! external API. + +use propolis_api_types::{ + InstanceMigrateStatusResponse, InstanceMigrationStatus, InstanceState, + InstanceStateMonitorResponse, +}; +use slog::info; +use uuid::Uuid; + +use crate::migrate::MigrateRole; + +use super::{InstanceStateRx, InstanceStateTx}; + +/// An update to an instance's migration's state. +pub(crate) struct MigrationStateUpdate { + /// The migration's new state. + pub state: propolis_api_types::MigrationState, + + /// The migration's ID. + pub id: Uuid, + + /// The role this VM was playing in the migration of interest. + pub role: MigrateRole, +} + +impl MigrationStateUpdate { + /// Applies an update to a previous migration status and returns the new + /// status. + fn apply_to( + self, + old: InstanceMigrateStatusResponse, + ) -> InstanceMigrateStatusResponse { + let new = InstanceMigrationStatus { id: self.id, state: self.state }; + match self.role { + MigrateRole::Destination => InstanceMigrateStatusResponse { + migration_in: Some(new), + migration_out: old.migration_out, + }, + MigrateRole::Source => InstanceMigrateStatusResponse { + migration_in: old.migration_in, + migration_out: Some(new), + }, + } + } +} + +/// A kind of state update to publish. +pub(crate) enum ExternalStateUpdate { + /// Update the instance state (but not any migration state). + Instance(InstanceState), + + /// Update migration state (but not the instance's state). + Migration(MigrationStateUpdate), + + /// Update both instance and migration state. + Complete(InstanceState, MigrationStateUpdate), +} + +/// A channel to which to publish externally-visible instance state updates. +pub(crate) struct StatePublisher { + tx: InstanceStateTx, + log: slog::Logger, +} + +impl StatePublisher { + pub(super) fn new( + log: &slog::Logger, + initial_state: InstanceStateMonitorResponse, + ) -> (Self, InstanceStateRx) { + let (tx, rx) = tokio::sync::watch::channel(initial_state); + (Self { tx, log: log.clone() }, rx) + } + + /// Updates an instance's externally-visible state and publishes that state + /// with a successor generation number. + pub(crate) fn update(&mut self, update: ExternalStateUpdate) { + let (instance_state, migration_state) = match update { + ExternalStateUpdate::Instance(i) => (Some(i), None), + ExternalStateUpdate::Migration(m) => (None, Some(m)), + ExternalStateUpdate::Complete(i, m) => (Some(i), Some(m)), + }; + + let InstanceStateMonitorResponse { + state: old_instance, + migration: old_migration, + gen: old_gen, + } = self.tx.borrow().clone(); + + let state = instance_state.unwrap_or(old_instance); + let migration = if let Some(migration_state) = migration_state { + migration_state.apply_to(old_migration) + } else { + old_migration + }; + + let gen = old_gen + 1; + info!(self.log, "publishing new instance state"; + "gen" => gen, + "state" => ?state, + "migration" => ?migration); + + let _ = + self.tx.send(propolis_api_types::InstanceStateMonitorResponse { + gen, + state, + migration, + }); + } +} diff --git a/bin/propolis-server/src/lib/vnc.rs b/bin/propolis-server/src/lib/vnc.rs index 7f2b402e5..9a93643f7 100644 --- a/bin/propolis-server/src/lib/vnc.rs +++ b/bin/propolis-server/src/lib/vnc.rs @@ -13,13 +13,11 @@ use rfb::rfb::{ SecurityTypes, }; use rfb::server::{Server, VncServer, VncServerConfig, VncServerData}; -use slog::{debug, error, info, o, trace, Logger}; +use slog::{debug, error, info, o, trace, warn, Logger}; use std::net::SocketAddr; use std::sync::Arc; use tokio::sync::Mutex; -use crate::vm::VmController; - const INITIAL_WIDTH: u16 = 1024; const INITIAL_HEIGHT: u16 = 768; @@ -55,7 +53,7 @@ enum Framebuffer { struct PropolisVncServerInner { framebuffer: Framebuffer, ps2ctrl: Option>, - vm: Option>, + vm: Option>, } #[derive(Clone)] @@ -79,11 +77,11 @@ impl PropolisVncServer { } } - pub async fn initialize( + pub(crate) async fn initialize( &self, fb: RamFb, ps2ctrl: Arc, - vm: Arc, + vm: Arc, ) { let mut inner = self.inner.lock().await; inner.framebuffer = Framebuffer::Initialized(fb); @@ -154,15 +152,22 @@ impl Server for PropolisVncServer { let len = fb.height as usize * fb.width as usize * 4; let mut buf = vec![0u8; len]; - - let read = tokio::task::block_in_place(|| { - let machine = inner.vm.as_ref().unwrap().machine(); - let memctx = machine.acc_mem.access().unwrap(); - memctx.read_into(GuestAddr(fb.addr), &mut buf, len) - }); - - assert!(read.is_some()); - debug!(self.log, "read {} bytes from guest", read.unwrap()); + if let Some(vm) = inner.vm.as_ref().unwrap().active_vm().await { + let vm_objects = vm.objects().lock_shared().await; + let read = tokio::task::block_in_place(|| { + let memctx = vm_objects.access_mem().unwrap(); + memctx.read_into(GuestAddr(fb.addr), &mut buf, len) + }); + + assert!(read.is_some()); + debug!(self.log, "read {} bytes from guest", read.unwrap()); + } else { + warn!( + self.log, + "got framebuffer init message but VM is gone" + ); + buf = vec![0xffu8; len]; + } let r = Rectangle::new( 0, diff --git a/bin/propolis-standalone/src/config.rs b/bin/propolis-standalone/src/config.rs index cc9bde61f..3aff6c0da 100644 --- a/bin/propolis-standalone/src/config.rs +++ b/bin/propolis-standalone/src/config.rs @@ -330,7 +330,11 @@ fn create_crucible_backend( }; info!(log, "Creating Crucible disk from request {:?}", req); // QUESTION: is producer_registry: None correct here? - block::CrucibleBackend::create(req, opts, None, None, log.clone()).unwrap() + tokio::runtime::Handle::current().block_on(async move { + block::CrucibleBackend::create(req, opts, None, None, log.clone()) + .await + .unwrap() + }) } #[cfg(feature = "crucible")] @@ -345,7 +349,11 @@ fn create_crucible_mem_backend( } let parsed: CrucibleMemConfig = opt_deser(&be.options).unwrap(); - block::CrucibleBackend::create_mem(parsed.size, opts, log.clone()).unwrap() + tokio::runtime::Handle::current().block_on(async move { + block::CrucibleBackend::create_mem(parsed.size, opts, log.clone()) + .await + .unwrap() + }) } #[cfg(not(feature = "crucible"))] diff --git a/bin/propolis-standalone/src/main.rs b/bin/propolis-standalone/src/main.rs index bdd4a3f97..1f210b661 100644 --- a/bin/propolis-standalone/src/main.rs +++ b/bin/propolis-standalone/src/main.rs @@ -359,22 +359,26 @@ impl Instance { // Drive block backends through their necessary states too match state { State::Run if first_boot => { - for (_name, be) in guard.inventory.block.iter() { - be.start().expect("blockdev start succeeds"); - } + tokio::runtime::Handle::current().block_on(async { + for (_name, be) in guard.inventory.block.iter() { + be.start().await.expect("blockdev start succeeds"); + } + }); } State::Halt => { - for (name, be) in guard.inventory.block.iter() { - be.stop(); - if let Err(err) = be.detach() { - slog::error!( - log, - "Error during detach of block backend {}: {:?}", - name, - err - ); + tokio::runtime::Handle::current().block_on(async { + for (name, be) in guard.inventory.block.iter() { + be.stop().await; + if let Err(err) = be.detach() { + slog::error!( + log, + "Error during detach of block backend {}: {:?}", + name, + err + ); + } } - } + }); } _ => {} } diff --git a/lib/propolis/Cargo.toml b/lib/propolis/Cargo.toml index 6d75a067f..c663b437f 100644 --- a/lib/propolis/Cargo.toml +++ b/lib/propolis/Cargo.toml @@ -34,6 +34,7 @@ crucible-client-types = { workspace = true, optional = true } crucible = { workspace = true, optional = true } oximeter = { workspace = true, optional = true } nexus-client = { workspace = true, optional = true } +async-trait.workspace = true # falcon libloading = { workspace = true, optional = true } diff --git a/lib/propolis/src/block/crucible.rs b/lib/propolis/src/block/crucible.rs index d662f01af..241019c60 100644 --- a/lib/propolis/src/block/crucible.rs +++ b/lib/propolis/src/block/crucible.rs @@ -80,37 +80,17 @@ impl WorkerState { } impl CrucibleBackend { - pub fn create( + pub async fn create( request: VolumeConstructionRequest, opts: block::BackendOpts, producer_registry: Option, nexus_client: Option, log: slog::Logger, ) -> io::Result> { - let rt = tokio::runtime::Handle::current(); - rt.block_on(async move { - CrucibleBackend::_create( - request, - opts, - producer_registry, - nexus_client, - log, - ) - .await - }) - .map_err(CrucibleError::into) - } - - async fn _create( - request: VolumeConstructionRequest, - opts: block::BackendOpts, - producer_registry: Option, - nexus_client: Option, - log: slog::Logger, - ) -> Result, crucible::CrucibleError> { // Construct the volume. - let volume = - Volume::construct(request, producer_registry, log.clone()).await?; + let volume = Volume::construct(request, producer_registry, log.clone()) + .await + .map_err(|e| io::Error::from(CrucibleError::from(e)))?; // Decide if we need to scrub this volume or not. if volume.has_read_only_parent() { @@ -176,42 +156,41 @@ impl CrucibleBackend { /// Create Crucible backend using the in-memory volume backend, rather than /// "real" Crucible downstairs instances. - pub fn create_mem( + pub async fn create_mem( size: u64, opts: block::BackendOpts, log: slog::Logger, ) -> io::Result> { - let rt = tokio::runtime::Handle::current(); - rt.block_on(async move { - let block_size = u64::from(opts.block_size.ok_or_else(|| { - CrucibleError::GenericError( - "block_size is required parameter".into(), - ) - })?); - // Allocate and construct the volume. - let mem_disk = Arc::new(crucible::InMemoryBlockIO::new( - Uuid::new_v4(), - block_size, - size as usize, - )); - let mut volume = Volume::new(block_size, log); - volume.add_subvolume(mem_disk).await?; - - Ok(Arc::new(CrucibleBackend { - state: Arc::new(WorkerState { - attachment: block::BackendAttachment::new(), - volume, - info: block::DeviceInfo { - block_size: block_size as u32, - total_size: size / block_size, - read_only: opts.read_only.unwrap_or(false), - }, - skip_flush: opts.skip_flush.unwrap_or(false), - }), - workers: TaskGroup::new(), - })) - }) - .map_err(CrucibleError::into) + let block_size = u64::from(opts.block_size.ok_or_else(|| { + CrucibleError::GenericError( + "block_size is required parameter".into(), + ) + })?); + // Allocate and construct the volume. + let mem_disk = Arc::new(crucible::InMemoryBlockIO::new( + Uuid::new_v4(), + block_size, + size as usize, + )); + let mut volume = Volume::new(block_size, log); + volume + .add_subvolume(mem_disk) + .await + .map_err(|e| std::io::Error::from(e))?; + + Ok(Arc::new(CrucibleBackend { + state: Arc::new(WorkerState { + attachment: block::BackendAttachment::new(), + volume, + info: block::DeviceInfo { + block_size: block_size as u32, + total_size: size / block_size, + read_only: opts.read_only.unwrap_or(false), + }, + skip_flush: opts.skip_flush.unwrap_or(false), + }), + workers: TaskGroup::new(), + })) } // Communicate to Nexus that we can remove the read only parent for @@ -242,10 +221,8 @@ impl CrucibleBackend { } /// Retrieve the UUID identifying this Crucible backend. - pub fn get_uuid(&self) -> io::Result { - let rt = tokio::runtime::Handle::current(); - rt.block_on(async { self.state.volume.get_uuid().await }) - .map_err(CrucibleError::into) + pub async fn get_uuid(&self) -> io::Result { + self.state.volume.get_uuid().await.map_err(CrucibleError::into) } /// Issue a snapshot request @@ -297,6 +274,7 @@ impl CrucibleBackend { } } +#[async_trait::async_trait] impl block::Backend for CrucibleBackend { fn attachment(&self) -> &block::BackendAttachment { &self.state.attachment @@ -304,17 +282,15 @@ impl block::Backend for CrucibleBackend { fn info(&self) -> DeviceInfo { self.state.info } - fn start(&self) -> anyhow::Result<()> { - let rt = tokio::runtime::Handle::current(); - rt.block_on(async move { self.state.volume.activate().await })?; - + async fn start(&self) -> anyhow::Result<()> { + self.state.volume.activate().await?; self.state.attachment.start(); self.spawn_workers(); Ok(()) } - fn stop(&self) { + async fn stop(&self) -> () { self.state.attachment.stop(); - self.workers.block_until_joined(); + self.workers.join_all().await; } } diff --git a/lib/propolis/src/block/file.rs b/lib/propolis/src/block/file.rs index 1d1e91d52..aa8b3e99d 100644 --- a/lib/propolis/src/block/file.rs +++ b/lib/propolis/src/block/file.rs @@ -211,6 +211,7 @@ impl FileBackend { } } +#[async_trait::async_trait] impl block::Backend for FileBackend { fn attachment(&self) -> &block::BackendAttachment { &self.state.attachment @@ -219,7 +220,8 @@ impl block::Backend for FileBackend { fn info(&self) -> DeviceInfo { self.state.info } - fn start(&self) -> anyhow::Result<()> { + + async fn start(&self) -> anyhow::Result<()> { self.state.attachment.start(); if let Err(e) = self.spawn_workers() { self.state.attachment.stop(); @@ -229,7 +231,8 @@ impl block::Backend for FileBackend { Ok(()) } } - fn stop(&self) { + + async fn stop(&self) -> () { self.state.attachment.stop(); self.workers.block_until_joined(); } diff --git a/lib/propolis/src/block/in_memory.rs b/lib/propolis/src/block/in_memory.rs index 17ea47f44..3636bc8fa 100644 --- a/lib/propolis/src/block/in_memory.rs +++ b/lib/propolis/src/block/in_memory.rs @@ -140,14 +140,17 @@ impl InMemoryBackend { } } +#[async_trait::async_trait] impl block::Backend for InMemoryBackend { fn attachment(&self) -> &block::BackendAttachment { &self.state.attachment } + fn info(&self) -> block::DeviceInfo { self.state.info } - fn start(&self) -> anyhow::Result<()> { + + async fn start(&self) -> anyhow::Result<()> { self.state.attachment.start(); if let Err(e) = self.spawn_workers() { self.state.attachment.stop(); @@ -157,7 +160,8 @@ impl block::Backend for InMemoryBackend { Ok(()) } } - fn stop(&self) { + + async fn stop(&self) -> () { self.state.attachment.stop(); self.workers.block_until_joined(); } diff --git a/lib/propolis/src/block/mem_async.rs b/lib/propolis/src/block/mem_async.rs index 86cff9a1b..ff733c56e 100644 --- a/lib/propolis/src/block/mem_async.rs +++ b/lib/propolis/src/block/mem_async.rs @@ -205,21 +205,24 @@ impl Drop for MmapSeg { unsafe impl Send for MmapSeg {} unsafe impl Sync for MmapSeg {} +#[async_trait::async_trait] impl block::Backend for MemAsyncBackend { fn info(&self) -> block::DeviceInfo { self.work_state.info } + fn attachment(&self) -> &block::BackendAttachment { &self.work_state.attachment } - fn start(&self) -> anyhow::Result<()> { + + async fn start(&self) -> anyhow::Result<()> { self.work_state.attachment.start(); self.spawn_workers(); Ok(()) } - fn stop(&self) { + async fn stop(&self) -> () { self.work_state.attachment.stop(); - self.workers.block_until_joined(); + self.workers.join_all().await; } } diff --git a/lib/propolis/src/block/mod.rs b/lib/propolis/src/block/mod.rs index 722076efd..5e838e62d 100644 --- a/lib/propolis/src/block/mod.rs +++ b/lib/propolis/src/block/mod.rs @@ -233,6 +233,7 @@ pub trait Device: Send + Sync + 'static { /// Top-level trait for block backends which will attach to [Device]s in order /// to process [Request]s posted by the guest. +#[async_trait::async_trait] pub trait Backend: Send + Sync + 'static { /// Access to the [BackendAttachment] representing this backend. fn attachment(&self) -> &BackendAttachment; @@ -244,7 +245,7 @@ pub trait Backend: Send + Sync + 'static { /// /// Spawning of any tasks required to do such request processing can be done /// as part of this start-up. - fn start(&self) -> anyhow::Result<()>; + async fn start(&self) -> anyhow::Result<()>; /// Stop attempting to process new [Request]s from [Device] (if attached) /// @@ -253,7 +254,7 @@ pub trait Backend: Send + Sync + 'static { /// /// If any tasks were spawned as part of [Backend::start()], they should be /// brought to rest as part of this call. - fn stop(&self); + async fn stop(&self) -> (); /// Attempt to detach from associated [Device] /// diff --git a/lib/propolis/src/hw/chipset/mod.rs b/lib/propolis/src/hw/chipset/mod.rs index 47f930d72..40177f674 100644 --- a/lib/propolis/src/hw/chipset/mod.rs +++ b/lib/propolis/src/hw/chipset/mod.rs @@ -9,7 +9,7 @@ use crate::intr_pins::IntrPin; pub mod i440fx; -pub trait Chipset { +pub trait Chipset: Send + Sync { fn pci_attach( &self, bdf: Bdf, diff --git a/lib/propolis/src/hw/virtio/softnpu.rs b/lib/propolis/src/hw/virtio/softnpu.rs index ef55e5827..cf1740fae 100644 --- a/lib/propolis/src/hw/virtio/softnpu.rs +++ b/lib/propolis/src/hw/virtio/softnpu.rs @@ -978,7 +978,7 @@ impl P9Handler for SoftNpuP9Handler { // may discover us from trying to use us as some sort of normal P9 // file system. It also helps clients that are actually looking for the // SoftNpu P9 device to identify us as such. - msg.version = "9P2000.P4".to_owned(); + "9P2000.P4".clone_into(&mut msg.version); let mut out = ispf::to_bytes_le(&msg).unwrap(); let buf = out.as_mut_slice(); diff --git a/lib/propolis/src/tasks.rs b/lib/propolis/src/tasks.rs index efcc45404..536742eea 100644 --- a/lib/propolis/src/tasks.rs +++ b/lib/propolis/src/tasks.rs @@ -434,22 +434,27 @@ impl TaskGroup { guard.extend(tasks); } - /// Block until all held tasks have been joined, returning any resulting - /// [task::JoinError]s after doing so. - pub fn block_until_joined(&self) -> Option> { - let mut guard = self.0.lock().unwrap(); - let workers = std::mem::replace(&mut *guard, Vec::new()); + /// Waits until all the workers in this task group have completed. + /// + /// # Return value + /// + /// `None` if all the tasks completed successfully. `Some` if at least one + /// task failed; the wrapped value is a `Vec` of all of the returned errors. + pub async fn join_all(&self) -> Option> { + let workers = { + let mut guard = self.0.lock().unwrap(); + std::mem::replace(&mut *guard, Vec::new()) + }; + if workers.is_empty() { return None; } - let rt = tokio::runtime::Handle::current(); - let errors = rt.block_on(async { - FuturesUnordered::from_iter(workers) - .filter_map(|res| futures::future::ready(res.err())) - .collect::>() - .await - }); + let errors = FuturesUnordered::from_iter(workers) + .filter_map(|res| futures::future::ready(res.err())) + .collect::>() + .await; + if errors.is_empty() { None } else { diff --git a/phd-tests/framework/Cargo.toml b/phd-tests/framework/Cargo.toml index fd7cf8307..53b63afbe 100644 --- a/phd-tests/framework/Cargo.toml +++ b/phd-tests/framework/Cargo.toml @@ -19,6 +19,7 @@ errno.workspace = true futures.workspace = true flate2.workspace = true hex.workspace = true +http.workspace = true libc.workspace = true propolis-client.workspace = true propolis-server-config.workspace = true diff --git a/phd-tests/framework/src/test_vm/mod.rs b/phd-tests/framework/src/test_vm/mod.rs index 4c415baac..c1201ba0e 100644 --- a/phd-tests/framework/src/test_vm/mod.rs +++ b/phd-tests/framework/src/test_vm/mod.rs @@ -976,10 +976,20 @@ async fn try_ensure_vm_destroyed(client: &Client) { .send() .await { - error!( - %error, - "error stopping VM to move it to Destroyed" - ); + // If the put fails because the instance was already run down, there's + // nothing else to do. If it fails for some other reason, there's + // nothing else that *can* be done, but the error is unusual and should + // be logged. + match error.status() { + Some(http::status::StatusCode::FAILED_DEPENDENCY) => {} + _ => { + error!( + %error, + "error stopping VM to move it to Destroyed" + ); + } + } + return; } diff --git a/phd-tests/tests/src/migrate.rs b/phd-tests/tests/src/migrate.rs index b234bd4cf..c059ecc76 100644 --- a/phd-tests/tests/src/migrate.rs +++ b/phd-tests/tests/src/migrate.rs @@ -161,6 +161,15 @@ mod running_process { .unwrap_err(); info!(%error, "first migration failed as expected"); + // Also verify that the target reports that it failed. + let target_migration_state = target1 + .get_migration_state() + .await? + .migration_in + .expect("target should have a migration-in status") + .state; + assert_eq!(target_migration_state, MigrationState::Error); + // try again. this time, it should work! target2 .migrate_from(&source, Uuid::new_v4(), MigrationTimeout::default()) @@ -205,6 +214,15 @@ mod running_process { .unwrap_err(); info!(%error, "first migration failed as expected"); + // Also verify that the target reports that it failed. + let target_migration_state = target1 + .get_migration_state() + .await? + .migration_in + .expect("target should have a migration-in status") + .state; + assert_eq!(target_migration_state, MigrationState::Error); + // try again. this time, it should work! target2 .migrate_from(&source, Uuid::new_v4(), MigrationTimeout::default())