From 0ad25eb16e76d17d0bcb34599cb87f685b8a02e9 Mon Sep 17 00:00:00 2001 From: "Andrew J. Stone" Date: Sat, 23 Dec 2023 00:59:46 +0000 Subject: [PATCH 01/17] Add a background task for update plan execution This PR is the first step in creating a background task that is capable of taking a `Blueprint` and then reifying that blueprint into deployed or updated software. This PR uses the initial version of a Blueprint introduced in #4804. A basic executor that sends the related `OmicronZonesConfig` to the appropriate sled-agents for newly added sleds was created. A test is included that shows how a hypothetical planner for an `add-sled` workflow will deploy Omicron zones in a manner similar to RSS, where first the internal DNS zone is deployed and then the internal DNS and NTP zones are deployed. Deployment alwyas contains all zones expected to be running on the sled-agent. Any zones running that are not included are expected to be shut down. --- nexus/src/app/background/common.rs | 2 +- nexus/src/app/background/mod.rs | 1 + nexus/src/app/background/plan_execution.rs | 408 +++++++++++++++++++++ 3 files changed, 410 insertions(+), 1 deletion(-) create mode 100644 nexus/src/app/background/plan_execution.rs diff --git a/nexus/src/app/background/common.rs b/nexus/src/app/background/common.rs index 4fcce74714..f954a35639 100644 --- a/nexus/src/app/background/common.rs +++ b/nexus/src/app/background/common.rs @@ -408,7 +408,7 @@ impl TaskExec { start_time, start_instant, reason, - iteration: iteration, + iteration, }); }); diff --git a/nexus/src/app/background/mod.rs b/nexus/src/app/background/mod.rs index 70b20224d4..40f2549909 100644 --- a/nexus/src/app/background/mod.rs +++ b/nexus/src/app/background/mod.rs @@ -13,6 +13,7 @@ mod init; mod inventory_collection; mod nat_cleanup; mod phantom_disks; +mod plan_execution; mod status; pub use common::Driver; diff --git a/nexus/src/app/background/plan_execution.rs b/nexus/src/app/background/plan_execution.rs new file mode 100644 index 0000000000..c85d96abbe --- /dev/null +++ b/nexus/src/app/background/plan_execution.rs @@ -0,0 +1,408 @@ +// 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/. + +//! Background task for realizing a plan blueprint + +use super::common::BackgroundTask; +use anyhow::Context; +use futures::future::BoxFuture; +use futures::stream; +use futures::FutureExt; +use futures::StreamExt; +use nexus_db_queries::context::OpContext; +use nexus_db_queries::db::lookup::LookupPath; +use nexus_db_queries::db::DataStore; +use nexus_types::deployment::{Blueprint, OmicronZonesConfig}; +use serde_json::json; +use sled_agent_client::Client as SledAgentClient; +use slog::Logger; +use std::collections::BTreeMap; +use std::sync::Arc; +use tokio::sync::watch; +use uuid::Uuid; + +/// Background task that takes a [`Blueprint`] and realizes the change to +/// the state of the system based on the `Blueprint`. +pub struct PlanExecutor { + datastore: Arc, + rx_blueprint: watch::Receiver>>, +} + +impl PlanExecutor { + // Temporary until we wire up the background task + #[allow(unused)] + pub fn new( + datastore: Arc, + rx_blueprint: watch::Receiver>>, + ) -> PlanExecutor { + PlanExecutor { datastore, rx_blueprint } + } + + // This is a modified copy of the functionality from `nexus/src/app/sled.rs`. + // There's no good way to access this functionality right now since it is a + // method on the `Nexus` type. We want to have a more constrained type we can + // pass into background tasks for this type of functionality, but for now we + // just copy the functionality. + async fn sled_client( + &self, + opctx: &OpContext, + sled_id: &Uuid, + ) -> Result { + let (.., sled) = LookupPath::new(opctx, &self.datastore) + .sled_id(*sled_id) + .fetch() + .await + .with_context(|| { + format!( + "Failed to create sled_agent::Client for sled_id: {}", + sled_id + ) + })?; + let dur = std::time::Duration::from_secs(60); + let client = reqwest::ClientBuilder::new() + .connect_timeout(dur) + .timeout(dur) + .build() + .unwrap(); + Ok(SledAgentClient::new_with_client( + &format!("http://{}", sled.address()), + client, + opctx.log.clone(), + )) + } + + async fn realize_blueprint( + &self, + opctx: &OpContext, + blueprint: &std::sync::Arc, + ) -> Result<(), Vec> { + let log = opctx.log.new(o!("comment" => blueprint.comment.clone())); + self.deploy_zones(&log, opctx, &blueprint.omicron_zones).await + } + + async fn deploy_zones( + &self, + log: &Logger, + opctx: &OpContext, + zones: &BTreeMap, + ) -> Result<(), Vec> { + let errors: Vec<_> = stream::iter(zones.clone()) + .filter_map(|(sled_id, config)| async move { + let client = match self.sled_client(&opctx, &sled_id).await { + Ok(client) => client, + Err(err) => { + warn!(log, "{}", err); + return Some(err); + } + }; + let result = client + .omicron_zones_put(&config) + .await + .with_context(|| { + format!("Failed to put {config:#?} to sled {sled_id}") + }); + + match result { + Err(error) => { + warn!(log, "{}", error); + Some(error) + } + Ok(_) => { + info!( + log, + "Successfully deployed zones for sled agent"; + "sled_id" => %sled_id, + "generation" => config.generation.to_string() + ); + None + } + } + }) + .collect() + .await; + + if errors.is_empty() { + Ok(()) + } else { + Err(errors) + } + } +} + +impl BackgroundTask for PlanExecutor { + fn activate<'a>( + &'a mut self, + opctx: &'a OpContext, + ) -> BoxFuture<'a, serde_json::Value> { + async { + // Get the latest blueprint, cloning to prevent holding a read lock + // on the watch. + let blueprint = self.rx_blueprint.borrow().clone(); + + let Some(blueprint) = blueprint else { + warn!(&opctx.log, + "Plan execution: skipped"; + "reason" => "no blueprint"); + return json!({"error": "no blueprint" }); + }; + + let result = self.realize_blueprint(opctx, &blueprint).await; + + // Return the result as a `serde_json::Value` + match result { + Ok(()) => json!({}), + Err(errors) => { + let errors: Vec<_> = errors + .into_iter() + .map(|e| format!("{:#}", e)) + .collect(); + json!({"errors": errors}) + } + } + } + .boxed() + } +} +#[cfg(test)] +mod test { + use super::*; + use crate::app::background::common::BackgroundTask; + use httptest::matchers::{all_of, json_decoded, request}; + use httptest::responders::status_code; + use httptest::Expectation; + use nexus_db_model::{ + ByteCount, SledBaseboard, SledSystemHardware, SledUpdate, + }; + use nexus_test_utils_macros::nexus_test; + use nexus_types::inventory::{ + OmicronZoneConfig, OmicronZoneDataset, OmicronZoneType, + }; + use omicron_common::api::external::Generation; + use serde::Deserialize; + use std::collections::BTreeSet; + use std::net::SocketAddr; + + type ControlPlaneTestContext = + nexus_test_utils::ControlPlaneTestContext; + + fn create_blueprint( + omicron_zones: BTreeMap, + ) -> Blueprint { + Blueprint { + id: Uuid::new_v4(), + omicron_zones, + zones_in_service: BTreeSet::new(), + parent_blueprint_id: None, + time_created: chrono::Utc::now(), + creator: "test".to_string(), + comment: "test blueprint".to_string(), + } + } + + #[nexus_test(server = crate::Server)] + async fn test_deploy_omicron_zones(cptestctx: &ControlPlaneTestContext) { + let nexus = &cptestctx.server.apictx().nexus; + let datastore = nexus.datastore(); + let opctx = OpContext::for_tests( + cptestctx.logctx.log.clone(), + datastore.clone(), + ); + + let (blueprint_tx, blueprint_rx) = watch::channel(None); + let mut task = PlanExecutor::new(datastore.clone(), blueprint_rx); + + // With no blueprint we should fail with an appropriate message. + let value = task.activate(&opctx).await; + assert_eq!(value, json!({"error": "no blueprint"})); + + // Get a success (empty) result back when the blueprint has an empty set of zones + let blueprint = Arc::new(create_blueprint(BTreeMap::new())); + blueprint_tx.send(Some(blueprint)).unwrap(); + let value = task.activate(&opctx).await; + assert_eq!(value, json!({})); + + // Create some fake sled-agent servers to respond to zone puts and add + // sleds to CRDB. + let mut s1 = httptest::Server::run(); + let mut s2 = httptest::Server::run(); + let sled_id1 = Uuid::new_v4(); + let sled_id2 = Uuid::new_v4(); + let rack_id = Uuid::new_v4(); + for (i, (sled_id, server)) in + [(sled_id1, &s1), (sled_id2, &s2)].iter().enumerate() + { + let SocketAddr::V6(addr) = server.addr() else { + panic!("Expected Ipv6 address. Got {}", server.addr()); + }; + let update = SledUpdate::new( + *sled_id, + addr, + SledBaseboard { + serial_number: i.to_string(), + part_number: "test".into(), + revision: 1, + }, + SledSystemHardware { + is_scrimlet: false, + usable_hardware_threads: 4, + usable_physical_ram: ByteCount(1000.into()), + reservoir_size: ByteCount(999.into()), + }, + rack_id, + ); + datastore + .sled_upsert(update) + .await + .expect("Failed to insert sled to db"); + } + + // The particular dataset doesn't matter for this test. + // We re-use the same one to not obfuscate things + let dataset = OmicronZoneDataset { + pool_name: format!("oxp_{}", Uuid::new_v4()).parse().unwrap(), + }; + + let generation = Generation::new(); + + // Zones are updated in a particular order, but each request contains + // the full set of zones that must be running. + // See https://github.com/oxidecomputer/omicron/blob/main/sled-agent/src/rack_setup/service.rs#L976-L998 + // for more details. + let mut zones = OmicronZonesConfig { + generation, + zones: vec![OmicronZoneConfig { + id: Uuid::new_v4(), + underlay_address: "::1".parse().unwrap(), + zone_type: OmicronZoneType::InternalDns { + dataset, + dns_address: "oh-hello-internal-dns".into(), + gz_address: "::1".parse().unwrap(), + gz_address_index: 0, + http_address: "some-ipv6-address".into(), + }, + }], + }; + + // Create a blueprint with only the `InternalDns` zone for both servers + // We reuse the same `OmicronZonesConfig` because the details don't + // matter for this test. + let blueprint = Arc::new(create_blueprint(BTreeMap::from([ + (sled_id1, zones.clone()), + (sled_id2, zones.clone()), + ]))); + + // Send the blueprint with the first set of zones to the task + blueprint_tx.send(Some(blueprint)).unwrap(); + + // Check that the initial requests were sent to the fake sled-agents + for s in [&mut s1, &mut s2] { + s.expect( + Expectation::matching(all_of![ + request::method_path("PUT", "/omicron-zones",), + // Our generation number should be 1 and there should + // be only a single zone. + request::body(json_decoded(|c: &OmicronZonesConfig| { + c.generation == 1u32.into() && c.zones.len() == 1 + })) + ]) + .respond_with(status_code(204)), + ); + } + + // Activate the task to trigger zone configuration on the sled-agents + let value = task.activate(&opctx).await; + assert_eq!(value, json!({})); + s1.verify_and_clear(); + s2.verify_and_clear(); + + // Do it again. This should trigger the same request. + for s in [&mut s1, &mut s2] { + s.expect( + Expectation::matching(request::method_path( + "PUT", + "/omicron-zones", + )) + .respond_with(status_code(204)), + ); + } + let value = task.activate(&opctx).await; + assert_eq!(value, json!({})); + s1.verify_and_clear(); + s2.verify_and_clear(); + + // Take another lap, but this time, have one server fail the request and + // try again. + s1.expect( + Expectation::matching(request::method_path( + "PUT", + "/omicron-zones", + )) + .respond_with(status_code(204)), + ); + s2.expect( + Expectation::matching(request::method_path( + "PUT", + "/omicron-zones", + )) + .respond_with(status_code(500)), + ); + + // Define a type we can use to pick stuff out of error objects. + #[derive(Deserialize)] + struct ErrorResult { + errors: Vec, + } + + let value = task.activate(&opctx).await; + println!("{:?}", value); + let result: ErrorResult = serde_json::from_value(value).unwrap(); + assert_eq!(result.errors.len(), 1); + assert!( + result.errors[0].starts_with("Failed to put OmicronZonesConfig") + ); + s1.verify_and_clear(); + s2.verify_and_clear(); + + // Add an `InternalNtp` zone for our next update + zones.generation = generation.next(); + zones.zones.push(OmicronZoneConfig { + id: Uuid::new_v4(), + underlay_address: "::1".parse().unwrap(), + zone_type: OmicronZoneType::InternalNtp { + address: "::1".into(), + dns_servers: vec!["::1".parse().unwrap()], + domain: None, + ntp_servers: vec!["some-ntp-server-addr".into()], + }, + }); + + // Update our watch channel + let blueprint = Arc::new(create_blueprint(BTreeMap::from([ + (sled_id1, zones.clone()), + (sled_id2, zones.clone()), + ]))); + blueprint_tx.send(Some(blueprint)).unwrap(); + + // Set our new expectations + for s in [&mut s1, &mut s2] { + s.expect( + Expectation::matching(all_of![ + request::method_path("PUT", "/omicron-zones",), + // Our generation number should be bumped and there should + // be two zones. + request::body(json_decoded(|c: &OmicronZonesConfig| { + c.generation == 2u32.into() && c.zones.len() == 2 + })) + ]) + .respond_with(status_code(204)), + ); + } + + // Activate the task + let value = task.activate(&opctx).await; + assert_eq!(value, json!({})); + s1.verify_and_clear(); + s2.verify_and_clear(); + } +} From 0eced33bab77845a7b397d9a26c317f36edc6fad Mon Sep 17 00:00:00 2001 From: "Andrew J. Stone" Date: Thu, 25 Jan 2024 20:08:07 +0000 Subject: [PATCH 02/17] Add blueprint loader background task --- nexus/src/app/background/mod.rs | 1 + .../src/app/background/plan_blueprint_load.rs | 176 ++++++++++++++++++ nexus/src/app/deployment.rs | 4 +- 3 files changed, 179 insertions(+), 2 deletions(-) create mode 100644 nexus/src/app/background/plan_blueprint_load.rs diff --git a/nexus/src/app/background/mod.rs b/nexus/src/app/background/mod.rs index 40f2549909..3b5a9b3ad9 100644 --- a/nexus/src/app/background/mod.rs +++ b/nexus/src/app/background/mod.rs @@ -13,6 +13,7 @@ mod init; mod inventory_collection; mod nat_cleanup; mod phantom_disks; +mod plan_blueprint_load; mod plan_execution; mod status; diff --git a/nexus/src/app/background/plan_blueprint_load.rs b/nexus/src/app/background/plan_blueprint_load.rs new file mode 100644 index 0000000000..a8b7d56fd3 --- /dev/null +++ b/nexus/src/app/background/plan_blueprint_load.rs @@ -0,0 +1,176 @@ +// 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/. + +//! Background task for loading the target blueprint from the DB +//! +//! This task triggers the `plan_execution` background task when the blueprint +//! changes. + +use super::common::BackgroundTask; +use crate::app::deployment; +use futures::future::BoxFuture; +use futures::FutureExt; +use nexus_db_queries::authz; +use nexus_db_queries::authz::Action; +use nexus_db_queries::context::OpContext; +use nexus_db_queries::db::DataStore; +use nexus_types::deployment::Blueprint; +use serde_json::json; +use std::sync::Arc; +use tokio::sync::watch; + +pub struct TargetBlueprintLoader { + blueprints: std::sync::Mutex, + #[allow(unused)] + datastore: Arc, + last: Option>, + tx: watch::Sender>>, + rx: watch::Receiver>>, +} + +impl TargetBlueprintLoader { + pub fn new( + blueprints: std::sync::Mutex, + datastore: Arc, + ) -> TargetBlueprintLoader { + let (tx, rx) = watch::channel(None); + TargetBlueprintLoader { blueprints, datastore, last: None, tx, rx } + } + + /// Expose the target blueprint + pub fn watcher(&self) -> watch::Receiver>> { + self.rx.clone() + } + + // This functoin is a modified copy from `nexus/src/app/deployment.rs` for + // use until the types are in the datastore. + // + // This is a stand-in for a datastore function that fetches the current + // target information and the target blueprint's contents. This helper + // exists to combine the authz check with the lookup, which is what the + // datastore function will eventually do. + async fn blueprint_target( + &self, + opctx: &OpContext, + ) -> Result, anyhow::Error> { + opctx.authorize(Action::Read, &authz::BLUEPRINT_CONFIG).await?; + let blueprints = self.blueprints.lock().unwrap(); + Ok(blueprints.target.target_id.and_then(|target_id| { + blueprints.all_blueprints.get(&target_id).cloned() + })) + } +} + +impl BackgroundTask for TargetBlueprintLoader { + fn activate<'a>( + &'a mut self, + opctx: &'a OpContext, + ) -> BoxFuture<'a, serde_json::Value> { + async { + // Set up a logger for this activation that includes metadata about + // the current target. + let log = match &self.last { + None => opctx.log.clone(), + Some(old) => opctx.log.new(o!( + "current_target_id" => old.id.to_string(), + "current_time_created" => old.time_created.to_string(), + )), + }; + + // Retrieve the latest target blueprint + let result = self.blueprint_target(opctx).await; + + // Decide what to do with the result + match (&self.last, result) { + (_, Err(error)) => { + // We failed to read the blueprint. There's nothing to do + // but log an error. We'll retry when we're activated again. + let message = format!("{:#}", error); + warn!( + &log, + "failed to read target blueprint"; + "error" => &message + ); + json!({ + "error": + format!("failed to read target blueprint: {message}")}) + } + (None, Ok(None)) => { + // We haven't found a blueprint yet. Do nothing. + json!({}) + } + (Some(old), Ok(None)) => { + // We have transitioned from having a blueprint to not having one. + // This should not happen. + let message = format!( + "target blueprint with id {} was removed. There is no \ + longer any target blueprint", + old.id + ); + error!(&log, "{}", message); + json!({"error": message}) + + } + (None, Ok(Some(new_target))) => { + // We've found a new target blueprint for the first time. + // Save it and notify any watchers. + let target_id = new_target.id.to_string(); + let time_created = new_target.time_created.to_string(); + info!( + log, + "found new target blueprint (first find)"; + "target_id" => &target_id, + "time_created" => &time_created + ); + self.last = Some(Arc::new(new_target)); + self.tx.send_replace(self.last.clone()); + json!({"target_id": target_id, "time_created": time_created}) + } + (Some(old), Ok(Some(new))) => { + let target_id = new.id.to_string(); + let time_created = new.time_created.to_string(); + if old.id != new.id { + // The current target blueprint has been updated + info!( + log, + "found new target blueprint"; + "target_id" => &target_id, + "time_created" => &time_created + ); + self.last = Some(Arc::new(new)); + self.tx.send_replace(self.last.clone()); + json!({"target_id": target_id, "time_created": time_created}) + } else { + // The new target id matches the old target id + // + // Let's see if the blueprints hold the same contents. + // It should not be possible for the contents of a + // blueprint to change, but we check to catch possible + // bugs further up the stack. + if **old != new { + let message = format!( + "blueprint for id {} changed. \ + Blueprints are supposed to be immutable.", + target_id + ); + error!(&log, "{}", message); + json!({"error": message}) + } else { + // We found a new target blueprint that exactly matches + // the old target blueprint. This is the common case + // when we're activated by a timeout. + debug!( + log, + "found latest target blueprint (unchanged)"; + "target_id" => &target_id, + "time_created" => &time_created.clone() + ); + json!({"target_id": target_id, "time_created": time_created}) + } + } + } + } + }.boxed() + } +} diff --git a/nexus/src/app/deployment.rs b/nexus/src/app/deployment.rs index 9439cdc6d5..2dda850fb9 100644 --- a/nexus/src/app/deployment.rs +++ b/nexus/src/app/deployment.rs @@ -52,8 +52,8 @@ const SQL_LIMIT_INVENTORY: NonZeroU32 = /// Blueprints eventually need to be stored in the database. That will obviate /// the need for this structure. pub struct Blueprints { - all_blueprints: BTreeMap, - target: BlueprintTarget, + pub all_blueprints: BTreeMap, + pub target: BlueprintTarget, } impl Blueprints { From 2b2ad63922ce5f0d6cdb7f520fa4d2204e1c38ad Mon Sep 17 00:00:00 2001 From: "Andrew J. Stone" Date: Thu, 25 Jan 2024 22:15:56 +0000 Subject: [PATCH 03/17] Add blueprint related bg task init --- nexus/src/app/background/init.rs | 45 ++++++++++++++++++- .../src/app/background/plan_blueprint_load.rs | 6 +-- nexus/src/app/mod.rs | 9 +++- 3 files changed, 54 insertions(+), 6 deletions(-) diff --git a/nexus/src/app/background/init.rs b/nexus/src/app/background/init.rs index d30d2162c4..a7b7b5c4fe 100644 --- a/nexus/src/app/background/init.rs +++ b/nexus/src/app/background/init.rs @@ -4,6 +4,8 @@ //! Background task initialization +use crate::app::deployment; + use super::common; use super::dns_config; use super::dns_propagation; @@ -12,6 +14,8 @@ use super::external_endpoints; use super::inventory_collection; use super::nat_cleanup; use super::phantom_disks; +use super::plan_blueprint_load; +use super::plan_execution; use nexus_db_model::DnsGroup; use nexus_db_queries::context::OpContext; use nexus_db_queries::db::DataStore; @@ -56,6 +60,12 @@ pub struct BackgroundTasks { /// task handle for the task that detects phantom disks pub task_phantom_disks: common::TaskHandle, + + /// task handle for blueprint target loader + pub task_plan_blueprint_target_loader: common::TaskHandle, + + /// task handle for plan execution background task + pub task_plan_executor: common::TaskHandle, } impl BackgroundTasks { @@ -67,6 +77,7 @@ impl BackgroundTasks { dpd_clients: &HashMap>, nexus_id: Uuid, resolver: internal_dns::resolver::Resolver, + blueprints: Arc>, ) -> BackgroundTasks { let mut driver = common::Driver::new(); @@ -149,7 +160,8 @@ impl BackgroundTasks { // Background task: phantom disk detection let task_phantom_disks = { - let detector = phantom_disks::PhantomDiskDetector::new(datastore); + let detector = + phantom_disks::PhantomDiskDetector::new(datastore.clone()); let task = driver.register( String::from("phantom_disks"), @@ -163,6 +175,35 @@ impl BackgroundTasks { task }; + // Background task: blueprint loader + let blueprint_loader = plan_blueprint_load::TargetBlueprintLoader::new( + blueprints, + datastore.clone(), + ); + let rx_blueprint = blueprint_loader.watcher(); + let task_plan_blueprint_target_loader = driver.register( + String::from("blueprint_target_loader"), + String::from("Loads the current target blueprint from the DB"), + // TODO: Add to `BackgroundTaskConfig`? + std::time::Duration::from_secs(5), + Box::new(blueprint_loader), + opctx.child(BTreeMap::new()), + vec![], + ); + + // Background task: plan executor + let plan_executor = + plan_execution::PlanExecutor::new(datastore, rx_blueprint.clone()); + let task_plan_executor = driver.register( + String::from("plan_executor"), + String::from("Executes the target blueprint"), + // TODO: Add to `BackgroundTaskConfig`? + std::time::Duration::from_secs(60), + Box::new(plan_executor), + opctx.child(BTreeMap::new()), + vec![Box::new(rx_blueprint)], + ); + BackgroundTasks { driver, task_internal_dns_config, @@ -174,6 +215,8 @@ impl BackgroundTasks { nat_cleanup, task_inventory_collection, task_phantom_disks, + task_plan_blueprint_target_loader, + task_plan_executor, } } diff --git a/nexus/src/app/background/plan_blueprint_load.rs b/nexus/src/app/background/plan_blueprint_load.rs index a8b7d56fd3..f6299da8bf 100644 --- a/nexus/src/app/background/plan_blueprint_load.rs +++ b/nexus/src/app/background/plan_blueprint_load.rs @@ -21,7 +21,7 @@ use std::sync::Arc; use tokio::sync::watch; pub struct TargetBlueprintLoader { - blueprints: std::sync::Mutex, + blueprints: Arc>, #[allow(unused)] datastore: Arc, last: Option>, @@ -31,7 +31,7 @@ pub struct TargetBlueprintLoader { impl TargetBlueprintLoader { pub fn new( - blueprints: std::sync::Mutex, + blueprints: Arc>, datastore: Arc, ) -> TargetBlueprintLoader { let (tx, rx) = watch::channel(None); @@ -43,7 +43,7 @@ impl TargetBlueprintLoader { self.rx.clone() } - // This functoin is a modified copy from `nexus/src/app/deployment.rs` for + // This function is a modified copy from `nexus/src/app/deployment.rs` for // use until the types are in the datastore. // // This is a stand-in for a datastore function that fetches the current diff --git a/nexus/src/app/mod.rs b/nexus/src/app/mod.rs index 80bfd5ef22..b63032ed82 100644 --- a/nexus/src/app/mod.rs +++ b/nexus/src/app/mod.rs @@ -183,7 +183,7 @@ pub struct Nexus { /// information about blueprints (deployment configurations) // This will go away once these are stored in the database. - blueprints: std::sync::Mutex, + blueprints: Arc>, } impl Nexus { @@ -357,6 +357,10 @@ impl Nexus { authn::Context::internal_api(), Arc::clone(&db_datastore), ); + + let blueprints = + Arc::new(std::sync::Mutex::new(deployment::Blueprints::new())); + let background_tasks = background::BackgroundTasks::start( &background_ctx, Arc::clone(&db_datastore), @@ -364,6 +368,7 @@ impl Nexus { &dpd_clients, config.deployment.id, resolver.clone(), + Arc::clone(&blueprints), ); let external_resolver = { @@ -416,7 +421,7 @@ impl Nexus { .pkg .default_region_allocation_strategy .clone(), - blueprints: std::sync::Mutex::new(deployment::Blueprints::new()), + blueprints, }; // TODO-cleanup all the extra Arcs here seems wrong From 24b34f70f76a5795d51987442f392dc374cc71eb Mon Sep 17 00:00:00 2001 From: "Andrew J. Stone" Date: Sat, 27 Jan 2024 00:07:52 +0000 Subject: [PATCH 04/17] Use DB backed blueprints from #4899 --- ...an_execution.rs => blueprint_execution.rs} | 21 +++-- ...an_blueprint_load.rs => blueprint_load.rs} | 79 ++++++++----------- nexus/src/app/background/init.rs | 41 +++++----- nexus/src/app/background/mod.rs | 4 +- nexus/src/app/mod.rs | 4 - 5 files changed, 62 insertions(+), 87 deletions(-) rename nexus/src/app/background/{plan_execution.rs => blueprint_execution.rs} (96%) rename nexus/src/app/background/{plan_blueprint_load.rs => blueprint_load.rs} (69%) diff --git a/nexus/src/app/background/plan_execution.rs b/nexus/src/app/background/blueprint_execution.rs similarity index 96% rename from nexus/src/app/background/plan_execution.rs rename to nexus/src/app/background/blueprint_execution.rs index c85d96abbe..c2e0f3dd55 100644 --- a/nexus/src/app/background/plan_execution.rs +++ b/nexus/src/app/background/blueprint_execution.rs @@ -24,19 +24,19 @@ use uuid::Uuid; /// Background task that takes a [`Blueprint`] and realizes the change to /// the state of the system based on the `Blueprint`. -pub struct PlanExecutor { +pub struct BlueprintExecutor { datastore: Arc, rx_blueprint: watch::Receiver>>, } -impl PlanExecutor { +impl BlueprintExecutor { // Temporary until we wire up the background task #[allow(unused)] pub fn new( datastore: Arc, rx_blueprint: watch::Receiver>>, - ) -> PlanExecutor { - PlanExecutor { datastore, rx_blueprint } + ) -> BlueprintExecutor { + BlueprintExecutor { datastore, rx_blueprint } } // This is a modified copy of the functionality from `nexus/src/app/sled.rs`. @@ -92,7 +92,7 @@ impl PlanExecutor { let client = match self.sled_client(&opctx, &sled_id).await { Ok(client) => client, Err(err) => { - warn!(log, "{}", err); + warn!(log, "{err:#}"); return Some(err); } }; @@ -105,7 +105,7 @@ impl PlanExecutor { match result { Err(error) => { - warn!(log, "{}", error); + warn!(log, "{error:#}"); Some(error) } Ok(_) => { @@ -130,7 +130,7 @@ impl PlanExecutor { } } -impl BackgroundTask for PlanExecutor { +impl BackgroundTask for BlueprintExecutor { fn activate<'a>( &'a mut self, opctx: &'a OpContext, @@ -138,7 +138,7 @@ impl BackgroundTask for PlanExecutor { async { // Get the latest blueprint, cloning to prevent holding a read lock // on the watch. - let blueprint = self.rx_blueprint.borrow().clone(); + let blueprint = self.rx_blueprint.borrow_and_update().clone(); let Some(blueprint) = blueprint else { warn!(&opctx.log, @@ -210,7 +210,7 @@ mod test { ); let (blueprint_tx, blueprint_rx) = watch::channel(None); - let mut task = PlanExecutor::new(datastore.clone(), blueprint_rx); + let mut task = BlueprintExecutor::new(datastore.clone(), blueprint_rx); // With no blueprint we should fail with an appropriate message. let value = task.activate(&opctx).await; @@ -267,8 +267,7 @@ mod test { // Zones are updated in a particular order, but each request contains // the full set of zones that must be running. - // See https://github.com/oxidecomputer/omicron/blob/main/sled-agent/src/rack_setup/service.rs#L976-L998 - // for more details. + // See `rack_setup::service::ServiceInner::run` for more details. let mut zones = OmicronZonesConfig { generation, zones: vec![OmicronZoneConfig { diff --git a/nexus/src/app/background/plan_blueprint_load.rs b/nexus/src/app/background/blueprint_load.rs similarity index 69% rename from nexus/src/app/background/plan_blueprint_load.rs rename to nexus/src/app/background/blueprint_load.rs index f6299da8bf..b61a383a86 100644 --- a/nexus/src/app/background/plan_blueprint_load.rs +++ b/nexus/src/app/background/blueprint_load.rs @@ -8,11 +8,8 @@ //! changes. use super::common::BackgroundTask; -use crate::app::deployment; use futures::future::BoxFuture; use futures::FutureExt; -use nexus_db_queries::authz; -use nexus_db_queries::authz::Action; use nexus_db_queries::context::OpContext; use nexus_db_queries::db::DataStore; use nexus_types::deployment::Blueprint; @@ -21,8 +18,6 @@ use std::sync::Arc; use tokio::sync::watch; pub struct TargetBlueprintLoader { - blueprints: Arc>, - #[allow(unused)] datastore: Arc, last: Option>, tx: watch::Sender>>, @@ -30,36 +25,15 @@ pub struct TargetBlueprintLoader { } impl TargetBlueprintLoader { - pub fn new( - blueprints: Arc>, - datastore: Arc, - ) -> TargetBlueprintLoader { + pub fn new(datastore: Arc) -> TargetBlueprintLoader { let (tx, rx) = watch::channel(None); - TargetBlueprintLoader { blueprints, datastore, last: None, tx, rx } + TargetBlueprintLoader { datastore, last: None, tx, rx } } /// Expose the target blueprint pub fn watcher(&self) -> watch::Receiver>> { self.rx.clone() } - - // This function is a modified copy from `nexus/src/app/deployment.rs` for - // use until the types are in the datastore. - // - // This is a stand-in for a datastore function that fetches the current - // target information and the target blueprint's contents. This helper - // exists to combine the authz check with the lookup, which is what the - // datastore function will eventually do. - async fn blueprint_target( - &self, - opctx: &OpContext, - ) -> Result, anyhow::Error> { - opctx.authorize(Action::Read, &authz::BLUEPRINT_CONFIG).await?; - let blueprints = self.blueprints.lock().unwrap(); - Ok(blueprints.target.target_id.and_then(|target_id| { - blueprints.all_blueprints.get(&target_id).cloned() - })) - } } impl BackgroundTask for TargetBlueprintLoader { @@ -79,7 +53,8 @@ impl BackgroundTask for TargetBlueprintLoader { }; // Retrieve the latest target blueprint - let result = self.blueprint_target(opctx).await; + let result = + self.datastore.blueprint_target_get_current_full(opctx).await; // Decide what to do with the result match (&self.last, result) { @@ -101,8 +76,8 @@ impl BackgroundTask for TargetBlueprintLoader { json!({}) } (Some(old), Ok(None)) => { - // We have transitioned from having a blueprint to not having one. - // This should not happen. + // We have transitioned from having a blueprint to not + // having one. This should not happen. let message = format!( "target blueprint with id {} was removed. There is no \ longer any target blueprint", @@ -110,9 +85,8 @@ impl BackgroundTask for TargetBlueprintLoader { ); error!(&log, "{}", message); json!({"error": message}) - } - (None, Ok(Some(new_target))) => { + (None, Ok(Some((_, new_target)))) => { // We've found a new target blueprint for the first time. // Save it and notify any watchers. let target_id = new_target.id.to_string(); @@ -125,9 +99,11 @@ impl BackgroundTask for TargetBlueprintLoader { ); self.last = Some(Arc::new(new_target)); self.tx.send_replace(self.last.clone()); - json!({"target_id": target_id, "time_created": time_created}) + json!({ + "target_id": target_id, "time_created": time_created + }) } - (Some(old), Ok(Some(new))) => { + (Some(old), Ok(Some((_, new)))) => { let target_id = new.id.to_string(); let time_created = new.time_created.to_string(); if old.id != new.id { @@ -140,7 +116,10 @@ impl BackgroundTask for TargetBlueprintLoader { ); self.last = Some(Arc::new(new)); self.tx.send_replace(self.last.clone()); - json!({"target_id": target_id, "time_created": time_created}) + json!({ + "target_id": target_id, + "time_created": time_created + }) } else { // The new target id matches the old target id // @@ -157,20 +136,24 @@ impl BackgroundTask for TargetBlueprintLoader { error!(&log, "{}", message); json!({"error": message}) } else { - // We found a new target blueprint that exactly matches - // the old target blueprint. This is the common case - // when we're activated by a timeout. - debug!( - log, - "found latest target blueprint (unchanged)"; - "target_id" => &target_id, - "time_created" => &time_created.clone() - ); - json!({"target_id": target_id, "time_created": time_created}) - } + // We found a new target blueprint that exactly matches + // the old target blueprint. This is the common case + // when we're activated by a timeout. + debug!( + log, + "found latest target blueprint (unchanged)"; + "target_id" => &target_id, + "time_created" => &time_created.clone() + ); + json!({ + "target_id": target_id, + "time_created": time_created + }) + } } } } - }.boxed() + } + .boxed() } } diff --git a/nexus/src/app/background/init.rs b/nexus/src/app/background/init.rs index a7b7b5c4fe..6dfea99657 100644 --- a/nexus/src/app/background/init.rs +++ b/nexus/src/app/background/init.rs @@ -4,8 +4,8 @@ //! Background task initialization -use crate::app::deployment; - +use super::blueprint_execution; +use super::blueprint_load; use super::common; use super::dns_config; use super::dns_propagation; @@ -14,8 +14,6 @@ use super::external_endpoints; use super::inventory_collection; use super::nat_cleanup; use super::phantom_disks; -use super::plan_blueprint_load; -use super::plan_execution; use nexus_db_model::DnsGroup; use nexus_db_queries::context::OpContext; use nexus_db_queries::db::DataStore; @@ -62,10 +60,10 @@ pub struct BackgroundTasks { pub task_phantom_disks: common::TaskHandle, /// task handle for blueprint target loader - pub task_plan_blueprint_target_loader: common::TaskHandle, + pub task_blueprint_loader: common::TaskHandle, - /// task handle for plan execution background task - pub task_plan_executor: common::TaskHandle, + /// task handle for blueprint execution background task + pub task_blueprint_executor: common::TaskHandle, } impl BackgroundTasks { @@ -77,7 +75,6 @@ impl BackgroundTasks { dpd_clients: &HashMap>, nexus_id: Uuid, resolver: internal_dns::resolver::Resolver, - blueprints: Arc>, ) -> BackgroundTasks { let mut driver = common::Driver::new(); @@ -176,13 +173,11 @@ impl BackgroundTasks { }; // Background task: blueprint loader - let blueprint_loader = plan_blueprint_load::TargetBlueprintLoader::new( - blueprints, - datastore.clone(), - ); + let blueprint_loader = + blueprint_load::TargetBlueprintLoader::new(datastore.clone()); let rx_blueprint = blueprint_loader.watcher(); - let task_plan_blueprint_target_loader = driver.register( - String::from("blueprint_target_loader"), + let task_blueprint_loader = driver.register( + String::from("blueprint_loader"), String::from("Loads the current target blueprint from the DB"), // TODO: Add to `BackgroundTaskConfig`? std::time::Duration::from_secs(5), @@ -191,15 +186,17 @@ impl BackgroundTasks { vec![], ); - // Background task: plan executor - let plan_executor = - plan_execution::PlanExecutor::new(datastore, rx_blueprint.clone()); - let task_plan_executor = driver.register( - String::from("plan_executor"), + // Background task: blueprint executor + let blueprint_executor = blueprint_execution::BlueprintExecutor::new( + datastore, + rx_blueprint.clone(), + ); + let task_blueprint_executor = driver.register( + String::from("blueprint_executor"), String::from("Executes the target blueprint"), // TODO: Add to `BackgroundTaskConfig`? std::time::Duration::from_secs(60), - Box::new(plan_executor), + Box::new(blueprint_executor), opctx.child(BTreeMap::new()), vec![Box::new(rx_blueprint)], ); @@ -215,8 +212,8 @@ impl BackgroundTasks { nat_cleanup, task_inventory_collection, task_phantom_disks, - task_plan_blueprint_target_loader, - task_plan_executor, + task_blueprint_loader, + task_blueprint_executor, } } diff --git a/nexus/src/app/background/mod.rs b/nexus/src/app/background/mod.rs index 3b5a9b3ad9..236881271d 100644 --- a/nexus/src/app/background/mod.rs +++ b/nexus/src/app/background/mod.rs @@ -4,6 +4,8 @@ //! Background tasks +mod blueprint_execution; +mod blueprint_load; mod common; mod dns_config; mod dns_propagation; @@ -13,8 +15,6 @@ mod init; mod inventory_collection; mod nat_cleanup; mod phantom_disks; -mod plan_blueprint_load; -mod plan_execution; mod status; pub use common::Driver; diff --git a/nexus/src/app/mod.rs b/nexus/src/app/mod.rs index 42d031dc9a..5d8c321cd4 100644 --- a/nexus/src/app/mod.rs +++ b/nexus/src/app/mod.rs @@ -357,9 +357,6 @@ impl Nexus { Arc::clone(&db_datastore), ); - let blueprints = - Arc::new(std::sync::Mutex::new(deployment::Blueprints::new())); - let background_tasks = background::BackgroundTasks::start( &background_ctx, Arc::clone(&db_datastore), @@ -367,7 +364,6 @@ impl Nexus { &dpd_clients, config.deployment.id, resolver.clone(), - Arc::clone(&blueprints), ); let external_resolver = { From aa88e651801abaff901cfea80602511b89e42ea4 Mon Sep 17 00:00:00 2001 From: "Andrew J. Stone" Date: Sat, 27 Jan 2024 00:46:27 +0000 Subject: [PATCH 05/17] Add BlueprintTasksConfig --- common/src/nexus_config.rs | 36 +++++++++++++++++++---- nexus/examples/config.toml | 2 ++ nexus/src/app/background/init.rs | 6 ++-- smf/nexus/multi-sled/config-partial.toml | 2 ++ smf/nexus/single-sled/config-partial.toml | 2 ++ 5 files changed, 38 insertions(+), 10 deletions(-) diff --git a/common/src/nexus_config.rs b/common/src/nexus_config.rs index be4b05ffdf..9d7fac988e 100644 --- a/common/src/nexus_config.rs +++ b/common/src/nexus_config.rs @@ -334,6 +334,8 @@ pub struct BackgroundTaskConfig { pub inventory: InventoryConfig, /// configuration for phantom disks task pub phantom_disks: PhantomDiskConfig, + /// configuration for blueprint related tasks + pub blueprints: BlueprintTasksConfig, } #[serde_as] @@ -408,6 +410,20 @@ pub struct PhantomDiskConfig { pub period_secs: Duration, } +#[serde_as] +#[derive(Clone, Debug, Deserialize, Eq, PartialEq, Serialize)] +pub struct BlueprintTasksConfig { + /// period (in seconds) for periodic activations of the background task that + /// reads the latest target blueprint from the database + #[serde_as(as = "DurationSeconds")] + pub period_secs_load: Duration, + + /// period (in seconds) for periodic activations of the background task that + /// executes the latest target blueprint + #[serde_as(as = "DurationSeconds")] + pub period_secs_execute: Duration, +} + /// Configuration for a nexus server #[derive(Clone, Debug, Deserialize, PartialEq, Serialize)] pub struct PackageConfig { @@ -508,12 +524,12 @@ impl std::fmt::Display for SchemeName { mod test { use super::{ default_techport_external_server_port, AuthnConfig, - BackgroundTaskConfig, Config, ConfigDropshotWithTls, ConsoleConfig, - Database, DeploymentConfig, DnsTasksConfig, DpdConfig, - ExternalEndpointsConfig, InternalDns, InventoryConfig, LoadError, - LoadErrorKind, MgdConfig, NatCleanupConfig, PackageConfig, - PhantomDiskConfig, SchemeName, TimeseriesDbConfig, Tunables, - UpdatesConfig, + BackgroundTaskConfig, BlueprintTasksConfig, Config, + ConfigDropshotWithTls, ConsoleConfig, Database, DeploymentConfig, + DnsTasksConfig, DpdConfig, ExternalEndpointsConfig, InternalDns, + InventoryConfig, LoadError, LoadErrorKind, MgdConfig, NatCleanupConfig, + PackageConfig, PhantomDiskConfig, SchemeName, TimeseriesDbConfig, + Tunables, UpdatesConfig, }; use crate::address::{Ipv6Subnet, RACK_PREFIX}; use crate::api::internal::shared::SwitchLocation; @@ -665,6 +681,8 @@ mod test { inventory.nkeep = 11 inventory.disable = false phantom_disks.period_secs = 30 + blueprints.period_secs_load = 10 + blueprints.period_secs_execute = 60 [default_region_allocation_strategy] type = "random" seed = 0 @@ -769,6 +787,10 @@ mod test { phantom_disks: PhantomDiskConfig { period_secs: Duration::from_secs(30), }, + blueprints: BlueprintTasksConfig { + period_secs_load: Duration::from_secs(10), + period_secs_execute: Duration::from_secs(60) + } }, default_region_allocation_strategy: crate::nexus_config::RegionAllocationStrategy::Random { @@ -827,6 +849,8 @@ mod test { inventory.nkeep = 3 inventory.disable = false phantom_disks.period_secs = 30 + blueprints.period_secs_load = 10 + blueprints.period_secs_execute = 60 [default_region_allocation_strategy] type = "random" "##, diff --git a/nexus/examples/config.toml b/nexus/examples/config.toml index f13ea721b8..90a2143061 100644 --- a/nexus/examples/config.toml +++ b/nexus/examples/config.toml @@ -105,6 +105,8 @@ inventory.nkeep = 5 # Disable inventory collection altogether (for emergencies) inventory.disable = false phantom_disks.period_secs = 30 +blueprints.period_secs_load = 10 +blueprints.period_secs_execute = 60 [default_region_allocation_strategy] # allocate region on 3 random distinct zpools, on 3 random distinct sleds. diff --git a/nexus/src/app/background/init.rs b/nexus/src/app/background/init.rs index 6dfea99657..d9b3ed956f 100644 --- a/nexus/src/app/background/init.rs +++ b/nexus/src/app/background/init.rs @@ -179,8 +179,7 @@ impl BackgroundTasks { let task_blueprint_loader = driver.register( String::from("blueprint_loader"), String::from("Loads the current target blueprint from the DB"), - // TODO: Add to `BackgroundTaskConfig`? - std::time::Duration::from_secs(5), + config.blueprints.period_secs_load, Box::new(blueprint_loader), opctx.child(BTreeMap::new()), vec![], @@ -194,8 +193,7 @@ impl BackgroundTasks { let task_blueprint_executor = driver.register( String::from("blueprint_executor"), String::from("Executes the target blueprint"), - // TODO: Add to `BackgroundTaskConfig`? - std::time::Duration::from_secs(60), + config.blueprints.period_secs_execute, Box::new(blueprint_executor), opctx.child(BTreeMap::new()), vec![Box::new(rx_blueprint)], diff --git a/smf/nexus/multi-sled/config-partial.toml b/smf/nexus/multi-sled/config-partial.toml index d330f32ab6..809b5b9e64 100644 --- a/smf/nexus/multi-sled/config-partial.toml +++ b/smf/nexus/multi-sled/config-partial.toml @@ -47,6 +47,8 @@ inventory.nkeep = 3 # Disable inventory collection altogether (for emergencies) inventory.disable = false phantom_disks.period_secs = 30 +blueprints.period_secs_load = 10 +blueprints.period_secs_execute = 60 [default_region_allocation_strategy] # by default, allocate across 3 distinct sleds diff --git a/smf/nexus/single-sled/config-partial.toml b/smf/nexus/single-sled/config-partial.toml index cbd4851613..20dc9adf66 100644 --- a/smf/nexus/single-sled/config-partial.toml +++ b/smf/nexus/single-sled/config-partial.toml @@ -47,6 +47,8 @@ inventory.nkeep = 3 # Disable inventory collection altogether (for emergencies) inventory.disable = false phantom_disks.period_secs = 30 +blueprints.period_secs_load = 10 +blueprints.period_secs_execute = 60 [default_region_allocation_strategy] # by default, allocate without requirement for distinct sleds. From 80dfeea4a001adbd996911233ec8dcdd3ba11bbc Mon Sep 17 00:00:00 2001 From: "Andrew J. Stone" Date: Tue, 30 Jan 2024 14:38:52 +0000 Subject: [PATCH 06/17] test config fix --- nexus/tests/config.test.toml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/nexus/tests/config.test.toml b/nexus/tests/config.test.toml index 476b8fe6c8..bf8022f75d 100644 --- a/nexus/tests/config.test.toml +++ b/nexus/tests/config.test.toml @@ -99,6 +99,8 @@ inventory.nkeep = 3 # Disable inventory collection altogether (for emergencies) inventory.disable = false phantom_disks.period_secs = 30 +blueprints.period_secs_load = 10 +blueprints.period_secs_execute = 60 sync_service_zone_nat.period_secs = 30 [default_region_allocation_strategy] From 596c9be466c0ba54c41201edf0c18d9b8c6c6280 Mon Sep 17 00:00:00 2001 From: "Andrew J. Stone" Date: Tue, 30 Jan 2024 15:10:31 +0000 Subject: [PATCH 07/17] some review fixes --- common/src/nexus_config.rs | 1 + .../src/app/background/blueprint_execution.rs | 9 +++--- nexus/src/app/background/blueprint_load.rs | 30 ++++++++++++------- nexus/src/app/background/init.rs | 2 +- 4 files changed, 26 insertions(+), 16 deletions(-) diff --git a/common/src/nexus_config.rs b/common/src/nexus_config.rs index ceb0834c12..b1d392d7dd 100644 --- a/common/src/nexus_config.rs +++ b/common/src/nexus_config.rs @@ -802,6 +802,7 @@ mod test { blueprints: BlueprintTasksConfig { period_secs_load: Duration::from_secs(10), period_secs_execute: Duration::from_secs(60) + }, sync_service_zone_nat: SyncServiceZoneNatConfig { period_secs: Duration::from_secs(30) } diff --git a/nexus/src/app/background/blueprint_execution.rs b/nexus/src/app/background/blueprint_execution.rs index c2e0f3dd55..5785c00c40 100644 --- a/nexus/src/app/background/blueprint_execution.rs +++ b/nexus/src/app/background/blueprint_execution.rs @@ -30,8 +30,6 @@ pub struct BlueprintExecutor { } impl BlueprintExecutor { - // Temporary until we wire up the background task - #[allow(unused)] pub fn new( datastore: Arc, rx_blueprint: watch::Receiver>>, @@ -75,7 +73,7 @@ impl BlueprintExecutor { async fn realize_blueprint( &self, opctx: &OpContext, - blueprint: &std::sync::Arc, + blueprint: &Blueprint, ) -> Result<(), Vec> { let log = opctx.log.new(o!("comment" => blueprint.comment.clone())); self.deploy_zones(&log, opctx, &blueprint.omicron_zones).await @@ -157,7 +155,10 @@ impl BackgroundTask for BlueprintExecutor { .into_iter() .map(|e| format!("{:#}", e)) .collect(); - json!({"errors": errors}) + json!({ + "target_id": blueprint.id.to_string(), + "errors": errors + }) } } } diff --git a/nexus/src/app/background/blueprint_load.rs b/nexus/src/app/background/blueprint_load.rs index b61a383a86..13c3104fa0 100644 --- a/nexus/src/app/background/blueprint_load.rs +++ b/nexus/src/app/background/blueprint_load.rs @@ -4,8 +4,8 @@ //! Background task for loading the target blueprint from the DB //! -//! This task triggers the `plan_execution` background task when the blueprint -//! changes. +//! This task triggers the `blueprint_execution` background task when the +//! blueprint changes. use super::common::BackgroundTask; use futures::future::BoxFuture; @@ -67,13 +67,13 @@ impl BackgroundTask for TargetBlueprintLoader { "failed to read target blueprint"; "error" => &message ); - json!({ - "error": - format!("failed to read target blueprint: {message}")}) + let e = + format!("failed to read target blueprint: {message}"); + json!({"error": e}) } (None, Ok(None)) => { // We haven't found a blueprint yet. Do nothing. - json!({}) + json!({"status": "no blueprint"}) } (Some(old), Ok(None)) => { // We have transitioned from having a blueprint to not @@ -83,11 +83,13 @@ impl BackgroundTask for TargetBlueprintLoader { longer any target blueprint", old.id ); - error!(&log, "{}", message); + self.last = None; + self.tx.send_replace(self.last.clone()); + error!(&log, "{message:?}"); json!({"error": message}) } (None, Ok(Some((_, new_target)))) => { - // We've found a new target blueprint for the first time. + // We've found a target blueprint for the first time. // Save it and notify any watchers. let target_id = new_target.id.to_string(); let time_created = new_target.time_created.to_string(); @@ -100,7 +102,10 @@ impl BackgroundTask for TargetBlueprintLoader { self.last = Some(Arc::new(new_target)); self.tx.send_replace(self.last.clone()); json!({ - "target_id": target_id, "time_created": time_created + "target_id": target_id, + "time_created": time_created, + "time_found": chrono::Utc::now().to_string(), + "status": "first target blueprint" }) } (Some(old), Ok(Some((_, new)))) => { @@ -118,7 +123,9 @@ impl BackgroundTask for TargetBlueprintLoader { self.tx.send_replace(self.last.clone()); json!({ "target_id": target_id, - "time_created": time_created + "time_created": time_created, + "time_found": chrono::Utc::now().to_string(), + "status": "target blueprint updated" }) } else { // The new target id matches the old target id @@ -147,7 +154,8 @@ impl BackgroundTask for TargetBlueprintLoader { ); json!({ "target_id": target_id, - "time_created": time_created + "time_created": time_created, + "status": "target blueprint unchanged" }) } } diff --git a/nexus/src/app/background/init.rs b/nexus/src/app/background/init.rs index fbece9ae10..f7c85e9455 100644 --- a/nexus/src/app/background/init.rs +++ b/nexus/src/app/background/init.rs @@ -194,7 +194,7 @@ impl BackgroundTasks { // Background task: blueprint executor let blueprint_executor = blueprint_execution::BlueprintExecutor::new( - datastore, + datastore.clone(), rx_blueprint.clone(), ); let task_blueprint_executor = driver.register( From 5d72471ad30c517f19c94d1eab9274fde74bf61d Mon Sep 17 00:00:00 2001 From: "Andrew J. Stone" Date: Tue, 30 Jan 2024 15:27:32 +0000 Subject: [PATCH 08/17] some more review fixes --- nexus/src/app/background/blueprint_load.rs | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/nexus/src/app/background/blueprint_load.rs b/nexus/src/app/background/blueprint_load.rs index 13c3104fa0..ead51f2514 100644 --- a/nexus/src/app/background/blueprint_load.rs +++ b/nexus/src/app/background/blueprint_load.rs @@ -57,7 +57,7 @@ impl BackgroundTask for TargetBlueprintLoader { self.datastore.blueprint_target_get_current_full(opctx).await; // Decide what to do with the result - match (&self.last, result) { + match (&mut self.last, result) { (_, Err(error)) => { // We failed to read the blueprint. There's nothing to do // but log an error. We'll retry when we're activated again. @@ -73,7 +73,7 @@ impl BackgroundTask for TargetBlueprintLoader { } (None, Ok(None)) => { // We haven't found a blueprint yet. Do nothing. - json!({"status": "no blueprint"}) + json!({"status": "no target blueprint"}) } (Some(old), Ok(None)) => { // We have transitioned from having a blueprint to not @@ -83,10 +83,15 @@ impl BackgroundTask for TargetBlueprintLoader { longer any target blueprint", old.id ); + let old_id = old.id.to_string(); self.last = None; self.tx.send_replace(self.last.clone()); error!(&log, "{message:?}"); - json!({"error": message}) + json!({ + "removed_target_id": old_id, + "status": "no target blueprint (removed)", + "error": message + }) } (None, Ok(Some((_, new_target)))) => { // We've found a target blueprint for the first time. @@ -141,7 +146,11 @@ impl BackgroundTask for TargetBlueprintLoader { target_id ); error!(&log, "{}", message); - json!({"error": message}) + json!({ + "target_id": target_id, + "status": "target blueprint unchanged (error)", + "error": message + }) } else { // We found a new target blueprint that exactly matches // the old target blueprint. This is the common case From 9d95fd84dc23eee190c1924cf57ff7574617acb7 Mon Sep 17 00:00:00 2001 From: "Andrew J. Stone" Date: Tue, 30 Jan 2024 17:05:40 +0000 Subject: [PATCH 09/17] Only execute enabled target blueprint --- .../src/app/background/blueprint_execution.rs | 56 +++++++++++++------ nexus/src/app/background/blueprint_load.rs | 41 +++++++------- 2 files changed, 61 insertions(+), 36 deletions(-) diff --git a/nexus/src/app/background/blueprint_execution.rs b/nexus/src/app/background/blueprint_execution.rs index 5785c00c40..8d6ea8d8ce 100644 --- a/nexus/src/app/background/blueprint_execution.rs +++ b/nexus/src/app/background/blueprint_execution.rs @@ -13,7 +13,7 @@ use futures::StreamExt; use nexus_db_queries::context::OpContext; use nexus_db_queries::db::lookup::LookupPath; use nexus_db_queries::db::DataStore; -use nexus_types::deployment::{Blueprint, OmicronZonesConfig}; +use nexus_types::deployment::{Blueprint, BlueprintTarget, OmicronZonesConfig}; use serde_json::json; use sled_agent_client::Client as SledAgentClient; use slog::Logger; @@ -26,13 +26,15 @@ use uuid::Uuid; /// the state of the system based on the `Blueprint`. pub struct BlueprintExecutor { datastore: Arc, - rx_blueprint: watch::Receiver>>, + rx_blueprint: watch::Receiver>>, } impl BlueprintExecutor { pub fn new( datastore: Arc, - rx_blueprint: watch::Receiver>>, + rx_blueprint: watch::Receiver< + Option>, + >, ) -> BlueprintExecutor { BlueprintExecutor { datastore, rx_blueprint } } @@ -136,16 +138,28 @@ impl BackgroundTask for BlueprintExecutor { async { // Get the latest blueprint, cloning to prevent holding a read lock // on the watch. - let blueprint = self.rx_blueprint.borrow_and_update().clone(); + let update = self.rx_blueprint.borrow_and_update().clone(); - let Some(blueprint) = blueprint else { + let Some(update) = update else { warn!(&opctx.log, - "Plan execution: skipped"; + "Blueprint execution: skipped"; "reason" => "no blueprint"); return json!({"error": "no blueprint" }); }; - let result = self.realize_blueprint(opctx, &blueprint).await; + let (bp_target, blueprint) = &*update; + if !bp_target.enabled { + warn!(&opctx.log, + "Blueprint execution: skipped"; + "reason" => "blueprint disabled", + "target_id" => %blueprint.id); + return json!({ + "target_id": blueprint.id.to_string(), + "error": "blueprint disabled" + }); + } + + let result = self.realize_blueprint(opctx, blueprint).await; // Return the result as a `serde_json::Value` match result { @@ -189,16 +203,24 @@ mod test { fn create_blueprint( omicron_zones: BTreeMap, - ) -> Blueprint { - Blueprint { - id: Uuid::new_v4(), - omicron_zones, - zones_in_service: BTreeSet::new(), - parent_blueprint_id: None, - time_created: chrono::Utc::now(), - creator: "test".to_string(), - comment: "test blueprint".to_string(), - } + ) -> (BlueprintTarget, Blueprint) { + let id = Uuid::new_v4(); + ( + BlueprintTarget { + target_id: id, + enabled: true, + time_made_target: chrono::Utc::now(), + }, + Blueprint { + id, + omicron_zones, + zones_in_service: BTreeSet::new(), + parent_blueprint_id: None, + time_created: chrono::Utc::now(), + creator: "test".to_string(), + comment: "test blueprint".to_string(), + }, + ) } #[nexus_test(server = crate::Server)] diff --git a/nexus/src/app/background/blueprint_load.rs b/nexus/src/app/background/blueprint_load.rs index ead51f2514..a6c45e51c2 100644 --- a/nexus/src/app/background/blueprint_load.rs +++ b/nexus/src/app/background/blueprint_load.rs @@ -12,16 +12,16 @@ use futures::future::BoxFuture; use futures::FutureExt; use nexus_db_queries::context::OpContext; use nexus_db_queries::db::DataStore; -use nexus_types::deployment::Blueprint; +use nexus_types::deployment::{Blueprint, BlueprintTarget}; use serde_json::json; use std::sync::Arc; use tokio::sync::watch; pub struct TargetBlueprintLoader { datastore: Arc, - last: Option>, - tx: watch::Sender>>, - rx: watch::Receiver>>, + last: Option>, + tx: watch::Sender>>, + rx: watch::Receiver>>, } impl TargetBlueprintLoader { @@ -31,7 +31,9 @@ impl TargetBlueprintLoader { } /// Expose the target blueprint - pub fn watcher(&self) -> watch::Receiver>> { + pub fn watcher( + &self, + ) -> watch::Receiver>> { self.rx.clone() } } @@ -47,8 +49,8 @@ impl BackgroundTask for TargetBlueprintLoader { let log = match &self.last { None => opctx.log.clone(), Some(old) => opctx.log.new(o!( - "current_target_id" => old.id.to_string(), - "current_time_created" => old.time_created.to_string(), + "current_target_id" => old.1.id.to_string(), + "current_time_created" => old.1.time_created.to_string(), )), }; @@ -81,9 +83,9 @@ impl BackgroundTask for TargetBlueprintLoader { let message = format!( "target blueprint with id {} was removed. There is no \ longer any target blueprint", - old.id + old.1.id ); - let old_id = old.id.to_string(); + let old_id = old.1.id.to_string(); self.last = None; self.tx.send_replace(self.last.clone()); error!(&log, "{message:?}"); @@ -93,18 +95,18 @@ impl BackgroundTask for TargetBlueprintLoader { "error": message }) } - (None, Ok(Some((_, new_target)))) => { + (None, Ok(Some((new_bp_target, new_blueprint)))) => { // We've found a target blueprint for the first time. // Save it and notify any watchers. - let target_id = new_target.id.to_string(); - let time_created = new_target.time_created.to_string(); + let target_id = new_blueprint.id.to_string(); + let time_created = new_blueprint.time_created.to_string(); info!( log, "found new target blueprint (first find)"; "target_id" => &target_id, "time_created" => &time_created ); - self.last = Some(Arc::new(new_target)); + self.last = Some(Arc::new((new_bp_target, new_blueprint))); self.tx.send_replace(self.last.clone()); json!({ "target_id": target_id, @@ -113,10 +115,10 @@ impl BackgroundTask for TargetBlueprintLoader { "status": "first target blueprint" }) } - (Some(old), Ok(Some((_, new)))) => { - let target_id = new.id.to_string(); - let time_created = new.time_created.to_string(); - if old.id != new.id { + (Some(old), Ok(Some((new_bp_target, new_blueprint)))) => { + let target_id = new_blueprint.id.to_string(); + let time_created = new_blueprint.time_created.to_string(); + if old.1.id != new_blueprint.id { // The current target blueprint has been updated info!( log, @@ -124,7 +126,8 @@ impl BackgroundTask for TargetBlueprintLoader { "target_id" => &target_id, "time_created" => &time_created ); - self.last = Some(Arc::new(new)); + self.last = + Some(Arc::new((new_bp_target, new_blueprint))); self.tx.send_replace(self.last.clone()); json!({ "target_id": target_id, @@ -139,7 +142,7 @@ impl BackgroundTask for TargetBlueprintLoader { // It should not be possible for the contents of a // blueprint to change, but we check to catch possible // bugs further up the stack. - if **old != new { + if old.1 != new_blueprint { let message = format!( "blueprint for id {} changed. \ Blueprints are supposed to be immutable.", From bbc131818ea7c13e9dd52897a3b812aefc9e5bf2 Mon Sep 17 00:00:00 2001 From: "Andrew J. Stone" Date: Tue, 30 Jan 2024 17:20:43 +0000 Subject: [PATCH 10/17] fix omdb tests --- dev-tools/omdb/src/bin/omdb/nexus.rs | 2 ++ dev-tools/omdb/tests/env.out | 24 ++++++++++++++++++++++++ dev-tools/omdb/tests/successes.out | 22 ++++++++++++++++++++++ 3 files changed, 48 insertions(+) diff --git a/dev-tools/omdb/src/bin/omdb/nexus.rs b/dev-tools/omdb/src/bin/omdb/nexus.rs index ea89923caa..f00c05f1ec 100644 --- a/dev-tools/omdb/src/bin/omdb/nexus.rs +++ b/dev-tools/omdb/src/bin/omdb/nexus.rs @@ -256,6 +256,8 @@ async fn cmd_nexus_background_tasks_show( "dns_servers_external", "dns_propagation_external", "nat_v4_garbage_collector", + "blueprint_loader", + "blueprint_executor", ] { if let Some(bgtask) = tasks.remove(name) { print_task(&bgtask); diff --git a/dev-tools/omdb/tests/env.out b/dev-tools/omdb/tests/env.out index 8cca1b063a..b6fd3fe9d1 100644 --- a/dev-tools/omdb/tests/env.out +++ b/dev-tools/omdb/tests/env.out @@ -23,6 +23,14 @@ EXECUTING COMMAND: omdb ["nexus", "--nexus-internal-url", "http://127.0.0.1:REDA termination: Exited(0) --------------------------------------------- stdout: +task: "blueprint_executor" + Executes the target blueprint + + +task: "blueprint_loader" + Loads the current target blueprint from the DB + + task: "dns_config_external" watches external DNS data stored in CockroachDB @@ -96,6 +104,14 @@ EXECUTING COMMAND: omdb ["nexus", "background-tasks", "doc"] termination: Exited(0) --------------------------------------------- stdout: +task: "blueprint_executor" + Executes the target blueprint + + +task: "blueprint_loader" + Loads the current target blueprint from the DB + + task: "dns_config_external" watches external DNS data stored in CockroachDB @@ -156,6 +172,14 @@ EXECUTING COMMAND: omdb ["--dns-server", "[::1]:REDACTED_PORT", "nexus", "backgr termination: Exited(0) --------------------------------------------- stdout: +task: "blueprint_executor" + Executes the target blueprint + + +task: "blueprint_loader" + Loads the current target blueprint from the DB + + task: "dns_config_external" watches external DNS data stored in CockroachDB diff --git a/dev-tools/omdb/tests/successes.out b/dev-tools/omdb/tests/successes.out index f291bbb6a0..01feedfec8 100644 --- a/dev-tools/omdb/tests/successes.out +++ b/dev-tools/omdb/tests/successes.out @@ -217,6 +217,14 @@ EXECUTING COMMAND: omdb ["nexus", "background-tasks", "doc"] termination: Exited(0) --------------------------------------------- stdout: +task: "blueprint_executor" + Executes the target blueprint + + +task: "blueprint_loader" + Loads the current target blueprint from the DB + + task: "dns_config_external" watches external DNS data stored in CockroachDB @@ -339,6 +347,20 @@ task: "nat_v4_garbage_collector" started at (s ago) and ran for ms warning: unknown background task: "nat_v4_garbage_collector" (don't know how to interpret details: Null) +task: "blueprint_loader" + configured period: every 10s + currently executing: no + last completed activation: iter 2, triggered by an explicit signal + started at (s ago) and ran for ms +warning: unknown background task: "blueprint_loader" (don't know how to interpret details: Object {"status": String("no target blueprint")}) + +task: "blueprint_executor" + configured period: every 1m + currently executing: no + last completed activation: iter 2, triggered by an explicit signal + started at (s ago) and ran for ms + last completion reported error: no blueprint + task: "external_endpoints" configured period: every 1m currently executing: no From 769cda3c5c3a5209c40b4f635ac4bcfd7fefbf47 Mon Sep 17 00:00:00 2001 From: "Andrew J. Stone" Date: Tue, 30 Jan 2024 17:21:02 +0000 Subject: [PATCH 11/17] nit fix --- nexus/src/app/background/blueprint_load.rs | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/nexus/src/app/background/blueprint_load.rs b/nexus/src/app/background/blueprint_load.rs index a6c45e51c2..6238c2b5d7 100644 --- a/nexus/src/app/background/blueprint_load.rs +++ b/nexus/src/app/background/blueprint_load.rs @@ -21,20 +21,19 @@ pub struct TargetBlueprintLoader { datastore: Arc, last: Option>, tx: watch::Sender>>, - rx: watch::Receiver>>, } impl TargetBlueprintLoader { pub fn new(datastore: Arc) -> TargetBlueprintLoader { - let (tx, rx) = watch::channel(None); - TargetBlueprintLoader { datastore, last: None, tx, rx } + let (tx, _) = watch::channel(None); + TargetBlueprintLoader { datastore, last: None, tx } } /// Expose the target blueprint pub fn watcher( &self, ) -> watch::Receiver>> { - self.rx.clone() + self.tx.subscribe() } } @@ -49,7 +48,7 @@ impl BackgroundTask for TargetBlueprintLoader { let log = match &self.last { None => opctx.log.clone(), Some(old) => opctx.log.new(o!( - "current_target_id" => old.1.id.to_string(), + "currentatarget_id" => old.1.id.to_string(), "current_time_created" => old.1.time_created.to_string(), )), }; From aab1bb02d62eec2de4c14c4a8e5dfd358a4e5dc5 Mon Sep 17 00:00:00 2001 From: "Andrew J. Stone" Date: Tue, 30 Jan 2024 17:22:23 +0000 Subject: [PATCH 12/17] nit fix --- nexus/src/app/background/blueprint_load.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/nexus/src/app/background/blueprint_load.rs b/nexus/src/app/background/blueprint_load.rs index 6238c2b5d7..c711aa6746 100644 --- a/nexus/src/app/background/blueprint_load.rs +++ b/nexus/src/app/background/blueprint_load.rs @@ -48,8 +48,8 @@ impl BackgroundTask for TargetBlueprintLoader { let log = match &self.last { None => opctx.log.clone(), Some(old) => opctx.log.new(o!( - "currentatarget_id" => old.1.id.to_string(), - "current_time_created" => old.1.time_created.to_string(), + "original_target_id" => old.1.id.to_string(), + "original_time_created" => old.1.time_created.to_string(), )), }; From ea5a8e0f2dddf2d6b0ec0c0094d8b495e94a68f6 Mon Sep 17 00:00:00 2001 From: "Andrew J. Stone" Date: Tue, 30 Jan 2024 22:35:55 +0000 Subject: [PATCH 13/17] Add a test for loading blueprints --- nexus/src/app/background/blueprint_load.rs | 116 +++++++++++++++++++++ 1 file changed, 116 insertions(+) diff --git a/nexus/src/app/background/blueprint_load.rs b/nexus/src/app/background/blueprint_load.rs index c711aa6746..24cae9afa6 100644 --- a/nexus/src/app/background/blueprint_load.rs +++ b/nexus/src/app/background/blueprint_load.rs @@ -176,3 +176,119 @@ impl BackgroundTask for TargetBlueprintLoader { .boxed() } } + +#[cfg(test)] +mod test { + use super::*; + use crate::app::background::common::BackgroundTask; + use nexus_inventory::now_db_precision; + use nexus_test_utils_macros::nexus_test; + use nexus_types::deployment::{Blueprint, BlueprintTarget}; + use serde::Deserialize; + use std::collections::{BTreeMap, BTreeSet}; + use uuid::Uuid; + + type ControlPlaneTestContext = + nexus_test_utils::ControlPlaneTestContext; + + fn create_blueprint( + parent_blueprint_id: Option, + ) -> (BlueprintTarget, Blueprint) { + let id = Uuid::new_v4(); + ( + BlueprintTarget { + target_id: id, + enabled: true, + time_made_target: now_db_precision(), + }, + Blueprint { + id, + omicron_zones: BTreeMap::new(), + zones_in_service: BTreeSet::new(), + parent_blueprint_id, + time_created: now_db_precision(), + creator: "test".to_string(), + comment: "test blueprint".to_string(), + }, + ) + } + + #[derive(Deserialize)] + #[allow(unused)] + struct TargetUpdate { + pub target_id: Uuid, + pub time_created: chrono::DateTime, + pub time_found: Option>, + pub status: String, + } + + #[nexus_test(server = crate::Server)] + async fn test_load_blueprints(cptestctx: &ControlPlaneTestContext) { + let nexus = &cptestctx.server.apictx().nexus; + let datastore = nexus.datastore(); + let opctx = OpContext::for_tests( + cptestctx.logctx.log.clone(), + datastore.clone(), + ); + + let mut task = TargetBlueprintLoader::new(datastore.clone()); + let mut rx = task.watcher(); + + // We expect an appropriate status with no blueprint in the datastore + let value = task.activate(&opctx).await; + assert_eq!(json!({"status": "no target blueprint"}), value); + assert!(rx.borrow().is_none()); + + let (target, blueprint) = create_blueprint(None); + + // Inserting a blueprint, but not making it the target returns the same + // status + datastore.blueprint_insert(&opctx, &blueprint).await.unwrap(); + let value = task.activate(&opctx).await; + assert_eq!(json!({"status": "no target blueprint"}), value); + assert!(rx.borrow().is_none()); + + // Setting a target blueprint makes the loader see it and broadcast it + datastore + .blueprint_target_set_current(&opctx, target.clone()) + .await + .unwrap(); + let value = task.activate(&opctx).await; + let update = serde_json::from_value::(value).unwrap(); + assert_eq!(update.target_id, blueprint.id); + assert_eq!(update.status, "first target blueprint"); + let rx_update = rx.borrow_and_update().clone().unwrap(); + assert_eq!(rx_update.0, target); + assert_eq!(rx_update.1, blueprint); + + // Activation without changing the target blueprint results in no update + let value = task.activate(&opctx).await; + let update = serde_json::from_value::(value).unwrap(); + assert_eq!(update.target_id, blueprint.id); + assert_eq!(update.status, "target blueprint unchanged"); + assert_eq!(false, rx.has_changed().unwrap()); + + // Adding a new blueprint and updating the target triggers a change + let (new_target, new_blueprint) = create_blueprint(Some(blueprint.id)); + datastore.blueprint_insert(&opctx, &new_blueprint).await.unwrap(); + datastore + .blueprint_target_set_current(&opctx, new_target.clone()) + .await + .unwrap(); + let value = task.activate(&opctx).await; + let update = serde_json::from_value::(value).unwrap(); + assert_eq!(update.target_id, new_blueprint.id); + assert_eq!(update.status, "target blueprint updated"); + let rx_update = rx.borrow_and_update().clone().unwrap(); + assert_eq!(rx_update.0, new_target); + assert_eq!(rx_update.1, new_blueprint); + + // Activating again without changing the target blueprint results in + // no update + let value = task.activate(&opctx).await; + let update = serde_json::from_value::(value).unwrap(); + assert_eq!(update.target_id, new_blueprint.id); + assert_eq!(update.status, "target blueprint unchanged"); + assert_eq!(false, rx.has_changed().unwrap()); + } +} From 61c3a00e3dc160ff601ed1be976321997b55df9b Mon Sep 17 00:00:00 2001 From: "Andrew J. Stone" Date: Tue, 30 Jan 2024 22:54:50 +0000 Subject: [PATCH 14/17] clippy --- nexus/src/app/background/blueprint_load.rs | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/nexus/src/app/background/blueprint_load.rs b/nexus/src/app/background/blueprint_load.rs index 24cae9afa6..760f7808b3 100644 --- a/nexus/src/app/background/blueprint_load.rs +++ b/nexus/src/app/background/blueprint_load.rs @@ -249,10 +249,7 @@ mod test { assert!(rx.borrow().is_none()); // Setting a target blueprint makes the loader see it and broadcast it - datastore - .blueprint_target_set_current(&opctx, target.clone()) - .await - .unwrap(); + datastore.blueprint_target_set_current(&opctx, target).await.unwrap(); let value = task.activate(&opctx).await; let update = serde_json::from_value::(value).unwrap(); assert_eq!(update.target_id, blueprint.id); From 57641dfcc7b891ceeb570dcfc41f3b01858ed486 Mon Sep 17 00:00:00 2001 From: "Andrew J. Stone" Date: Tue, 30 Jan 2024 22:57:07 +0000 Subject: [PATCH 15/17] remove unnecessary changes to omdb --- dev-tools/omdb/src/bin/omdb/nexus.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/dev-tools/omdb/src/bin/omdb/nexus.rs b/dev-tools/omdb/src/bin/omdb/nexus.rs index f00c05f1ec..ea89923caa 100644 --- a/dev-tools/omdb/src/bin/omdb/nexus.rs +++ b/dev-tools/omdb/src/bin/omdb/nexus.rs @@ -256,8 +256,6 @@ async fn cmd_nexus_background_tasks_show( "dns_servers_external", "dns_propagation_external", "nat_v4_garbage_collector", - "blueprint_loader", - "blueprint_executor", ] { if let Some(bgtask) = tasks.remove(name) { print_task(&bgtask); From dcf982d0bc3c957e92077c34af0b1fb39238b552 Mon Sep 17 00:00:00 2001 From: "Andrew J. Stone" Date: Tue, 30 Jan 2024 23:09:59 +0000 Subject: [PATCH 16/17] clippy --- nexus/src/app/background/blueprint_load.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nexus/src/app/background/blueprint_load.rs b/nexus/src/app/background/blueprint_load.rs index 760f7808b3..c34d2ab103 100644 --- a/nexus/src/app/background/blueprint_load.rs +++ b/nexus/src/app/background/blueprint_load.rs @@ -269,7 +269,7 @@ mod test { let (new_target, new_blueprint) = create_blueprint(Some(blueprint.id)); datastore.blueprint_insert(&opctx, &new_blueprint).await.unwrap(); datastore - .blueprint_target_set_current(&opctx, new_target.clone()) + .blueprint_target_set_current(&opctx, new_target) .await .unwrap(); let value = task.activate(&opctx).await; From 5661c86697e674383dc17765c075ed065d962bb1 Mon Sep 17 00:00:00 2001 From: "Andrew J. Stone" Date: Wed, 31 Jan 2024 19:12:07 +0000 Subject: [PATCH 17/17] fix tests --- dev-tools/omdb/src/bin/omdb/nexus.rs | 2 ++ dev-tools/omdb/tests/successes.out | 4 ++-- nexus/tests/config.test.toml | 4 ++-- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/dev-tools/omdb/src/bin/omdb/nexus.rs b/dev-tools/omdb/src/bin/omdb/nexus.rs index ea89923caa..f00c05f1ec 100644 --- a/dev-tools/omdb/src/bin/omdb/nexus.rs +++ b/dev-tools/omdb/src/bin/omdb/nexus.rs @@ -256,6 +256,8 @@ async fn cmd_nexus_background_tasks_show( "dns_servers_external", "dns_propagation_external", "nat_v4_garbage_collector", + "blueprint_loader", + "blueprint_executor", ] { if let Some(bgtask) = tasks.remove(name) { print_task(&bgtask); diff --git a/dev-tools/omdb/tests/successes.out b/dev-tools/omdb/tests/successes.out index 01feedfec8..f097681423 100644 --- a/dev-tools/omdb/tests/successes.out +++ b/dev-tools/omdb/tests/successes.out @@ -348,14 +348,14 @@ task: "nat_v4_garbage_collector" warning: unknown background task: "nat_v4_garbage_collector" (don't know how to interpret details: Null) task: "blueprint_loader" - configured period: every 10s + configured period: every 1m 40s currently executing: no last completed activation: iter 2, triggered by an explicit signal started at (s ago) and ran for ms warning: unknown background task: "blueprint_loader" (don't know how to interpret details: Object {"status": String("no target blueprint")}) task: "blueprint_executor" - configured period: every 1m + configured period: every 10m currently executing: no last completed activation: iter 2, triggered by an explicit signal started at (s ago) and ran for ms diff --git a/nexus/tests/config.test.toml b/nexus/tests/config.test.toml index bf8022f75d..cff9462c33 100644 --- a/nexus/tests/config.test.toml +++ b/nexus/tests/config.test.toml @@ -99,8 +99,8 @@ inventory.nkeep = 3 # Disable inventory collection altogether (for emergencies) inventory.disable = false phantom_disks.period_secs = 30 -blueprints.period_secs_load = 10 -blueprints.period_secs_execute = 60 +blueprints.period_secs_load = 100 +blueprints.period_secs_execute = 600 sync_service_zone_nat.period_secs = 30 [default_region_allocation_strategy]