From aba9f19f583d3c0f60713973a5da15d9b8e8fc8d Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Thu, 20 Jun 2024 11:50:03 -0700 Subject: [PATCH] remove most of `instance_set_migration_ids` --- clients/sled-agent-client/src/lib.rs | 40 +++++ nexus/src/app/instance.rs | 73 +-------- nexus/src/app/sagas/instance_migrate.rs | 22 +-- nexus/tests/integration_tests/instances.rs | 75 ++++++++- sled-agent/src/common/instance.rs | 30 +--- sled-agent/src/http_entrypoints.rs | 19 +-- sled-agent/src/instance.rs | 50 +----- sled-agent/src/instance_manager.rs | 53 +------ sled-agent/src/sim/http_entrypoints.rs | 35 +++-- sled-agent/src/sim/instance.rs | 173 +++++++++++---------- sled-agent/src/sim/sled_agent.rs | 55 ++++--- sled-agent/src/sled_agent.rs | 25 +-- 12 files changed, 294 insertions(+), 356 deletions(-) diff --git a/clients/sled-agent-client/src/lib.rs b/clients/sled-agent-client/src/lib.rs index 29c94936005..f43ee1f18d2 100644 --- a/clients/sled-agent-client/src/lib.rs +++ b/clients/sled-agent-client/src/lib.rs @@ -7,6 +7,9 @@ use anyhow::Context; use async_trait::async_trait; use omicron_common::api::internal::shared::NetworkInterface; +use schemars::JsonSchema; +use serde::Deserialize; +use serde::Serialize; use std::convert::TryFrom; use std::fmt; use std::hash::Hash; @@ -571,6 +574,11 @@ impl From #[async_trait] pub trait TestInterfaces { async fn instance_finish_transition(&self, id: Uuid); + async fn instance_simulate_migration_source( + &self, + id: Uuid, + params: SimulateMigrationSource, + ); async fn disk_finish_transition(&self, id: Uuid); } @@ -597,4 +605,36 @@ impl TestInterfaces for Client { .await .expect("disk_finish_transition() failed unexpectedly"); } + + async fn instance_simulate_migration_source( + &self, + id: Uuid, + params: SimulateMigrationSource, + ) { + let baseurl = self.baseurl(); + let client = self.client(); + let url = format!("{baseurl}/instances/{id}/sim-migration-source"); + client + .post(url) + .send() + .await + .expect("instance_simulate_migration_source() failed unexpectedly"); + } +} + +// N.B. that this needs to be kept in sync with the types defined in +// `sled_agent::sim`! AFAICT this is the first simulated-only interface that has +// a body, so I wasn't sure whether there was a nice way to do this without +// creating a cyclic dependency or taking a giant pile of query params instead +// of JSON... +#[derive(Serialize, Deserialize, JsonSchema)] +pub struct SimulateMigrationSource { + pub migration_id: Uuid, + pub result: SimulatedMigrationResult, +} + +#[derive(Serialize, Deserialize, JsonSchema)] +pub enum SimulatedMigrationResult { + Success, + Failure, } diff --git a/nexus/src/app/instance.rs b/nexus/src/app/instance.rs index ce6a7a94795..232f59df879 100644 --- a/nexus/src/app/instance.rs +++ b/nexus/src/app/instance.rs @@ -538,7 +538,7 @@ impl super::Nexus { &self, opctx: &OpContext, instance_id: InstanceUuid, - sled_id: SledUuid, + src_propolis_id: PropolisUuid, prev_instance_runtime: &db::model::InstanceRuntimeState, migration_params: InstanceMigrationSourceParams, ) -> UpdateResult { @@ -550,42 +550,7 @@ impl super::Nexus { .lookup_for(authz::Action::Modify) .await?; - let sa = self.sled_client(&sled_id).await?; - let instance_put_result = sa - .instance_put_migration_ids( - &instance_id, - &InstancePutMigrationIdsBody { - old_runtime: prev_instance_runtime.clone().into(), - migration_params: Some(migration_params), - }, - ) - .await - .map(|res| Some(res.into_inner().into())) - .map_err(|e| SledAgentInstancePutError(e)); - - // Write the updated instance runtime state back to CRDB. If this - // outright fails, this operation fails. If the operation nominally - // succeeds but nothing was updated, this action is outdated and the - // caller should not proceed with migration. - let InstanceUpdateResult { instance_updated, .. } = - match instance_put_result { - Ok(state) => { - self.write_returned_instance_state(&instance_id, state) - .await? - } - Err(e) => { - if e.instance_unhealthy() { - let _ = self - .mark_instance_failed( - &instance_id, - &prev_instance_runtime, - &e, - ) - .await; - } - return Err(e.into()); - } - }; + let instance_updated = todo!("eliza: do this transition purely in nexus rather than in sled-agent..."); if instance_updated { Ok(self @@ -617,44 +582,12 @@ impl super::Nexus { pub(crate) async fn instance_clear_migration_ids( &self, instance_id: InstanceUuid, - sled_id: SledUuid, prev_instance_runtime: &db::model::InstanceRuntimeState, ) -> Result<(), Error> { assert!(prev_instance_runtime.migration_id.is_some()); assert!(prev_instance_runtime.dst_propolis_id.is_some()); - let sa = self.sled_client(&sled_id).await?; - let instance_put_result = sa - .instance_put_migration_ids( - &instance_id, - &InstancePutMigrationIdsBody { - old_runtime: prev_instance_runtime.clone().into(), - migration_params: None, - }, - ) - .await - .map(|res| Some(res.into_inner().into())) - .map_err(|e| SledAgentInstancePutError(e)); - - match instance_put_result { - Ok(state) => { - self.write_returned_instance_state(&instance_id, state).await?; - } - Err(e) => { - if e.instance_unhealthy() { - let _ = self - .mark_instance_failed( - &instance_id, - &prev_instance_runtime, - &e, - ) - .await; - } - return Err(e.into()); - } - } - - Ok(()) + todo!("eliza: do this transition in the DB rather than in sled-agent") } /// Reboot the specified instance. diff --git a/nexus/src/app/sagas/instance_migrate.rs b/nexus/src/app/sagas/instance_migrate.rs index 3546642bbb3..c393209bc72 100644 --- a/nexus/src/app/sagas/instance_migrate.rs +++ b/nexus/src/app/sagas/instance_migrate.rs @@ -77,14 +77,7 @@ declare_saga_actions! { // This step the instance's migration ID and destination Propolis ID - // fields. Because the instance is active, its current sled agent maintains - // its most recent runtime state, so to update it, the saga calls into the - // sled and asks it to produce an updated instance record with the - // appropriate migration IDs and a new generation number. - // - // The source sled agent synchronizes concurrent attempts to set these IDs. - // Setting a new migration ID and re-setting an existing ID are allowed, but - // trying to set an ID when a different ID is already present fails. + // fields. SET_MIGRATION_IDS -> "set_migration_ids" { + sim_set_migration_ids - sim_clear_migration_ids @@ -322,14 +315,15 @@ async fn sim_set_migration_ids( let db_instance = ¶ms.instance; let instance_id = InstanceUuid::from_untyped_uuid(db_instance.id()); - let src_sled_id = SledUuid::from_untyped_uuid(params.src_vmm.sled_id); + let src_propolis_id = + PropolisUuid::from_untyped_uuid(params.src_vmm.sled_id); let migration_id = sagactx.lookup::("migrate_id")?; let dst_propolis_id = sagactx.lookup::("dst_propolis_id")?; info!(osagactx.log(), "setting migration IDs on migration source sled"; "instance_id" => %db_instance.id(), - "sled_id" => %src_sled_id, "migration_id" => %migration_id, + "src_propolis_id" => %src_propolis_id, "dst_propolis_id" => %dst_propolis_id, "prev_runtime_state" => ?db_instance.runtime()); @@ -338,7 +332,7 @@ async fn sim_set_migration_ids( .instance_set_migration_ids( &opctx, instance_id, - src_sled_id, + src_propolis_id, db_instance.runtime(), InstanceMigrationSourceParams { dst_propolis_id, migration_id }, ) @@ -377,11 +371,7 @@ async fn sim_clear_migration_ids( // as failed. if let Err(e) = osagactx .nexus() - .instance_clear_migration_ids( - instance_id, - src_sled_id, - db_instance.runtime(), - ) + .instance_clear_migration_ids(instance_id, db_instance.runtime()) .await { warn!(osagactx.log(), diff --git a/nexus/tests/integration_tests/instances.rs b/nexus/tests/integration_tests/instances.rs index 4f7a1d1b771..4ad12046af2 100644 --- a/nexus/tests/integration_tests/instances.rs +++ b/nexus/tests/integration_tests/instances.rs @@ -813,10 +813,18 @@ async fn test_instance_migrate(cptestctx: &ControlPlaneTestContext) { // sufficient to move the instance back into a Running state (strictly // speaking no further updates from the source are required if the target // successfully takes over). - instance_simulate_on_sled(cptestctx, nexus, dst_sled_id, instance_id).await; + instance_simulate_migration_source( + cptestctx, + nexus, + original_sled, + instance_id, + migration_id, + ) + .await; // Ensure that both sled agents report that the migration has completed. instance_simulate_on_sled(cptestctx, nexus, original_sled, instance_id) .await; + instance_simulate_on_sled(cptestctx, nexus, dst_sled_id, instance_id).await; let instance = instance_get(&client, &instance_url).await; assert_eq!(instance.runtime.run_state, InstanceState::Running); @@ -943,8 +951,40 @@ async fn test_instance_migrate_v2p(cptestctx: &ControlPlaneTestContext) { .parsed_body::() .unwrap(); + let migration_id = { + let datastore = apictx.nexus.datastore(); + let opctx = OpContext::for_tests( + cptestctx.logctx.log.new(o!()), + datastore.clone(), + ); + let (.., authz_instance) = LookupPath::new(&opctx, &datastore) + .instance_id(instance.identity.id) + .lookup_for(nexus_db_queries::authz::Action::Read) + .await + .unwrap(); + datastore + .instance_refetch(&opctx, &authz_instance) + .await + .unwrap() + .runtime_state + .migration_id + .expect("since we've started a migration, the instance record must have a migration id!") + }; + + // Tell both sled-agents to pretend to do the migration. + instance_simulate_migration_source( + cptestctx, + nexus, + original_sled, + instance_id, + migration_id, + ) + .await; + instance_simulate_on_sled(cptestctx, nexus, original_sled_id, instance_id) + .await; instance_simulate_on_sled(cptestctx, nexus, dst_sled_id, instance_id).await; let instance = instance_get(&client, &instance_url).await; + assert_eq!(instance.runtime.run_state, InstanceState::Running); let current_sled = nexus .instance_sled_id(&instance_id) @@ -4804,3 +4844,36 @@ async fn instance_simulate_on_sled( let sa = nexus.sled_client(&sled_id).await.unwrap(); sa.instance_finish_transition(instance_id.into_untyped_uuid()).await; } + +/// Simulates a migration source for the provided instance ID, sled ID, and +/// migration ID. +// +// XXX(eliza): I had really wanted to have the migration target's simulated +// sled-agent do this automagically when it's told to start a migration in, but +// unfortunately, I wasn't able to figure out a way for it to get the simulated +// *sled-agent*'s IP --- it just gets the Propolis IP in the migration target +// params, and the propolis doesn't actually exist... +async fn instance_simulate_migration_source( + cptestctx: &ControlPlaneTestContext, + nexus: &Arc, + sled_id: SledUuid, + instance_id: InstanceUuid, + migration_id: Uuid, +) { + info!( + &cptestctx.logctx.log, + "Simulating migration source sled"; + "instance_id" => %instance_id, + "sled_id" => %sled_id, + "migration_id" => %migration_id, + ); + let sa = nexus.sled_client(&sled_id).await.unwrap(); + sa.instance_simulate_migrationSource( + instance_id.into_untyped_uuid(), + sled_agent_client::SimulateMigrationSource { + migration_id, + result: sled_agent_client::SimulatedMigrationResult::Success, + }, + ) + .await; +} diff --git a/sled-agent/src/common/instance.rs b/sled-agent/src/common/instance.rs index d8bffe929b3..37f6182f51e 100644 --- a/sled-agent/src/common/instance.rs +++ b/sled-agent/src/common/instance.rs @@ -4,7 +4,6 @@ //! Describes the states of VM instances. -use crate::params::InstanceMigrationSourceParams; use chrono::{DateTime, Utc}; use omicron_common::api::external::Generation; use omicron_common::api::internal::nexus::{ @@ -191,6 +190,14 @@ impl InstanceStates { self.propolis_id } + pub fn migration_in(&self) -> Option<&MigrationRuntimeState> { + self.migration_in.as_ref() + } + + pub fn migration_out(&self) -> Option<&MigrationRuntimeState> { + self.migration_out.as_ref() + } + /// Creates a `SledInstanceState` structure containing the entirety of this /// structure's runtime state. This requires cloning; for simple read access /// use the `instance` or `vmm` accessors instead. @@ -342,27 +349,6 @@ impl InstanceStates { self.apply_propolis_observation(&fake_observed); } - - /// Sets or clears this instance's migration IDs and advances its Propolis - /// generation number. - #[deprecated(note = "eliza get rid of this")] - pub(crate) fn set_migration_ids( - &mut self, - ids: &Option, - now: DateTime, - ) { - } - - /// Returns true if the migration IDs in this instance are already set as they - /// would be on a successful transition from the migration IDs in - /// `old_runtime` to the ones in `migration_ids`. - #[deprecated(note = "eliza get rid of this")] - pub(crate) fn migration_ids_already_set( - &self, - migration_ids: &Option, - ) -> bool { - false - } } #[cfg(test)] diff --git a/sled-agent/src/http_entrypoints.rs b/sled-agent/src/http_entrypoints.rs index 6defd18a951..fce21e56ae0 100644 --- a/sled-agent/src/http_entrypoints.rs +++ b/sled-agent/src/http_entrypoints.rs @@ -492,20 +492,13 @@ async fn instance_get_state( path = "/instances/{instance_id}/migration-ids", }] async fn instance_put_migration_ids( - rqctx: RequestContext, - path_params: Path, - body: TypedBody, + _: RequestContext, + _: Path, + _: TypedBody, ) -> Result, HttpError> { - let sa = rqctx.context(); - let instance_id = path_params.into_inner().instance_id; - let body_args = body.into_inner(); - Ok(HttpResponseOk( - sa.instance_put_migration_ids( - instance_id, - &body_args.old_runtime, - &body_args.migration_params, - ) - .await?, + Err(HttpError::for_bad_request( + None, + "operation no longer supported".to_string(), )) } diff --git a/sled-agent/src/instance.rs b/sled-agent/src/instance.rs index d0601df19b0..3a6d6481ab7 100644 --- a/sled-agent/src/instance.rs +++ b/sled-agent/src/instance.rs @@ -15,9 +15,9 @@ use crate::nexus::NexusClientWithResolver; use crate::params::ZoneBundleMetadata; use crate::params::{InstanceExternalIpBody, ZoneBundleCause}; use crate::params::{ - InstanceHardware, InstanceMetadata, InstanceMigrationSourceParams, - InstanceMigrationTargetParams, InstancePutStateResponse, - InstanceStateRequested, InstanceUnregisterResponse, VpcFirewallRule, + InstanceHardware, InstanceMetadata, InstanceMigrationTargetParams, + InstancePutStateResponse, InstanceStateRequested, + InstanceUnregisterResponse, VpcFirewallRule, }; use crate::profile::*; use crate::zone_bundle::BundleError; @@ -33,7 +33,7 @@ use illumos_utils::svc::wait_for_service; use illumos_utils::zone::PROPOLIS_ZONE_PREFIX; use omicron_common::address::NEXUS_INTERNAL_PORT; use omicron_common::api::internal::nexus::{ - InstanceRuntimeState, SledInstanceState, VmmRuntimeState, + SledInstanceState, VmmRuntimeState, }; use omicron_common::api::internal::shared::{ NetworkInterface, SourceNatConfig, @@ -221,11 +221,6 @@ enum InstanceRequest { state: crate::params::InstanceStateRequested, tx: oneshot::Sender>, }, - PutMigrationIds { - old_runtime: InstanceRuntimeState, - migration_ids: Option, - tx: oneshot::Sender>, - }, Terminate { tx: oneshot::Sender>, }, @@ -412,12 +407,6 @@ impl InstanceRunner { .map_err(|e| e.into())) .map_err(|_| Error::FailedSendClientClosed) }, - Some(PutMigrationIds{ old_runtime, migration_ids, tx }) => { - tx.send( - self.put_migration_ids(&migration_ids).await.map_err(|e| e.into()) - ) - .map_err(|_| Error::FailedSendClientClosed) - }, Some(Terminate { tx }) => { tx.send(Ok(InstanceUnregisterResponse { updated_runtime: Some(self.terminate().await) @@ -1073,23 +1062,6 @@ impl Instance { Ok(()) } - pub async fn put_migration_ids( - &self, - tx: oneshot::Sender>, - old_runtime: InstanceRuntimeState, - migration_ids: Option, - ) -> Result<(), Error> { - self.tx - .send(InstanceRequest::PutMigrationIds { - old_runtime, - migration_ids, - tx, - }) - .await - .map_err(|_| Error::FailedSendChannelClosed)?; - Ok(()) - } - /// Rudely terminates this instance's Propolis (if it has one) and /// immediately transitions the instance to the Destroyed state. pub async fn terminate( @@ -1266,20 +1238,6 @@ impl InstanceRunner { Ok(self.state.sled_instance_state()) } - async fn put_migration_ids( - &mut self, - migration_ids: &Option, - ) -> Result { - // Allow this transition for idempotency if the instance is - // already in the requested goal state. - if self.state.migration_ids_already_set(migration_ids) { - return Ok(self.state.sled_instance_state()); - } - - self.state.set_migration_ids(migration_ids, Utc::now()); - Ok(self.state.sled_instance_state()) - } - async fn setup_propolis_inner(&mut self) -> Result { // Create OPTE ports for the instance let mut opte_ports = Vec::with_capacity(self.requested_nics.len()); diff --git a/sled-agent/src/instance_manager.rs b/sled-agent/src/instance_manager.rs index 7128cb1ce3f..04cc984cd38 100644 --- a/sled-agent/src/instance_manager.rs +++ b/sled-agent/src/instance_manager.rs @@ -11,8 +11,8 @@ use crate::params::InstanceExternalIpBody; use crate::params::InstanceMetadata; use crate::params::ZoneBundleMetadata; use crate::params::{ - InstanceHardware, InstanceMigrationSourceParams, InstancePutStateResponse, - InstanceStateRequested, InstanceUnregisterResponse, + InstanceHardware, InstancePutStateResponse, InstanceStateRequested, + InstanceUnregisterResponse, }; use crate::vmm_reservoir::VmmReservoirManagerHandle; use crate::zone_bundle::BundleError; @@ -215,26 +215,6 @@ impl InstanceManager { } } - pub async fn put_migration_ids( - &self, - instance_id: InstanceUuid, - old_runtime: &InstanceRuntimeState, - migration_ids: &Option, - ) -> Result { - let (tx, rx) = oneshot::channel(); - self.inner - .tx - .send(InstanceManagerRequest::PutMigrationIds { - instance_id, - old_runtime: old_runtime.clone(), - migration_ids: *migration_ids, - tx, - }) - .await - .map_err(|_| Error::FailedSendInstanceManagerClosed)?; - rx.await? - } - pub async fn instance_issue_disk_snapshot_request( &self, instance_id: InstanceUuid, @@ -354,12 +334,7 @@ enum InstanceManagerRequest { target: InstanceStateRequested, tx: oneshot::Sender>, }, - PutMigrationIds { - instance_id: InstanceUuid, - old_runtime: InstanceRuntimeState, - migration_ids: Option, - tx: oneshot::Sender>, - }, + InstanceIssueDiskSnapshot { instance_id: InstanceUuid, disk_id: Uuid, @@ -471,9 +446,6 @@ impl InstanceManagerRunner { Some(EnsureState { instance_id, target, tx }) => { self.ensure_state(tx, instance_id, target).await }, - Some(PutMigrationIds { instance_id, old_runtime, migration_ids, tx }) => { - self.put_migration_ids(tx, instance_id, &old_runtime, &migration_ids).await - }, Some(InstanceIssueDiskSnapshot { instance_id, disk_id, snapshot_id, tx }) => { self.instance_issue_disk_snapshot_request(tx, instance_id, disk_id, snapshot_id).await }, @@ -676,25 +648,6 @@ impl InstanceManagerRunner { Ok(()) } - /// Idempotently attempts to set the instance's migration IDs to the - /// supplied IDs. - async fn put_migration_ids( - &mut self, - tx: oneshot::Sender>, - instance_id: InstanceUuid, - old_runtime: &InstanceRuntimeState, - migration_ids: &Option, - ) -> Result<(), Error> { - let (_, instance) = self - .instances - .get(&instance_id) - .ok_or_else(|| Error::NoSuchInstance(instance_id))?; - instance - .put_migration_ids(tx, old_runtime.clone(), *migration_ids) - .await?; - Ok(()) - } - async fn instance_issue_disk_snapshot_request( &self, tx: oneshot::Sender>, diff --git a/sled-agent/src/sim/http_entrypoints.rs b/sled-agent/src/sim/http_entrypoints.rs index 012889c664f..b518692211c 100644 --- a/sled-agent/src/sim/http_entrypoints.rs +++ b/sled-agent/src/sim/http_entrypoints.rs @@ -154,20 +154,13 @@ async fn instance_get_state( path = "/instances/{instance_id}/migration-ids", }] async fn instance_put_migration_ids( - rqctx: RequestContext>, - path_params: Path, - body: TypedBody, + _: RequestContext>, + _: Path, + _: TypedBody, ) -> Result, HttpError> { - let sa = rqctx.context(); - let instance_id = path_params.into_inner().instance_id; - let body_args = body.into_inner(); - Ok(HttpResponseOk( - sa.instance_put_migration_ids( - instance_id, - &body_args.old_runtime, - &body_args.migration_params, - ) - .await?, + Err(HttpError::for_bad_request( + None, + "operation no longer supported".to_string(), )) } @@ -217,6 +210,22 @@ async fn instance_poke_post( Ok(HttpResponseUpdatedNoContent()) } +#[endpoint { + method = POST, + path = "/instances/{instance_id}/sim-migration-source", +}] +async fn instance_post_sim_migration_source( + rqctx: RequestContext>, + path_params: Path, + body: TypedBody, +) -> Result { + let sa = rqctx.context(); + let instance_id = path_params.into_inner().instance_id; + sa.instance_simulate_migration_source(instance_id, body.into_inner()) + .await?; + Ok(HttpResponseUpdatedNoContent()) +} + /// Path parameters for Disk requests (sled agent API) #[derive(Deserialize, JsonSchema)] struct DiskPathParam { diff --git a/sled-agent/src/sim/instance.rs b/sled-agent/src/sim/instance.rs index b99b50807c1..2dcdd5034a1 100644 --- a/sled-agent/src/sim/instance.rs +++ b/sled-agent/src/sim/instance.rs @@ -8,21 +8,22 @@ use super::simulatable::Simulatable; use crate::common::instance::{ObservedPropolisState, PublishedVmmState}; use crate::nexus::NexusClient; -use crate::params::{InstanceMigrationSourceParams, InstanceStateRequested}; +use crate::params::InstanceStateRequested; use async_trait::async_trait; use chrono::Utc; use nexus_client; use omicron_common::api::external::Error; use omicron_common::api::external::Generation; use omicron_common::api::external::ResourceType; -use omicron_common::api::internal::nexus::{ - InstanceRuntimeState, SledInstanceState, VmmState, -}; +use omicron_common::api::internal::nexus::{SledInstanceState, VmmState}; use propolis_client::types::{ InstanceMigrateStatusResponse as PropolisMigrateResponse, InstanceMigrationStatus as PropolisMigrationStatus, InstanceState as PropolisInstanceState, InstanceStateMonitorResponse, }; +use schemars::JsonSchema; +use serde::Deserialize; +use serde::Serialize; use std::collections::VecDeque; use std::sync::Arc; use std::sync::Mutex; @@ -30,6 +31,18 @@ use uuid::Uuid; use crate::common::instance::{Action as InstanceAction, InstanceStates}; +#[derive(Serialize, Deserialize, JsonSchema)] +pub struct SimulateMigrationSource { + pub(in crate::sim) migration_id: Uuid, + pub(in crate::sim) result: SimulatedMigrationResult, +} + +#[derive(Serialize, Deserialize, JsonSchema)] +pub(in crate::sim) enum SimulatedMigrationResult { + Success, + Failure, +} + #[derive(Clone, Debug)] enum MonitorChange { PropolisState(PropolisInstanceState), @@ -79,49 +92,68 @@ impl SimInstanceInner { self.queue.push_back(MonitorChange::MigrateStatus(migrate_status)) } - /// Queue a successful simulated migration. - /// - fn queue_successful_migration(&mut self) { + /// Queue a simulated migration out. + fn queue_migration_out( + &mut self, + migration_id: Uuid, + result: SimulatedMigrationResult, + ) { + let migration_update = |state| PropolisMigrateResponse { + migration_in: None, + migration_out: Some(PropolisMigrationStatus { + id: migration_id, + state, + }), + }; + // Propolis transitions to the Migrating state once before + // actually starting migration. + self.queue_propolis_state(PropolisInstanceState::Migrating); + self.queue_migration_update(migration_update( + propolis_client::types::MigrationState::Sync, + )); + match result { + SimulatedMigrationResult::Success => { + self.queue_migration_update(migration_update( + propolis_client::types::MigrationState::Finish, + )); + self.queue_graceful_stop(); + } + SimulatedMigrationResult::Failure => { + todo!("finish this part when we actuall need it...") + } + } + } + + /// Queue a simulated migration in. + fn queue_migration_in( + &mut self, + migration_id: Uuid, + result: SimulatedMigrationResult, + ) { + let migration_update = |state| PropolisMigrateResponse { + migration_in: Some(PropolisMigrationStatus { + id: migration_id, + state, + }), + migration_out: None, + }; // Propolis transitions to the Migrating state once before // actually starting migration. self.queue_propolis_state(PropolisInstanceState::Migrating); - todo!("eliza: fix this bit") - // match role { - // MigrationRole::Source => { - // self.queue_migration_update(PropolisMigrateResponse { - // migration_in: None, - // migration_out: Some(PropolisMigrationStatus { - // id: todo! - // state: propolis_client::types::MigrationState::Sync, - // }), - // }); - // self.queue_migration_update(PropolisMigrateResponse { - // migration_in: None, - // migration_out: Some(PropolisMigrationStatus { - // id: migration_id, - // state: propolis_client::types::MigrationState::Finish, - // }), - // }); - // self.queue_graceful_stop(); - // } - // MigrationRole::Target => { - // self.queue_migration_update(PropolisMigrateResponse { - // migration_in: Some(PropolisMigrationStatus { - // id: migration_id, - // state: propolis_client::types::MigrationState::Sync, - // }), - // migration_out: None, - // }); - // self.queue_migration_update(PropolisMigrateResponse { - // migration_in: Some(PropolisMigrationStatus { - // id: migration_id, - // state: propolis_client::types::MigrationState::Finish, - // }), - // migration_out: None, - // }); - // self.queue_propolis_state(PropolisInstanceState::Running) - // } - // } + self.queue_migration_update(migration_update( + propolis_client::types::MigrationState::Sync, + )); + match result { + SimulatedMigrationResult::Success => { + self.queue_migration_update(migration_update( + propolis_client::types::MigrationState::Finish, + )); + self.queue_propolis_state(PropolisInstanceState::Running) + } + SimulatedMigrationResult::Failure => { + todo!("finish this part when we actuall need it...") + } + } } fn queue_graceful_stop(&mut self) { @@ -171,7 +203,13 @@ impl SimInstanceInner { ))); } - // self.queue_successful_migration(MigrationRole::Target) + let migration_id = self.state.migration_out() + .ok_or_else(|| Error::invalid_request("can't request migration in for a vmm that wasn't created with a migration ID"))? + .migration_id; + self.queue_migration_in( + migration_id, + SimulatedMigrationResult::Success, + ); } InstanceStateRequested::Running => { match self.next_resting_state() { @@ -359,38 +397,6 @@ impl SimInstanceInner { self.destroyed = true; self.state.sled_instance_state() } - - /// Stores a set of migration IDs in the instance's runtime state. - fn put_migration_ids( - &mut self, - ids: &Option, - ) -> Result { - if self.state.migration_ids_already_set(ids) { - return Ok(self.state.sled_instance_state()); - } - - self.state.set_migration_ids(ids, Utc::now()); - - // If we set migration IDs and are the migration source, ensure that we - // will perform the correct state transitions to simulate a successful - // migration. - // if ids.is_some() { - // let role = self - // .state - // .migration() - // .expect( - // "we just got a `put_migration_ids` request with `Some` IDs, \ - // so we should have a migration" - // ) - // .role; - // if role == MigrationRole::Source { - // self.queue_successful_migration(MigrationRole::Source) - // } - // } - todo!(); - - Ok(self.state.sled_instance_state()) - } } /// A simulation of an Instance created by the external Oxide API. @@ -418,13 +424,14 @@ impl SimInstance { self.inner.lock().unwrap().terminate() } - pub async fn put_migration_ids( + pub(crate) fn set_simulated_migration_source( &self, - old_runtime: &InstanceRuntimeState, - ids: &Option, - ) -> Result { - let mut inner = self.inner.lock().unwrap(); - inner.put_migration_ids(ids) + migration: SimulateMigrationSource, + ) { + self.inner + .lock() + .unwrap() + .queue_migration_in(migration.migration_id, migration.result); } } diff --git a/sled-agent/src/sim/sled_agent.rs b/sled-agent/src/sim/sled_agent.rs index 8a12e8c5d79..51fd6410601 100644 --- a/sled-agent/src/sim/sled_agent.rs +++ b/sled-agent/src/sim/sled_agent.rs @@ -7,7 +7,7 @@ use super::collection::{PokeMode, SimCollection}; use super::config::Config; use super::disk::SimDisk; -use super::instance::SimInstance; +use super::instance::{self, SimInstance}; use super::storage::CrucibleData; use super::storage::Storage; use crate::bootstrap::early_networking::{ @@ -16,9 +16,9 @@ use crate::bootstrap::early_networking::{ use crate::nexus::NexusClient; use crate::params::{ DiskStateRequested, InstanceExternalIpBody, InstanceHardware, - InstanceMetadata, InstanceMigrationSourceParams, InstancePutStateResponse, - InstanceStateRequested, InstanceUnregisterResponse, Inventory, - OmicronPhysicalDisksConfig, OmicronZonesConfig, SledRole, + InstanceMetadata, InstancePutStateResponse, InstanceStateRequested, + InstanceUnregisterResponse, Inventory, OmicronPhysicalDisksConfig, + OmicronZonesConfig, SledRole, }; use crate::sim::simulatable::Simulatable; use crate::updates::UpdateManager; @@ -31,7 +31,7 @@ use omicron_common::api::external::{ ByteCount, DiskState, Error, Generation, ResourceType, }; use omicron_common::api::internal::nexus::{ - DiskRuntimeState, SledInstanceState, + DiskRuntimeState, MigrationRuntimeState, MigrationState, SledInstanceState, }; use omicron_common::api::internal::nexus::{ InstanceRuntimeState, VmmRuntimeState, @@ -256,7 +256,7 @@ impl SledAgent { instance_id: InstanceUuid, propolis_id: PropolisUuid, hardware: InstanceHardware, - _instance_runtime: InstanceRuntimeState, + instance_runtime: InstanceRuntimeState, vmm_runtime: VmmRuntimeState, metadata: InstanceMetadata, ) -> Result { @@ -341,6 +341,15 @@ impl SledAgent { } } + let migration_in = instance_runtime.migration_id.map(|migration_id| { + MigrationRuntimeState { + migration_id, + state: MigrationState::Pending, + gen: Generation::new(), + time_updated: chrono::Utc::now(), + } + }); + let instance_run_time_state = self .instances .sim_ensure( @@ -348,7 +357,7 @@ impl SledAgent { SledInstanceState { vmm_state: vmm_runtime, propolis_id, - migration_in: None, + migration_in, migration_out: None, }, None, @@ -501,6 +510,24 @@ impl SledAgent { Ok(instance.current()) } + pub async fn instance_simulate_migration_source( + &self, + instance_id: InstanceUuid, + migration: instance::SimulateMigrationSource, + ) -> Result<(), HttpError> { + let instance = self + .instances + .sim_get_cloned_object(&instance_id.into_untyped_uuid()) + .await + .map_err(|_| { + crate::sled_agent::Error::Instance( + crate::instance_manager::Error::NoSuchInstance(instance_id), + ) + })?; + instance.set_simulated_migration_source(migration); + Ok(()) + } + pub async fn set_instance_ensure_state_error(&self, error: Option) { *self.instance_ensure_state_error.lock().await = error; } @@ -524,20 +551,6 @@ impl SledAgent { Ok(()) } - pub async fn instance_put_migration_ids( - self: &Arc, - instance_id: InstanceUuid, - old_runtime: &InstanceRuntimeState, - migration_ids: &Option, - ) -> Result { - let instance = self - .instances - .sim_get_cloned_object(&instance_id.into_untyped_uuid()) - .await?; - - instance.put_migration_ids(old_runtime, migration_ids).await - } - /// Idempotently ensures that the given API Disk (described by `api_disk`) /// is attached (or not) as specified. This simulates disk attach and /// detach, similar to instance boot and halt. diff --git a/sled-agent/src/sled_agent.rs b/sled-agent/src/sled_agent.rs index 993e5f6a94a..ecad23ab45b 100644 --- a/sled-agent/src/sled_agent.rs +++ b/sled-agent/src/sled_agent.rs @@ -20,10 +20,10 @@ use crate::nexus::{ }; use crate::params::{ DiskStateRequested, InstanceExternalIpBody, InstanceHardware, - InstanceMetadata, InstanceMigrationSourceParams, InstancePutStateResponse, - InstanceStateRequested, InstanceUnregisterResponse, Inventory, - OmicronPhysicalDisksConfig, OmicronZonesConfig, SledRole, TimeSync, - VpcFirewallRule, ZoneBundleMetadata, Zpool, + InstanceMetadata, InstancePutStateResponse, InstanceStateRequested, + InstanceUnregisterResponse, Inventory, OmicronPhysicalDisksConfig, + OmicronZonesConfig, SledRole, TimeSync, VpcFirewallRule, + ZoneBundleMetadata, Zpool, }; use crate::probe_manager::ProbeManager; use crate::services::{self, ServiceManager}; @@ -936,23 +936,6 @@ impl SledAgent { .map_err(|e| Error::Instance(e)) } - /// Idempotently ensures that the instance's runtime state contains the - /// supplied migration IDs, provided that the caller continues to meet the - /// conditions needed to change those IDs. See the doc comments for - /// [`crate::params::InstancePutMigrationIdsBody`]. - pub async fn instance_put_migration_ids( - &self, - instance_id: InstanceUuid, - old_runtime: &InstanceRuntimeState, - migration_ids: &Option, - ) -> Result { - self.inner - .instances - .put_migration_ids(instance_id, old_runtime, migration_ids) - .await - .map_err(|e| Error::Instance(e)) - } - /// Idempotently ensures that an instance's OPTE/port state includes the /// specified external IP address. ///