From 9c002d3ccf9f3e9e9cce69fb7d968731335f6ec7 Mon Sep 17 00:00:00 2001 From: iliana etaoin Date: Thu, 30 Jan 2025 21:14:41 +0000 Subject: [PATCH 1/9] split logic --- common/src/api/internal/nexus.rs | 3 + nexus/src/app/update/mod.rs | 15 +- tufaceous-lib/src/artifact.rs | 28 +- tufaceous-lib/src/assemble/manifest.rs | 7 +- .../src/artifacts/artifacts_with_plan.rs | 97 ++++++- update-common/src/artifacts/update_plan.rs | 249 +++++++++++++++--- wicketd/src/update_tracker.rs | 5 +- wicketd/tests/integration_tests/updates.rs | 6 +- 8 files changed, 343 insertions(+), 67 deletions(-) diff --git a/common/src/api/internal/nexus.rs b/common/src/api/internal/nexus.rs index 6705569ae4..0396ffc28f 100644 --- a/common/src/api/internal/nexus.rs +++ b/common/src/api/internal/nexus.rs @@ -305,7 +305,10 @@ pub enum KnownArtifactKind { GimletRotBootloader, Host, Trampoline, + /// Composite artifact of all control plane zones ControlPlane, + /// Individual control plane zone + Zone, // PSC Artifacts PscSp, diff --git a/nexus/src/app/update/mod.rs b/nexus/src/app/update/mod.rs index 05ed9d7189..01e7285478 100644 --- a/nexus/src/app/update/mod.rs +++ b/nexus/src/app/update/mod.rs @@ -13,7 +13,7 @@ use nexus_db_queries::context::OpContext; use omicron_common::api::external::{ Error, SemverVersion, TufRepoInsertResponse, TufRepoInsertStatus, }; -use update_common::artifacts::ArtifactsWithPlan; +use update_common::artifacts::{ArtifactsWithPlan, ControlPlaneZonesMode}; mod common_sp_update; mod host_phase1_updater; @@ -51,10 +51,15 @@ impl super::Nexus { Error::internal_error("updates system not initialized") })?; - let artifacts_with_plan = - ArtifactsWithPlan::from_stream(body, Some(file_name), &self.log) - .await - .map_err(|error| error.to_http_error())?; + let artifacts_with_plan = ArtifactsWithPlan::from_stream( + body, + Some(file_name), + ControlPlaneZonesMode::Split, + &self.log, + ) + .await + .map_err(|error| error.to_http_error())?; + // Now store the artifacts in the database. let tuf_repo_description = TufRepoDescription::from_external( artifacts_with_plan.description().clone(), diff --git a/tufaceous-lib/src/artifact.rs b/tufaceous-lib/src/artifact.rs index c46540617b..418d6e18b6 100644 --- a/tufaceous-lib/src/artifact.rs +++ b/tufaceous-lib/src/artifact.rs @@ -346,17 +346,33 @@ pub struct ControlPlaneZoneImages { impl ControlPlaneZoneImages { pub fn extract(reader: R) -> Result { + let mut zones = Vec::new(); + Self::extract_into(reader, |name, reader| { + let mut buf = Vec::new(); + io::copy(reader, &mut buf)?; + zones.push((name, buf.into())); + Ok(()) + })?; + Ok(Self { zones }) + } + + pub fn extract_into(reader: R, mut handler: F) -> Result<()> + where + R: io::Read, + F: FnMut(String, &mut dyn io::Read) -> Result<()>, + { let uncompressed = flate2::bufread::GzDecoder::new(BufReader::new(reader)); let mut archive = tar::Archive::new(uncompressed); let mut oxide_json_found = false; - let mut zones = Vec::new(); + let mut zone_found = false; for entry in archive .entries() .context("error building list of entries from archive")? { - let entry = entry.context("error reading entry from archive")?; + let mut entry = + entry.context("error reading entry from archive")?; let path = entry .header() .path() @@ -382,9 +398,9 @@ impl ControlPlaneZoneImages { .and_then(|s| s.to_str()) .map(|s| s.to_string()) { - let data = read_entry(entry, &name)?; - zones.push((name, data)); + handler(name, &mut entry)?; } + zone_found = true; } } @@ -395,14 +411,14 @@ impl ControlPlaneZoneImages { if !not_found.is_empty() { bail!("required files not found: {}", not_found.join(", ")) } - if zones.is_empty() { + if !zone_found { bail!( "no zone images found in `{}/`", CONTROL_PLANE_ARCHIVE_ZONE_DIRECTORY ); } - Ok(Self { zones }) + Ok(()) } } diff --git a/tufaceous-lib/src/assemble/manifest.rs b/tufaceous-lib/src/assemble/manifest.rs index fa5df79982..9da4e58861 100644 --- a/tufaceous-lib/src/assemble/manifest.rs +++ b/tufaceous-lib/src/assemble/manifest.rs @@ -224,7 +224,9 @@ impl ArtifactManifest { /// Checks that all expected artifacts are present, returning an error with /// details if any artifacts are missing. pub fn verify_all_present(&self) -> Result<()> { - let all_artifacts: BTreeSet<_> = KnownArtifactKind::iter().collect(); + let all_artifacts: BTreeSet<_> = KnownArtifactKind::iter() + .filter(|k| !matches!(k, KnownArtifactKind::Zone)) + .collect(); let present_artifacts: BTreeSet<_> = self.artifacts.keys().copied().collect(); @@ -261,7 +263,8 @@ impl<'a> FakeDataAttributes<'a> { // non-Hubris artifacts: just make fake data KnownArtifactKind::Host | KnownArtifactKind::Trampoline - | KnownArtifactKind::ControlPlane => return make_filler_text(size), + | KnownArtifactKind::ControlPlane + | KnownArtifactKind::Zone => return make_filler_text(size), // hubris artifacts: build a fake archive (SimGimletSp and // SimGimletRot are used by sp-sim) diff --git a/update-common/src/artifacts/artifacts_with_plan.rs b/update-common/src/artifacts/artifacts_with_plan.rs index 650efccdfb..2a7a27a9f6 100644 --- a/update-common/src/artifacts/artifacts_with_plan.rs +++ b/update-common/src/artifacts/artifacts_with_plan.rs @@ -75,6 +75,7 @@ impl ArtifactsWithPlan { pub async fn from_stream( body: impl Stream> + Send, file_name: Option, + zone_mode: ControlPlaneZonesMode, log: &Logger, ) -> Result { // Create a temporary file to store the incoming archive.`` @@ -115,6 +116,7 @@ impl ArtifactsWithPlan { io::BufReader::new(tempfile), file_name, repo_hash, + zone_mode, log, ) .await?; @@ -126,6 +128,7 @@ impl ArtifactsWithPlan { zip_data: T, file_name: Option, repo_hash: ArtifactHash, + zone_mode: ControlPlaneZonesMode, log: &Logger, ) -> Result where @@ -174,8 +177,11 @@ impl ArtifactsWithPlan { // these are just direct copies of artifacts we just unpacked into // `dir`, but we'll also unpack nested artifacts like the RoT dual A/B // archives. - let mut builder = - UpdatePlanBuilder::new(artifacts.system_version.clone(), log)?; + let mut builder = UpdatePlanBuilder::new( + artifacts.system_version.clone(), + zone_mode, + log, + )?; // Make a pass through each artifact in the repo. For each artifact, we // do one of the following: @@ -316,6 +322,16 @@ impl ArtifactsWithPlan { } } +#[derive(Debug, Clone, Copy)] +pub enum ControlPlaneZonesMode { + /// Ensure the control plane zones are combined into a single composite + /// `ControlPlane` artifact, used by Wicket. + Composite, + /// Ensure the control plane zones are individual `Zone` artifacts, used + /// by Nexus. + Split, +} + fn unzip_into_tempdir( zip_data: T, log: &Logger, @@ -366,8 +382,12 @@ mod tests { create_fake_archive(&logctx.log, &archive_path).await?; // Now check that it can be read by the archive extractor. - let plan = - build_artifacts_with_plan(&logctx.log, &archive_path).await?; + let plan = build_artifacts_with_plan( + &logctx.log, + &archive_path, + ControlPlaneZonesMode::Composite, + ) + .await?; // Check that all known artifact kinds are present in the map. let by_id_kinds: BTreeSet<_> = plan.by_id().keys().map(|id| id.kind.clone()).collect(); @@ -380,9 +400,12 @@ mod tests { .map(|meta| meta.id.kind.clone()) .collect(); - // `by_id` should contain one entry for every `KnownArtifactKind`... - let mut expected_kinds: BTreeSet<_> = - KnownArtifactKind::iter().map(ArtifactKind::from).collect(); + // `by_id` should contain one entry for every `KnownArtifactKind` + // (except `Zone`)... + let mut expected_kinds: BTreeSet<_> = KnownArtifactKind::iter() + .filter(|k| !matches!(k, KnownArtifactKind::Zone)) + .map(ArtifactKind::from) + .collect(); assert_eq!( expected_kinds, by_id_kinds, "expected kinds match by_id kinds" @@ -441,6 +464,41 @@ mod tests { Ok(()) } + #[tokio::test(flavor = "multi_thread", worker_threads = 2)] + async fn test_extract_fake_split() -> Result<()> { + let logctx = test_setup_log("test_extract_fake"); + let temp_dir = Utf8TempDir::new()?; + let archive_path = temp_dir.path().join("archive.zip"); + + // Create the archive. + create_fake_archive(&logctx.log, &archive_path).await?; + + // Now check that it can be read by the archive extractor in split mode. + let plan = build_artifacts_with_plan( + &logctx.log, + &archive_path, + ControlPlaneZonesMode::Split, + ) + .await?; + + // `by_id` should contain one entry for every `KnownArtifactKind` + // (except `ControlPlane`). + let by_id_kinds: BTreeSet<_> = + plan.by_id().keys().map(|id| id.kind.clone()).collect(); + let expected_kinds: BTreeSet<_> = KnownArtifactKind::iter() + .filter(|k| !matches!(k, KnownArtifactKind::ControlPlane)) + .map(ArtifactKind::from) + .collect(); + assert_eq!( + expected_kinds, by_id_kinds, + "expected kinds match by_id kinds" + ); + + logctx.cleanup_successful(); + + Ok(()) + } + /// Test that the archive generated by running `tufaceous assemble` twice /// has the same artifacts and hashes. #[tokio::test(flavor = "multi_thread", worker_threads = 2)] @@ -451,8 +509,12 @@ mod tests { // Create the archive and build a plan from it. create_fake_archive(&logctx.log, &archive_path).await?; - let mut plan1 = - build_artifacts_with_plan(&logctx.log, &archive_path).await?; + let mut plan1 = build_artifacts_with_plan( + &logctx.log, + &archive_path, + ControlPlaneZonesMode::Composite, + ) + .await?; // Add a 2 second delay to ensure that if we bake any second-based // timestamps in, that they end up being different from those in the @@ -461,8 +523,12 @@ mod tests { let archive2_path = temp_dir.path().join("archive2.zip"); create_fake_archive(&logctx.log, &archive2_path).await?; - let mut plan2 = - build_artifacts_with_plan(&logctx.log, &archive2_path).await?; + let mut plan2 = build_artifacts_with_plan( + &logctx.log, + &archive2_path, + ControlPlaneZonesMode::Composite, + ) + .await?; // At the moment, the repo .zip itself doesn't match because it bakes // in timestamps. However, the artifacts inside should match exactly. @@ -499,15 +565,18 @@ mod tests { async fn build_artifacts_with_plan( log: &slog::Logger, archive_path: &Utf8Path, + zone_mode: ControlPlaneZonesMode, ) -> Result { let zip_bytes = std::fs::File::open(&archive_path) .context("error opening archive.zip")?; // We could also compute the hash from the file here, but the repo hash // doesn't matter for the test. let repo_hash = ArtifactHash([0u8; 32]); - let plan = ArtifactsWithPlan::from_zip(zip_bytes, None, repo_hash, log) - .await - .with_context(|| format!("error reading {archive_path}"))?; + let plan = ArtifactsWithPlan::from_zip( + zip_bytes, None, repo_hash, zone_mode, log, + ) + .await + .with_context(|| format!("error reading {archive_path}"))?; Ok(plan) } diff --git a/update-common/src/artifacts/update_plan.rs b/update-common/src/artifacts/update_plan.rs index f5ff081651..d1ec4652ad 100644 --- a/update-common/src/artifacts/update_plan.rs +++ b/update-common/src/artifacts/update_plan.rs @@ -11,6 +11,7 @@ use super::ArtifactIdData; use super::Board; +use super::ControlPlaneZonesMode; use super::ExtractedArtifactDataHandle; use super::ExtractedArtifacts; use super::HashingNamedUtf8TempFile; @@ -35,6 +36,7 @@ use std::collections::BTreeMap; use std::collections::HashMap; use std::io; use tokio::io::AsyncReadExt; +use tufaceous_lib::ControlPlaneZoneImages; use tufaceous_lib::HostPhaseImages; use tufaceous_lib::RotArchives; @@ -75,6 +77,9 @@ pub struct UpdatePlan { // We also need to send installinator the hash of the control_plane image it // should fetch from us. This is already present in the TUF repository, but // we record it here for use by the update process. + // + // When built with `ControlPlaneZonesMode::Split`, this hash does not + // reference any artifacts in our corresponding `ArtifactsWithPlan`. pub control_plane_hash: ArtifactHash, } @@ -146,12 +151,14 @@ pub struct UpdatePlanBuilder<'a> { // extra fields we use to build the plan extracted_artifacts: ExtractedArtifacts, + zone_mode: ControlPlaneZonesMode, log: &'a Logger, } impl<'a> UpdatePlanBuilder<'a> { pub fn new( system_version: SemverVersion, + zone_mode: ControlPlaneZonesMode, log: &'a Logger, ) -> Result { let extracted_artifacts = ExtractedArtifacts::new(log)?; @@ -181,6 +188,7 @@ impl<'a> UpdatePlanBuilder<'a> { artifacts_meta: Vec::new(), extracted_artifacts, + zone_mode, log, }) } @@ -246,6 +254,12 @@ impl<'a> UpdatePlanBuilder<'a> { ) .await } + KnownArtifactKind::Zone => { + // We don't currently support repos with already split-out + // zones. + self.add_unknown_artifact(artifact_id, artifact_hash, stream) + .await + } } } @@ -265,6 +279,7 @@ impl<'a> UpdatePlanBuilder<'a> { | KnownArtifactKind::Host | KnownArtifactKind::Trampoline | KnownArtifactKind::ControlPlane + | KnownArtifactKind::Zone | KnownArtifactKind::PscRot | KnownArtifactKind::SwitchRot | KnownArtifactKind::GimletRotBootloader @@ -357,6 +372,7 @@ impl<'a> UpdatePlanBuilder<'a> { | KnownArtifactKind::Host | KnownArtifactKind::Trampoline | KnownArtifactKind::ControlPlane + | KnownArtifactKind::Zone | KnownArtifactKind::PscRot | KnownArtifactKind::SwitchRot | KnownArtifactKind::GimletSp @@ -451,6 +467,7 @@ impl<'a> UpdatePlanBuilder<'a> { | KnownArtifactKind::Host | KnownArtifactKind::Trampoline | KnownArtifactKind::ControlPlane + | KnownArtifactKind::Zone | KnownArtifactKind::PscSp | KnownArtifactKind::SwitchSp | KnownArtifactKind::GimletRotBootloader @@ -708,24 +725,91 @@ impl<'a> UpdatePlanBuilder<'a> { )); } - // The control plane artifact is the easiest one: we just need to copy - // it into our tempdir and record it. Nothing to inspect or extract. - let artifact_hash_id = ArtifactHashId { - kind: artifact_id.kind.clone(), - hash: artifact_hash, - }; - - let data = - self.extracted_artifacts.store(artifact_hash_id, stream).await?; + match self.zone_mode { + ControlPlaneZonesMode::Composite => { + // Just copy it into our tempdir and record it. + let artifact_hash_id = ArtifactHashId { + kind: artifact_id.kind.clone(), + hash: artifact_hash, + }; + let data = self + .extracted_artifacts + .store(artifact_hash_id, stream) + .await?; + self.record_extracted_artifact( + artifact_id, + data, + KnownArtifactKind::ControlPlane.into(), + self.log, + )?; + } + ControlPlaneZonesMode::Split => { + // Extract each zone image into its own artifact. + + // Since stream isn't guaranteed to be 'static, we have to + // use block_in_place here, not spawn_blocking. This does mean + // that the current task is taken over, and that this function + // can only be used from a multithreaded Tokio runtime. (See + // `extract_nested_artifact_pair` for more commentary on + // alternatives.) + tokio::task::block_in_place(|| { + let stream = std::pin::pin!(stream); + let reader = tokio_util::io::StreamReader::new( + stream.map_err(|error| { + // StreamReader requires a conversion from tough's errors to + // std::io::Error. + std::io::Error::new(io::ErrorKind::Other, error) + }), + ); - self.control_plane_hash = Some(data.hash()); + // ControlPlaneZoneImages::extract_into takes a synchronous + // reader, so we need to use this bridge. The bridge can + // only be used from a blocking context. + let reader = tokio_util::io::SyncIoBridge::new(reader); + + ControlPlaneZoneImages::extract_into( + reader, + |name, reader| { + let mut out = + self.extracted_artifacts.new_tempfile()?; + io::copy(reader, &mut out)?; + let data = + self.extracted_artifacts.store_tempfile( + KnownArtifactKind::Zone.into(), + out, + )?; + let artifact_id = ArtifactId { + name, + version: artifact_id.version.clone(), + kind: KnownArtifactKind::Zone.into(), + }; + self.record_extracted_artifact( + artifact_id, + data, + KnownArtifactKind::Zone.into(), + self.log, + )?; + Ok(()) + }, + ) + }) + .map_err(|error| { + // Fish the original RepositoryError out of this + // anyhow::Error if it is one. + error.downcast().unwrap_or_else(|error| { + RepositoryError::TarballExtract { + kind: KnownArtifactKind::ControlPlane, + error, + } + }) + })?; + } + } - self.record_extracted_artifact( - artifact_id, - data, - KnownArtifactKind::ControlPlane.into(), - self.log, - )?; + // Even if we split the control plane artifact, use this as a marker + // that we've seen the artifact before. The hash is meaningless in + // `Split` mode. + self.control_plane_hash = Some(artifact_hash); Ok(()) } @@ -1184,7 +1268,9 @@ mod tests { use omicron_test_utils::dev::test_setup_log; use rand::{distributions::Standard, thread_rng, Rng}; use sha2::{Digest, Sha256}; - use tufaceous_lib::{CompositeEntry, MtimeSource}; + use tufaceous_lib::{ + CompositeControlPlaneArchiveBuilder, CompositeEntry, MtimeSource, + }; fn make_random_bytes() -> Vec { thread_rng().sample_iter(Standard).take(128).collect() @@ -1369,8 +1455,12 @@ mod tests { let logctx = test_setup_log("test_bad_rot_version"); - let mut plan_builder = - UpdatePlanBuilder::new(VERSION_0, &logctx.log).unwrap(); + let mut plan_builder = UpdatePlanBuilder::new( + VERSION_0, + ControlPlaneZonesMode::Composite, + &logctx.log, + ) + .unwrap(); // The control plane artifact can be arbitrary bytes; just populate it // with random data. @@ -1537,9 +1627,12 @@ mod tests { let logctx = test_setup_log("test_multi_rot_version"); - let mut plan_builder = - UpdatePlanBuilder::new("0.0.0".parse().unwrap(), &logctx.log) - .unwrap(); + let mut plan_builder = UpdatePlanBuilder::new( + "0.0.0".parse().unwrap(), + ControlPlaneZonesMode::Composite, + &logctx.log, + ) + .unwrap(); // The control plane artifact can be arbitrary bytes; just populate it // with random data. @@ -1722,9 +1815,12 @@ mod tests { let logctx = test_setup_log("test_update_plan_from_artifacts"); - let mut plan_builder = - UpdatePlanBuilder::new("0.0.0".parse().unwrap(), &logctx.log) - .unwrap(); + let mut plan_builder = UpdatePlanBuilder::new( + "0.0.0".parse().unwrap(), + ControlPlaneZonesMode::Composite, + &logctx.log, + ) + .unwrap(); // Add a couple artifacts with kinds wicketd/nexus don't understand; it // should still ingest and serve them. @@ -1919,6 +2015,12 @@ mod tests { assert_eq!(hash_ids.len(), 1); assert_eq!(plan.control_plane_hash, hash_ids[0].hash); } + KnownArtifactKind::Zone => { + unreachable!( + "tufaceous does not yet generate repos \ + with split-out control plane zones" + ); + } KnownArtifactKind::PscSp => { assert!( id.name.starts_with("test-psc-"), @@ -2018,9 +2120,12 @@ mod tests { let logctx = test_setup_log("test_bad_hubris_cabooses"); - let mut plan_builder = - UpdatePlanBuilder::new("0.0.0".parse().unwrap(), &logctx.log) - .unwrap(); + let mut plan_builder = UpdatePlanBuilder::new( + "0.0.0".parse().unwrap(), + ControlPlaneZonesMode::Composite, + &logctx.log, + ) + .unwrap(); let gimlet_rot = make_bad_rot_image("gimlet"); let psc_rot = make_bad_rot_image("psc"); @@ -2096,9 +2201,12 @@ mod tests { // bootloader let logctx = test_setup_log("test_too_many_rot_bootloader"); - let mut plan_builder = - UpdatePlanBuilder::new("0.0.0".parse().unwrap(), &logctx.log) - .unwrap(); + let mut plan_builder = UpdatePlanBuilder::new( + "0.0.0".parse().unwrap(), + ControlPlaneZonesMode::Composite, + &logctx.log, + ) + .unwrap(); let gimlet_rot_bootloader = make_fake_rot_bootloader_image("test-gimlet-a", "test-gimlet-a"); @@ -2159,9 +2267,12 @@ mod tests { let logctx = test_setup_log("test_update_plan_from_artifacts"); - let mut plan_builder = - UpdatePlanBuilder::new("0.0.0".parse().unwrap(), &logctx.log) - .unwrap(); + let mut plan_builder = UpdatePlanBuilder::new( + "0.0.0".parse().unwrap(), + ControlPlaneZonesMode::Composite, + &logctx.log, + ) + .unwrap(); // The control plane artifact can be arbitrary bytes; just populate it // with random data. @@ -2320,9 +2431,12 @@ mod tests { let logctx = test_setup_log("test_update_plan_from_artifacts"); - let mut plan_builder = - UpdatePlanBuilder::new("0.0.0".parse().unwrap(), &logctx.log) - .unwrap(); + let mut plan_builder = UpdatePlanBuilder::new( + "0.0.0".parse().unwrap(), + ControlPlaneZonesMode::Composite, + &logctx.log, + ) + .unwrap(); let gimlet_rot = make_random_rot_image("gimlet", "gimlet", "gitc1"); let gimlet2_rot = make_random_rot_image("gimlet", "gimlet", "gitc2"); @@ -2359,6 +2473,67 @@ mod tests { logctx.cleanup_successful(); } + #[tokio::test(flavor = "multi_thread", worker_threads = 2)] + async fn test_split_control_plane() { + const VERSION_0: SemverVersion = SemverVersion::new(0, 0, 0); + const ZONES: &[(&str, &[u8])] = + &[("first", b"the first zone"), ("second", b"the second zone")]; + + let logctx = test_setup_log("test_split_control_plane"); + + let mut cp_builder = CompositeControlPlaneArchiveBuilder::new( + Vec::new(), + MtimeSource::Now, + ) + .unwrap(); + for (name, data) in ZONES { + cp_builder + .append_zone( + name, + CompositeEntry { data, mtime_source: MtimeSource::Now }, + ) + .unwrap(); + } + let data = Bytes::from(cp_builder.finish().unwrap()); + + let mut plan_builder = UpdatePlanBuilder::new( + VERSION_0, + ControlPlaneZonesMode::Split, + &logctx.log, + ) + .unwrap(); + let hash = ArtifactHash(Sha256::digest(&data).into()); + let id = ArtifactId { + name: "control_plane".into(), + version: VERSION_0, + kind: KnownArtifactKind::ControlPlane.into(), + }; + plan_builder + .add_artifact(id, hash, futures::stream::iter([Ok(data.clone())])) + .await + .unwrap(); + + // All of the artifacts created should be Zones (and notably, not + // ControlPlane). Their artifact hashes should match the calculated hash + // of the zone contents. + for (id, vec) in &plan_builder.by_id { + let content = + ZONES.iter().find(|(name, _)| *name == id.name).unwrap().1; + let expected_hash = ArtifactHash(Sha256::digest(content).into()); + assert_eq!(id.version, VERSION_0); + assert_eq!(id.kind, KnownArtifactKind::Zone.into()); + assert_eq!( + vec, + &vec![ArtifactHashId { + kind: KnownArtifactKind::Zone.into(), + hash: expected_hash + }] + ); + } + + logctx.cleanup_successful(); + } + async fn read_to_vec(data: &ExtractedArtifactDataHandle) -> Vec { let mut buf = Vec::with_capacity(data.file_size()); let mut stream = data.reader_stream().await.unwrap(); diff --git a/wicketd/src/update_tracker.rs b/wicketd/src/update_tracker.rs index b1fb5ae326..43b550a2d8 100644 --- a/wicketd/src/update_tracker.rs +++ b/wicketd/src/update_tracker.rs @@ -62,6 +62,7 @@ use tokio::task::JoinHandle; use tokio_util::io::StreamReader; use update_common::artifacts::ArtifactIdData; use update_common::artifacts::ArtifactsWithPlan; +use update_common::artifacts::ControlPlaneZonesMode; use update_common::artifacts::UpdatePlan; use update_engine::events::ProgressUnits; use update_engine::AbortHandle; @@ -356,7 +357,9 @@ impl UpdateTracker { stream, // We don't have a good file name here because file contents are // uploaded over stdin, so let ArtifactsWithPlan pick the name. - None, &self.log, + None, + ControlPlaneZonesMode::Composite, + &self.log, ) .await .map_err(|error| error.to_http_error())?; diff --git a/wicketd/tests/integration_tests/updates.rs b/wicketd/tests/integration_tests/updates.rs index af3bbfe656..547601da8a 100644 --- a/wicketd/tests/integration_tests/updates.rs +++ b/wicketd/tests/integration_tests/updates.rs @@ -70,8 +70,10 @@ async fn test_updates() { .into_inner(); // We should have an artifact for every known artifact kind... - let expected_kinds: BTreeSet<_> = - KnownArtifactKind::iter().map(ArtifactKind::from).collect(); + let expected_kinds: BTreeSet<_> = KnownArtifactKind::iter() + .filter(|k| !matches!(k, KnownArtifactKind::Zone)) + .map(ArtifactKind::from) + .collect(); // ... and installable artifacts that replace the top level host, // trampoline, and RoT with their inner parts (phase1/phase2 for OS images From 7d75aabc10e4c869126eeb9af7d4c81e06bb6354 Mon Sep 17 00:00:00 2001 From: iliana etaoin Date: Thu, 30 Jan 2025 21:23:59 +0000 Subject: [PATCH 2/9] forbid zones with the same checksum --- tufaceous-lib/src/artifact/composite.rs | 14 +++++++++++++- tufaceous/manifests/fake.toml | 2 +- 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/tufaceous-lib/src/artifact/composite.rs b/tufaceous-lib/src/artifact/composite.rs index e7793868c9..6bcc183dc8 100644 --- a/tufaceous-lib/src/artifact/composite.rs +++ b/tufaceous-lib/src/artifact/composite.rs @@ -16,6 +16,9 @@ use anyhow::Result; use camino::Utf8Path; use flate2::write::GzEncoder; use flate2::Compression; +use sha2::Digest; +use sha2::Sha256; +use std::collections::HashMap; use std::io::BufWriter; use std::io::Write; @@ -33,6 +36,7 @@ pub struct CompositeEntry<'a> { pub struct CompositeControlPlaneArchiveBuilder { inner: CompositeTarballBuilder, + hashes: HashMap<[u8; 32], String>, } impl CompositeControlPlaneArchiveBuilder { @@ -44,7 +48,7 @@ impl CompositeControlPlaneArchiveBuilder { .context("error building oxide metadata")?; let inner = CompositeTarballBuilder::new(writer, metadata, mtime_source)?; - Ok(Self { inner }) + Ok(Self { inner, hashes: HashMap::new() }) } pub fn append_zone( @@ -56,6 +60,14 @@ impl CompositeControlPlaneArchiveBuilder { if name_path.file_name() != Some(name) { bail!("control plane zone filenames should not contain paths"); } + if let Some(duplicate) = + self.hashes.insert(Sha256::digest(&entry.data).into(), name.into()) + { + bail!( + "duplicate zones are not allowed \ + ({name} and {duplicate} have the same checksum)" + ); + } let path = Utf8Path::new(CONTROL_PLANE_ARCHIVE_ZONE_DIRECTORY).join(name_path); self.inner.append_file(path.as_str(), entry) diff --git a/tufaceous/manifests/fake.toml b/tufaceous/manifests/fake.toml index c3f6404f53..f425932bc2 100644 --- a/tufaceous/manifests/fake.toml +++ b/tufaceous/manifests/fake.toml @@ -39,7 +39,7 @@ version = "1.0.0" [artifact.control_plane.source] kind = "composite-control-plane" zones = [ - { kind = "fake", name = "zone1.tar.gz", size = "1MiB" }, + { kind = "fake", name = "zone1.tar.gz", size = "512KiB" }, { kind = "fake", name = "zone2.tar.gz", size = "1MiB" }, ] From 3287b535de4e9e616f0131822c5f62012b31e9f4 Mon Sep 17 00:00:00 2001 From: iliana etaoin Date: Thu, 30 Jan 2025 23:35:21 +0000 Subject: [PATCH 3/9] check the split in the nexus integration test --- nexus/tests/integration_tests/updates.rs | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/nexus/tests/integration_tests/updates.rs b/nexus/tests/integration_tests/updates.rs index 51d93f49ca..35252c8551 100644 --- a/nexus/tests/integration_tests/updates.rs +++ b/nexus/tests/integration_tests/updates.rs @@ -131,6 +131,17 @@ async fn test_repo_upload() -> Result<()> { .map(|artifact| artifact.hash) .collect::>() .len(); + // The repository description should have `Zone` artifacts instead of the + // composite `ControlPlane` artifact. + assert!(initial_description + .artifacts + .iter() + .any(|artifact| artifact.id.kind == KnownArtifactKind::Zone.into())); + assert!(!initial_description + .artifacts + .iter() + .any(|artifact| artifact.id.kind + == KnownArtifactKind::ControlPlane.into())); // The artifact replication background task should have been activated, and // we should see a local repo and successful PUTs. From 830255895c2b7b1b197928a7060728d43725d6ce Mon Sep 17 00:00:00 2001 From: iliana etaoin Date: Fri, 31 Jan 2025 04:33:01 +0000 Subject: [PATCH 4/9] read zone name and version from oxide.json --- Cargo.lock | 18 +++ Cargo.toml | 3 + brand-metadata/Cargo.toml | 15 ++ brand-metadata/src/lib.rs | 124 ++++++++++++++++ .../tasks/tuf_artifact_replication.rs | 2 +- nexus/tests/integration_tests/updates.rs | 18 ++- sled-agent/Cargo.toml | 1 + sled-agent/src/updates.rs | 89 +++--------- tufaceous-lib/Cargo.toml | 2 + tufaceous-lib/src/artifact.rs | 9 +- tufaceous-lib/src/artifact/composite.rs | 23 +-- tufaceous-lib/src/assemble/manifest.rs | 33 ++++- tufaceous-lib/src/lib.rs | 1 - tufaceous-lib/src/oxide_metadata.rs | 133 ------------------ tufaceous/manifests/fake.toml | 4 +- update-common/Cargo.toml | 4 + .../src/artifacts/extracted_artifacts.rs | 7 +- update-common/src/artifacts/update_plan.rs | 129 +++++++++-------- 18 files changed, 315 insertions(+), 300 deletions(-) create mode 100644 brand-metadata/Cargo.toml create mode 100644 brand-metadata/src/lib.rs delete mode 100644 tufaceous-lib/src/oxide_metadata.rs diff --git a/Cargo.lock b/Cargo.lock index 20a6d44e2e..41dacdffcb 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6673,6 +6673,17 @@ dependencies = [ "unicode-normalization", ] +[[package]] +name = "omicron-brand-metadata" +version = "0.1.0" +dependencies = [ + "omicron-workspace-hack", + "semver 1.0.24", + "serde", + "serde_json", + "tar", +] + [[package]] name = "omicron-certificates" version = "0.1.0" @@ -7336,6 +7347,7 @@ dependencies = [ "nexus-reconfigurator-blippy", "nexus-sled-agent-shared", "nexus-types", + "omicron-brand-metadata", "omicron-common", "omicron-ddm-admin-client", "omicron-test-utils", @@ -12406,11 +12418,13 @@ dependencies = [ "hex", "hubtools", "itertools 0.13.0", + "omicron-brand-metadata", "omicron-common", "omicron-test-utils", "omicron-workspace-hack", "parse-size", "rand", + "semver 1.0.24", "serde", "serde_json", "serde_path_to_error", @@ -12752,15 +12766,19 @@ dependencies = [ "debug-ignore", "display-error-chain", "dropshot 0.15.1", + "flate2", + "fs-err", "futures", "hex", "hubtools", + "omicron-brand-metadata", "omicron-common", "omicron-test-utils", "omicron-workspace-hack", "rand", "sha2", "slog", + "tar", "thiserror 1.0.69", "tokio", "tokio-util", diff --git a/Cargo.toml b/Cargo.toml index 98f571b826..b0505ae695 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -2,6 +2,7 @@ members = [ "api_identity", "bootstore", + "brand-metadata", "certificates", "clickhouse-admin", "clickhouse-admin/api", @@ -135,6 +136,7 @@ members = [ default-members = [ "api_identity", "bootstore", + "brand-metadata", "certificates", "clickhouse-admin", "clickhouse-admin/api", @@ -493,6 +495,7 @@ nexus-types = { path = "nexus/types" } nom = "7.1.3" num-integer = "0.1.46" num = { version = "0.4.3", default-features = false, features = [ "libm" ] } +omicron-brand-metadata = { path = "brand-metadata" } omicron-clickhouse-admin = { path = "clickhouse-admin" } omicron-certificates = { path = "certificates" } omicron-cockroach-admin = { path = "cockroach-admin" } diff --git a/brand-metadata/Cargo.toml b/brand-metadata/Cargo.toml new file mode 100644 index 0000000000..98462c695e --- /dev/null +++ b/brand-metadata/Cargo.toml @@ -0,0 +1,15 @@ +[package] +name = "omicron-brand-metadata" +version = "0.1.0" +edition = "2021" +license = "MPL-2.0" + +[dependencies] +omicron-workspace-hack.workspace = true +semver.workspace = true +serde.workspace = true +serde_json.workspace = true +tar.workspace = true + +[lints] +workspace = true diff --git a/brand-metadata/src/lib.rs b/brand-metadata/src/lib.rs new file mode 100644 index 0000000000..7dd13209cb --- /dev/null +++ b/brand-metadata/src/lib.rs @@ -0,0 +1,124 @@ +// 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/. + +//! Handling of `oxide.json` metadata files in tarballs. +//! +//! `oxide.json` is originally defined by the omicron1(7) zone brand, which +//! lives at https://github.com/oxidecomputer/helios-omicron-brand. tufaceous +//! extended this format with additional archive types for identifying other +//! types of tarballs; this crate covers those extensions so they can be used +//! across the Omicron codebase. + +use std::io::{Error, ErrorKind, Read, Result, Write}; + +use serde::{Deserialize, Serialize}; + +#[derive(Clone, Debug, Deserialize, Serialize)] +pub struct Metadata { + v: String, + + // helios-build-utils defines a top-level `i` field for extra information, + // but omicron-package doesn't use this for the package name and version. + // We can also benefit from having rich types for these extra fields, so + // any additional top-level fields (including `i`) that exist for a given + // archive type should be deserialized as part of `ArchiveType`. + #[serde(flatten)] + t: ArchiveType, +} + +#[derive(Clone, Debug, Deserialize, Serialize)] +pub enum ArchiveType { + // Originally defined in helios-build-utils (part of helios-omicron-brand): + Baseline, + Layer(LayerInfo), + Os, + + // tufaceous extensions: + Rot, + ControlPlane, +} + +#[derive(Clone, Debug, Deserialize, Serialize)] +pub struct LayerInfo { + pub pkg: String, + pub version: semver::Version, +} + +impl Metadata { + pub fn new(archive_type: ArchiveType) -> Metadata { + Metadata { v: "1".into(), t: archive_type } + } + + pub fn append_to_tar( + &self, + a: &mut tar::Builder, + mtime: u64, + ) -> Result<()> { + let mut b = serde_json::to_vec(self)?; + b.push(b'\n'); + + let mut h = tar::Header::new_ustar(); + h.set_entry_type(tar::EntryType::Regular); + h.set_username("root")?; + h.set_uid(0); + h.set_groupname("root")?; + h.set_gid(0); + h.set_path("oxide.json")?; + h.set_mode(0o444); + h.set_size(b.len().try_into().unwrap()); + h.set_mtime(mtime); + h.set_cksum(); + + a.append(&h, b.as_slice())?; + Ok(()) + } + + /// Read `Metadata` from a tar archive. + /// + /// `oxide.json` is generally the first file in the archive, so this should + /// be a just-opened archive with no entries already read. + pub fn read_from_tar(a: &mut tar::Archive) -> Result { + for entry in a.entries()? { + let mut entry = entry?; + if entry.path()? == std::path::Path::new("oxide.json") { + return Ok(serde_json::from_reader(&mut entry)?); + } + } + Err(Error::new(ErrorKind::InvalidData, "oxide.json is not present")) + } + + pub fn archive_type(&self) -> &ArchiveType { + &self.t + } + + pub fn is_layer(&self) -> bool { + matches!(&self.t, ArchiveType::Layer(_)) + } + + pub fn layer_info(&self) -> Result<&LayerInfo> { + match &self.t { + ArchiveType::Layer(info) => Ok(info), + _ => Err(Error::new( + ErrorKind::InvalidData, + "archive is not the \"layer\" type", + )), + } + } + + pub fn is_baseline(&self) -> bool { + matches!(&self.t, ArchiveType::Baseline) + } + + pub fn is_os(&self) -> bool { + matches!(&self.t, ArchiveType::Os) + } + + pub fn is_rot(&self) -> bool { + matches!(&self.t, ArchiveType::Rot) + } + + pub fn is_control_plane(&self) -> bool { + matches!(&self.t, ArchiveType::ControlPlane) + } +} diff --git a/nexus/src/app/background/tasks/tuf_artifact_replication.rs b/nexus/src/app/background/tasks/tuf_artifact_replication.rs index b540c7db90..3ae62b8d40 100644 --- a/nexus/src/app/background/tasks/tuf_artifact_replication.rs +++ b/nexus/src/app/background/tasks/tuf_artifact_replication.rs @@ -126,7 +126,7 @@ enum ArtifactHandle { } impl ArtifactHandle { - async fn file(&self) -> anyhow::Result { + async fn file(&self) -> std::io::Result { match self { ArtifactHandle::Extracted(handle) => handle.file().await, #[cfg(test)] diff --git a/nexus/tests/integration_tests/updates.rs b/nexus/tests/integration_tests/updates.rs index 35252c8551..37821a1b88 100644 --- a/nexus/tests/integration_tests/updates.rs +++ b/nexus/tests/integration_tests/updates.rs @@ -133,10 +133,20 @@ async fn test_repo_upload() -> Result<()> { .len(); // The repository description should have `Zone` artifacts instead of the // composite `ControlPlane` artifact. - assert!(initial_description - .artifacts - .iter() - .any(|artifact| artifact.id.kind == KnownArtifactKind::Zone.into())); + assert_eq!( + initial_description + .artifacts + .iter() + .filter_map(|artifact| { + if artifact.id.kind == KnownArtifactKind::Zone.into() { + Some(&artifact.id.name) + } else { + None + } + }) + .collect::>(), + ["zone1", "zone2"] + ); assert!(!initial_description .artifacts .iter() diff --git a/sled-agent/Cargo.toml b/sled-agent/Cargo.toml index 12efddda07..fa437cb6ba 100644 --- a/sled-agent/Cargo.toml +++ b/sled-agent/Cargo.toml @@ -108,6 +108,7 @@ omicron-workspace-hack.workspace = true slog-error-chain.workspace = true walkdir.workspace = true zip.workspace = true +omicron-brand-metadata.workspace = true [target.'cfg(target_os = "illumos")'.dependencies] opte-ioctl.workspace = true diff --git a/sled-agent/src/updates.rs b/sled-agent/src/updates.rs index 3633ec0dcc..3cdcead039 100644 --- a/sled-agent/src/updates.rs +++ b/sled-agent/src/updates.rs @@ -6,42 +6,28 @@ use bootstrap_agent_api::Component; use camino::{Utf8Path, Utf8PathBuf}; -use omicron_common::api::internal::nexus::UpdateArtifactId; +use omicron_brand_metadata::Metadata; +use omicron_common::api::external::SemverVersion; use serde::{Deserialize, Serialize}; -use std::io::Read; #[derive(thiserror::Error, Debug)] pub enum Error { - #[error("I/O Error: {message}: {err}")] + #[error("I/O Error: while accessing {path}: {err}")] Io { - message: String, + path: Utf8PathBuf, #[source] err: std::io::Error, }, - #[error("Utf-8 error converting path: {0}")] - FromPathBuf(#[from] camino::FromPathBufError), - - #[error( - "sled-agent only supports applying zones, found artifact ID {}/{} with kind {}", - .0.name, .0.version, .0.kind - )] - UnsupportedKind(UpdateArtifactId), - - #[error("Version not found in artifact {}", 0)] - VersionNotFound(Utf8PathBuf), - - #[error("Cannot parse json: {0}")] - Json(#[from] serde_json::Error), - - #[error("Malformed version in artifact {path}: {why}")] - VersionMalformed { path: Utf8PathBuf, why: String }, + #[error("failed to read zone version from {path}: {err}")] + ZoneVersion { + path: Utf8PathBuf, + #[source] + err: std::io::Error, + }, #[error("Cannot parse semver in {path}: {err}")] Semver { path: Utf8PathBuf, err: semver::Error }, - - #[error("Failed request to Nexus: {0}")] - Response(nexus_client::Error), } fn default_zone_artifact_path() -> Utf8PathBuf { @@ -61,16 +47,8 @@ impl Default for ConfigUpdates { } } -// Helper functions for returning errors -fn version_malformed_err(path: &Utf8Path, key: &str) -> Error { - Error::VersionMalformed { - path: path.to_path_buf(), - why: format!("Missing '{key}'"), - } -} - fn io_err(path: &Utf8Path, err: std::io::Error) -> Error { - Error::Io { message: format!("Cannot access {path}"), err } + Error::Io { path: path.into(), err } } pub struct UpdateManager { @@ -87,46 +65,19 @@ impl UpdateManager { &self, path: &Utf8Path, ) -> Result { - // Decode the zone image let file = std::fs::File::open(path).map_err(|err| io_err(path, err))?; let gzr = flate2::read::GzDecoder::new(file); let mut component_reader = tar::Archive::new(gzr); - let entries = - component_reader.entries().map_err(|err| io_err(path, err))?; - - // Look for the JSON file which contains the package information - for entry in entries { - let mut entry = entry.map_err(|err| io_err(path, err))?; - let entry_path = entry.path().map_err(|err| io_err(path, err))?; - if entry_path == Utf8Path::new("oxide.json") { - let mut contents = String::new(); - entry - .read_to_string(&mut contents) - .map_err(|err| io_err(path, err))?; - let json: serde_json::Value = - serde_json::from_str(contents.as_str())?; - - // Parse keys from the JSON file - let serde_json::Value::String(pkg) = &json["pkg"] else { - return Err(version_malformed_err(path, "pkg")); - }; - let serde_json::Value::String(version) = &json["version"] - else { - return Err(version_malformed_err(path, "version")); - }; - - // Extract the name and semver version - let name = pkg.to_string(); - let version = omicron_common::api::external::SemverVersion( - semver::Version::parse(version).map_err(|err| { - Error::Semver { path: path.to_path_buf(), err } - })?, - ); - return Ok(crate::updates::Component { name, version }); - } - } - Err(Error::VersionNotFound(path.to_path_buf())) + let metadata = Metadata::read_from_tar(&mut component_reader) + .map_err(|err| Error::ZoneVersion { path: path.into(), err })?; + let info = metadata + .layer_info() + .map_err(|err| Error::ZoneVersion { path: path.into(), err })?; + Ok(Component { + name: info.pkg.clone(), + version: SemverVersion(info.version.clone()), + }) } pub async fn components_get(&self) -> Result, Error> { diff --git a/tufaceous-lib/Cargo.toml b/tufaceous-lib/Cargo.toml index f5b7bab393..c3814b1215 100644 --- a/tufaceous-lib/Cargo.toml +++ b/tufaceous-lib/Cargo.toml @@ -25,10 +25,12 @@ futures.workspace = true hex.workspace = true hubtools.workspace = true itertools.workspace = true +omicron-brand-metadata.workspace = true omicron-common.workspace = true omicron-workspace-hack.workspace = true parse-size.workspace = true rand.workspace = true +semver.workspace = true serde.workspace = true serde_json.workspace = true serde_path_to_error.workspace = true diff --git a/tufaceous-lib/src/artifact.rs b/tufaceous-lib/src/artifact.rs index 418d6e18b6..486744e3ce 100644 --- a/tufaceous-lib/src/artifact.rs +++ b/tufaceous-lib/src/artifact.rs @@ -12,10 +12,9 @@ use buf_list::BufList; use bytes::Bytes; use camino::Utf8PathBuf; use fs_err::File; +use omicron_brand_metadata::Metadata; use omicron_common::{api::external::SemverVersion, update::ArtifactKind}; -use crate::oxide_metadata; - mod composite; pub use composite::CompositeControlPlaneArchiveBuilder; @@ -162,7 +161,7 @@ impl HostPhaseImages { .context("error reading path from archive")?; if path == Path::new(OXIDE_JSON_FILE_NAME) { let json_bytes = read_entry(entry, OXIDE_JSON_FILE_NAME)?; - let metadata: oxide_metadata::Metadata = + let metadata: Metadata = serde_json::from_slice(&json_bytes).with_context(|| { format!( "error deserializing JSON from {OXIDE_JSON_FILE_NAME}" @@ -282,7 +281,7 @@ impl RotArchives { .context("error reading path from archive")?; if path == Path::new(OXIDE_JSON_FILE_NAME) { let json_bytes = read_entry(entry, OXIDE_JSON_FILE_NAME)?; - let metadata: oxide_metadata::Metadata = + let metadata: Metadata = serde_json::from_slice(&json_bytes).with_context(|| { format!( "error deserializing JSON from {OXIDE_JSON_FILE_NAME}" @@ -379,7 +378,7 @@ impl ControlPlaneZoneImages { .context("error reading path from archive")?; if path == Path::new(OXIDE_JSON_FILE_NAME) { let json_bytes = read_entry(entry, OXIDE_JSON_FILE_NAME)?; - let metadata: oxide_metadata::Metadata = + let metadata: Metadata = serde_json::from_slice(&json_bytes).with_context(|| { format!( "error deserializing JSON from {OXIDE_JSON_FILE_NAME}" diff --git a/tufaceous-lib/src/artifact/composite.rs b/tufaceous-lib/src/artifact/composite.rs index 6bcc183dc8..50574d704f 100644 --- a/tufaceous-lib/src/artifact/composite.rs +++ b/tufaceous-lib/src/artifact/composite.rs @@ -7,8 +7,6 @@ use super::HOST_PHASE_1_FILE_NAME; use super::HOST_PHASE_2_FILE_NAME; use super::ROT_ARCHIVE_A_FILE_NAME; use super::ROT_ARCHIVE_B_FILE_NAME; -use crate::oxide_metadata; -use crate::oxide_metadata::Metadata; use anyhow::anyhow; use anyhow::bail; use anyhow::Context; @@ -16,6 +14,7 @@ use anyhow::Result; use camino::Utf8Path; use flate2::write::GzEncoder; use flate2::Compression; +use omicron_brand_metadata::{ArchiveType, Metadata}; use sha2::Digest; use sha2::Sha256; use std::collections::HashMap; @@ -41,11 +40,7 @@ pub struct CompositeControlPlaneArchiveBuilder { impl CompositeControlPlaneArchiveBuilder { pub fn new(writer: W, mtime_source: MtimeSource) -> Result { - let metadata = oxide_metadata::MetadataBuilder::new( - oxide_metadata::ArchiveType::ControlPlane, - ) - .build() - .context("error building oxide metadata")?; + let metadata = Metadata::new(ArchiveType::ControlPlane); let inner = CompositeTarballBuilder::new(writer, metadata, mtime_source)?; Ok(Self { inner, hashes: HashMap::new() }) @@ -84,11 +79,7 @@ pub struct CompositeRotArchiveBuilder { impl CompositeRotArchiveBuilder { pub fn new(writer: W, mtime_source: MtimeSource) -> Result { - let metadata = oxide_metadata::MetadataBuilder::new( - oxide_metadata::ArchiveType::Rot, - ) - .build() - .context("error building oxide metadata")?; + let metadata = Metadata::new(ArchiveType::Rot); let inner = CompositeTarballBuilder::new(writer, metadata, mtime_source)?; Ok(Self { inner }) @@ -119,11 +110,7 @@ pub struct CompositeHostArchiveBuilder { impl CompositeHostArchiveBuilder { pub fn new(writer: W, mtime_source: MtimeSource) -> Result { - let metadata = oxide_metadata::MetadataBuilder::new( - oxide_metadata::ArchiveType::Os, - ) - .build() - .context("error building oxide metadata")?; + let metadata = Metadata::new(ArchiveType::Os); let inner = CompositeTarballBuilder::new(writer, metadata, mtime_source)?; Ok(Self { inner }) @@ -156,7 +143,7 @@ impl CompositeTarballBuilder { BufWriter::new(writer), Compression::fast(), )); - metadata.append_to_tar(&mut builder, mtime_source)?; + metadata.append_to_tar(&mut builder, mtime_source.into_mtime())?; Ok(Self { builder }) } diff --git a/tufaceous-lib/src/assemble/manifest.rs b/tufaceous-lib/src/assemble/manifest.rs index 9da4e58861..3203cdfc82 100644 --- a/tufaceous-lib/src/assemble/manifest.rs +++ b/tufaceous-lib/src/assemble/manifest.rs @@ -542,15 +542,40 @@ impl DeserializedControlPlaneZoneSource { })?; // For now, always use the current time as the source. (Maybe // change this to use the mtime on disk in the future?) - (name, data, MtimeSource::Now) + (name.to_owned(), data, MtimeSource::Now) } DeserializedControlPlaneZoneSource::Fake { name, size } => { - let data = make_filler_text(*size as usize); - (name.as_str(), data, MtimeSource::Zero) + use flate2::{write::GzEncoder, Compression}; + use omicron_brand_metadata::{ + ArchiveType, LayerInfo, Metadata, + }; + + let mut tar = tar::Builder::new(GzEncoder::new( + Vec::new(), + Compression::fast(), + )); + + let metadata = Metadata::new(ArchiveType::Layer(LayerInfo { + pkg: name.clone(), + version: semver::Version::new(0, 0, 0), + })); + metadata.append_to_tar(&mut tar, 0)?; + + let mut h = tar::Header::new_ustar(); + h.set_entry_type(tar::EntryType::Regular); + h.set_path("fake")?; + h.set_mode(0o444); + h.set_size(*size); + h.set_mtime(0); + h.set_cksum(); + tar.append(&h, make_filler_text(*size as usize).as_slice())?; + + let data = tar.into_inner()?.finish()?; + (format!("{name}.tar.gz"), data, MtimeSource::Zero) } }; let entry = CompositeEntry { data: &data, mtime_source }; - f(name, entry) + f(&name, entry) } fn apply_size_delta(&mut self, size_delta: i64) -> Result<()> { diff --git a/tufaceous-lib/src/lib.rs b/tufaceous-lib/src/lib.rs index f1b0e39285..bf6fd5e03a 100644 --- a/tufaceous-lib/src/lib.rs +++ b/tufaceous-lib/src/lib.rs @@ -6,7 +6,6 @@ mod archive; mod artifact; pub mod assemble; mod key; -pub mod oxide_metadata; mod repository; mod root; mod target; diff --git a/tufaceous-lib/src/oxide_metadata.rs b/tufaceous-lib/src/oxide_metadata.rs deleted file mode 100644 index 43f0c67df7..0000000000 --- a/tufaceous-lib/src/oxide_metadata.rs +++ /dev/null @@ -1,133 +0,0 @@ -// 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/. - -//! This file is a copy of -//! . -//! Once that is open sourced, we should switch to using that. - -/* - * Copyright 2023 Oxide Computer Company - */ - -use std::collections::HashMap; - -use anyhow::{bail, Result}; -use serde::{Deserialize, Serialize}; - -use crate::MtimeSource; - -#[derive(Clone, Copy, Debug, Deserialize, Serialize)] -#[serde(rename_all = "snake_case")] -pub enum ArchiveType { - Baseline, - Layer, - Os, - Rot, - ControlPlane, -} - -#[derive(Clone, Debug, Deserialize, Serialize)] -pub struct Metadata { - v: String, - t: ArchiveType, - #[serde(default, skip_serializing_if = "HashMap::is_empty")] - i: HashMap, -} - -pub fn parse(s: &str) -> Result { - let m: Metadata = serde_json::from_str(s)?; - if m.v != "1" { - bail!("unexpected metadata version {}", m.v); - } - Ok(m) -} - -impl Metadata { - pub fn append_to_tar( - &self, - a: &mut tar::Builder, - mtime_source: MtimeSource, - ) -> Result<()> { - let mut b = serde_json::to_vec(self)?; - b.push(b'\n'); - - // XXX This was changed from upstream to add oxide.json with optionally - // a zero timestamp, to ensure stability of fake manifests. - let mtime = mtime_source.into_mtime(); - - let mut h = tar::Header::new_ustar(); - h.set_entry_type(tar::EntryType::Regular); - h.set_username("root")?; - h.set_uid(0); - h.set_groupname("root")?; - h.set_gid(0); - h.set_path("oxide.json")?; - h.set_mode(0o444); - h.set_size(b.len().try_into().unwrap()); - h.set_mtime(mtime); - h.set_cksum(); - - a.append(&h, b.as_slice())?; - Ok(()) - } - - pub fn is_layer(&self) -> bool { - matches!(&self.t, ArchiveType::Layer) - } - - pub fn is_baseline(&self) -> bool { - matches!(&self.t, ArchiveType::Baseline) - } - - pub fn is_os(&self) -> bool { - matches!(&self.t, ArchiveType::Os) - } - - pub fn is_rot(&self) -> bool { - matches!(&self.t, ArchiveType::Rot) - } - - pub fn is_control_plane(&self) -> bool { - matches!(&self.t, ArchiveType::ControlPlane) - } - - pub fn archive_type(&self) -> ArchiveType { - self.t - } - - pub fn info(&self) -> &HashMap { - &self.i - } -} - -pub struct MetadataBuilder { - archive_type: ArchiveType, - info: HashMap, -} - -impl MetadataBuilder { - pub fn new(archive_type: ArchiveType) -> MetadataBuilder { - MetadataBuilder { archive_type, info: Default::default() } - } - - pub fn info( - &mut self, - name: &str, - value: &str, - ) -> Result<&mut MetadataBuilder> { - if name.len() < 3 { - bail!("info property names must be at least three characters"); - } - self.info.insert(name.to_string(), value.to_string()); - Ok(self) - } - - pub fn build(&mut self) -> Result { - Ok(Metadata { - v: "1".into(), - t: self.archive_type, - i: self.info.clone(), - }) - } -} diff --git a/tufaceous/manifests/fake.toml b/tufaceous/manifests/fake.toml index f425932bc2..74b1b57147 100644 --- a/tufaceous/manifests/fake.toml +++ b/tufaceous/manifests/fake.toml @@ -39,8 +39,8 @@ version = "1.0.0" [artifact.control_plane.source] kind = "composite-control-plane" zones = [ - { kind = "fake", name = "zone1.tar.gz", size = "512KiB" }, - { kind = "fake", name = "zone2.tar.gz", size = "1MiB" }, + { kind = "fake", name = "zone1", size = "1MiB" }, + { kind = "fake", name = "zone2", size = "1MiB" }, ] [[artifact.psc_sp]] diff --git a/update-common/Cargo.toml b/update-common/Cargo.toml index 96675d47d7..789f646682 100644 --- a/update-common/Cargo.toml +++ b/update-common/Cargo.toml @@ -28,6 +28,10 @@ tokio-util.workspace = true tough.workspace = true tufaceous-lib.workspace = true omicron-workspace-hack.workspace = true +omicron-brand-metadata.workspace = true +tar.workspace = true +flate2.workspace = true +fs-err = { workspace = true, features = ["tokio"] } [dev-dependencies] clap.workspace = true diff --git a/update-common/src/artifacts/extracted_artifacts.rs b/update-common/src/artifacts/extracted_artifacts.rs index 309b188a9d..dd5b1edc8b 100644 --- a/update-common/src/artifacts/extracted_artifacts.rs +++ b/update-common/src/artifacts/extracted_artifacts.rs @@ -73,12 +73,9 @@ impl ExtractedArtifactDataHandle { /// /// This can fail due to I/O errors outside our control (e.g., something /// removed the contents of our temporary directory). - pub async fn file(&self) -> anyhow::Result { + pub async fn file(&self) -> std::io::Result { let path = path_for_artifact(&self.tempdir, &self.hash_id); - - tokio::fs::File::open(&path) - .await - .with_context(|| format!("failed to open {path}")) + fs_err::tokio::File::open(&path).await.map(|file| file.into_parts().0) } /// Async stream to read the contents of this artifact on demand. diff --git a/update-common/src/artifacts/update_plan.rs b/update-common/src/artifacts/update_plan.rs index d1ec4652ad..caf319f61e 100644 --- a/update-common/src/artifacts/update_plan.rs +++ b/update-common/src/artifacts/update_plan.rs @@ -36,6 +36,7 @@ use std::collections::BTreeMap; use std::collections::HashMap; use std::io; use tokio::io::AsyncReadExt; +use tokio::runtime::Handle; use tufaceous_lib::ControlPlaneZoneImages; use tufaceous_lib::HostPhaseImages; use tufaceous_lib::RotArchives; @@ -745,64 +746,7 @@ impl<'a> UpdatePlanBuilder<'a> { } ControlPlaneZonesMode::Split => { // Extract each zone image into its own artifact. - - // Since stream isn't guaranteed to be 'static, we have to - // use block_in_place here, not spawn_blocking. This does mean - // that the current task is taken over, and that this function - // can only be used from a multithreaded Tokio runtime. (See - // `extract_nested_artifact_pair` for more commentary on - // alternatives.) - tokio::task::block_in_place(|| { - let stream = std::pin::pin!(stream); - let reader = tokio_util::io::StreamReader::new( - stream.map_err(|error| { - // StreamReader requires a conversion from tough's errors to - // std::io::Error. - std::io::Error::new(io::ErrorKind::Other, error) - }), - ); - - // ControlPlaneZoneImages::extract_into takes a synchronous - // reader, so we need to use this bridge. The bridge can - // only be used from a blocking context. - let reader = tokio_util::io::SyncIoBridge::new(reader); - - ControlPlaneZoneImages::extract_into( - reader, - |name, reader| { - let mut out = - self.extracted_artifacts.new_tempfile()?; - io::copy(reader, &mut out)?; - let data = - self.extracted_artifacts.store_tempfile( - KnownArtifactKind::Zone.into(), - out, - )?; - let artifact_id = ArtifactId { - name, - version: artifact_id.version.clone(), - kind: KnownArtifactKind::Zone.into(), - }; - self.record_extracted_artifact( - artifact_id, - data, - KnownArtifactKind::Zone.into(), - self.log, - )?; - Ok(()) - }, - ) - }) - .map_err(|error| { - // Fish the original RepositoryError out of this - // anyhow::Error if it is one. - error.downcast().unwrap_or_else(|error| { - RepositoryError::TarballExtract { - kind: KnownArtifactKind::ControlPlane, - error, - } - }) - })?; + self.extract_control_plane_zones(stream)?; } } @@ -942,6 +886,75 @@ impl<'a> UpdatePlanBuilder<'a> { Ok((image1, image2)) } + /// Helper function for extracting and recording zones out of `stream`, a + /// composite control plane artifact. + /// + /// This code can only be used with multithreaded Tokio executors; see + /// `extract_nested_artifact_pair` for context on `block_in_place`. + fn extract_control_plane_zones( + &mut self, + stream: impl Stream> + Send, + ) -> Result<(), RepositoryError> { + tokio::task::block_in_place(|| { + let stream = std::pin::pin!(stream); + let reader = + tokio_util::io::StreamReader::new(stream.map_err(|error| { + // StreamReader requires a conversion from tough's errors to + // std::io::Error. + std::io::Error::new(io::ErrorKind::Other, error) + })); + let reader = tokio_util::io::SyncIoBridge::new(reader); + self.extract_control_plane_zones_impl(reader) + }) + } + + fn extract_control_plane_zones_impl( + &mut self, + reader: impl io::Read, + ) -> Result<(), RepositoryError> { + ControlPlaneZoneImages::extract_into(reader, |_, reader| { + let mut out = self.extracted_artifacts.new_tempfile()?; + io::copy(reader, &mut out)?; + let data = self + .extracted_artifacts + .store_tempfile(KnownArtifactKind::Zone.into(), out)?; + + // Read the zone name and version from the `oxide.json` at the root + // of the zone. + let data_clone = data.clone(); + let file = Handle::current().block_on(async move { + std::io::Result::Ok(data_clone.file().await?.into_std().await) + })?; + let mut tar = tar::Archive::new(flate2::read::GzDecoder::new(file)); + let metadata = + omicron_brand_metadata::Metadata::read_from_tar(&mut tar)?; + let info = metadata.layer_info()?; + + let artifact_id = ArtifactId { + name: info.pkg.clone(), + version: SemverVersion(info.version.clone()), + kind: KnownArtifactKind::Zone.into(), + }; + self.record_extracted_artifact( + artifact_id, + data, + KnownArtifactKind::Zone.into(), + self.log, + )?; + Ok(()) + }) + .map_err(|error| { + // Fish the original RepositoryError out of this + // anyhow::Error if it is one. + error.downcast().unwrap_or_else(|error| { + RepositoryError::TarballExtract { + kind: KnownArtifactKind::ControlPlane, + error, + } + }) + }) + } + // Record an artifact in `by_id` and `by_hash`, or fail if either already has an // entry for this id/hash. fn record_extracted_artifact( From dfd9fdc47eae42bf5a04933d43a609402c97adca Mon Sep 17 00:00:00 2001 From: iliana etaoin Date: Fri, 31 Jan 2025 04:51:42 +0000 Subject: [PATCH 5/9] markdown!!!!! --- brand-metadata/src/lib.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/brand-metadata/src/lib.rs b/brand-metadata/src/lib.rs index 7dd13209cb..e43b51e03c 100644 --- a/brand-metadata/src/lib.rs +++ b/brand-metadata/src/lib.rs @@ -5,7 +5,7 @@ //! Handling of `oxide.json` metadata files in tarballs. //! //! `oxide.json` is originally defined by the omicron1(7) zone brand, which -//! lives at https://github.com/oxidecomputer/helios-omicron-brand. tufaceous +//! lives at . tufaceous //! extended this format with additional archive types for identifying other //! types of tarballs; this crate covers those extensions so they can be used //! across the Omicron codebase. From 5fe84c3e7382ff7c70bd0e622f59f948b637e49a Mon Sep 17 00:00:00 2001 From: iliana etaoin Date: Fri, 31 Jan 2025 06:19:40 +0000 Subject: [PATCH 6/9] what if i wrote code that worked --- brand-metadata/src/lib.rs | 27 +++++++++++++++++++++++++++ 1 file changed, 27 insertions(+) diff --git a/brand-metadata/src/lib.rs b/brand-metadata/src/lib.rs index e43b51e03c..f25b120ac6 100644 --- a/brand-metadata/src/lib.rs +++ b/brand-metadata/src/lib.rs @@ -28,6 +28,7 @@ pub struct Metadata { } #[derive(Clone, Debug, Deserialize, Serialize)] +#[serde(rename_all = "snake_case", tag = "t")] pub enum ArchiveType { // Originally defined in helios-build-utils (part of helios-omicron-brand): Baseline, @@ -122,3 +123,29 @@ impl Metadata { matches!(&self.t, ArchiveType::ControlPlane) } } + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_deserialize() { + let metadata: Metadata = serde_json::from_str( + r#"{"v":"1","t":"layer","pkg":"nexus","version":"12.0.0-0.ci+git3a2ed5e97b3"}"#, + ) + .unwrap(); + assert!(metadata.is_layer()); + let info = metadata.layer_info().unwrap(); + assert_eq!(info.pkg, "nexus"); + assert_eq!(info.version, "12.0.0-0.ci+git3a2ed5e97b3".parse().unwrap()); + + let metadata: Metadata = serde_json::from_str( + r#"{"v":"1","t":"os","i":{"checksum":"42eda100ee0e3bf44b9d0bb6a836046fa3133c378cd9d3a4ba338c3ba9e56eb7","name":"ci 3a2ed5e/9d37813 2024-12-20 08:54"}}"#, + ).unwrap(); + assert!(metadata.is_os()); + + let metadata: Metadata = + serde_json::from_str(r#"{"v":"1","t":"control_plane"}"#).unwrap(); + assert!(metadata.is_control_plane()); + } +} From 52c4c8476e0e159b95a1b01eca794569659022ea Mon Sep 17 00:00:00 2001 From: iliana etaoin Date: Fri, 31 Jan 2025 19:21:44 +0000 Subject: [PATCH 7/9] fix the test i broke --- update-common/src/artifacts/update_plan.rs | 23 ++++++++++++++++++---- 1 file changed, 19 insertions(+), 4 deletions(-) diff --git a/update-common/src/artifacts/update_plan.rs b/update-common/src/artifacts/update_plan.rs index caf319f61e..297d66f986 100644 --- a/update-common/src/artifacts/update_plan.rs +++ b/update-common/src/artifacts/update_plan.rs @@ -1277,7 +1277,9 @@ mod tests { use super::*; use bytes::Bytes; + use flate2::{write::GzEncoder, Compression}; use futures::StreamExt; + use omicron_brand_metadata::{ArchiveType, LayerInfo, Metadata}; use omicron_test_utils::dev::test_setup_log; use rand::{distributions::Standard, thread_rng, Rng}; use sha2::{Digest, Sha256}; @@ -2489,17 +2491,30 @@ mod tests { #[tokio::test(flavor = "multi_thread", worker_threads = 2)] async fn test_split_control_plane() { const VERSION_0: SemverVersion = SemverVersion::new(0, 0, 0); - const ZONES: &[(&str, &[u8])] = - &[("first", b"the first zone"), ("second", b"the second zone")]; let logctx = test_setup_log("test_split_control_plane"); + let mut zones = Vec::new(); + for name in ["first", "second"] { + let mut tar = tar::Builder::new(GzEncoder::new( + Vec::new(), + Compression::fast(), + )); + let metadata = Metadata::new(ArchiveType::Layer(LayerInfo { + pkg: name.to_owned(), + version: VERSION_0.0.clone(), + })); + metadata.append_to_tar(&mut tar, 0).unwrap(); + let data = tar.into_inner().unwrap().finish().unwrap(); + zones.push((name, data)); + } + let mut cp_builder = CompositeControlPlaneArchiveBuilder::new( Vec::new(), MtimeSource::Now, ) .unwrap(); - for (name, data) in ZONES { + for (name, data) in &zones { cp_builder .append_zone( name, @@ -2531,7 +2546,7 @@ mod tests { // of the zone contents. for (id, vec) in &plan_builder.by_id { let content = - ZONES.iter().find(|(name, _)| *name == id.name).unwrap().1; + &zones.iter().find(|(name, _)| *name == id.name).unwrap().1; let expected_hash = ArtifactHash(Sha256::digest(content).into()); assert_eq!(id.version, VERSION_0); assert_eq!(id.kind, KnownArtifactKind::Zone.into()); From 142c952ecc34dba23d8b519b1fca0a7cc5228ff3 Mon Sep 17 00:00:00 2001 From: iliana etaoin Date: Fri, 31 Jan 2025 19:30:25 +0000 Subject: [PATCH 8/9] nit --- wicketd/tests/integration_tests/updates.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/wicketd/tests/integration_tests/updates.rs b/wicketd/tests/integration_tests/updates.rs index 547601da8a..d333a1104a 100644 --- a/wicketd/tests/integration_tests/updates.rs +++ b/wicketd/tests/integration_tests/updates.rs @@ -69,7 +69,8 @@ async fn test_updates() { .expect("get_artifacts_and_event_reports succeeded") .into_inner(); - // We should have an artifact for every known artifact kind... + // We should have an artifact for every known artifact kind (except + // `Zone`)... let expected_kinds: BTreeSet<_> = KnownArtifactKind::iter() .filter(|k| !matches!(k, KnownArtifactKind::Zone)) .map(ArtifactKind::from) From 0cdecbda3a3c75aba66b0887b5770bf40d32b13c Mon Sep 17 00:00:00 2001 From: iliana etaoin Date: Sat, 1 Feb 2025 18:55:53 +0000 Subject: [PATCH 9/9] nit --- sled-agent/src/bootstrap/http_entrypoints.rs | 10 ++++++---- sled-agent/src/updates.rs | 6 +++--- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/sled-agent/src/bootstrap/http_entrypoints.rs b/sled-agent/src/bootstrap/http_entrypoints.rs index a29eaed376..4e87e94e91 100644 --- a/sled-agent/src/bootstrap/http_entrypoints.rs +++ b/sled-agent/src/bootstrap/http_entrypoints.rs @@ -28,6 +28,7 @@ use sled_agent_types::rack_ops::RackOperationStatus; use sled_hardware_types::Baseboard; use sled_storage::manager::StorageHandle; use slog::Logger; +use slog_error_chain::InlineErrorChain; use sprockets_tls::keys::SprocketsConfig; use std::net::Ipv6Addr; use tokio::sync::mpsc::error::TrySendError; @@ -85,10 +86,11 @@ impl BootstrapAgentApi for BootstrapAgentImpl { ) -> Result>, HttpError> { let ctx = rqctx.context(); let updates = UpdateManager::new(ctx.updates.clone()); - let components = updates - .components_get() - .await - .map_err(|err| HttpError::for_internal_error(err.to_string()))?; + let components = updates.components_get().await.map_err(|err| { + HttpError::for_internal_error( + InlineErrorChain::new(&err).to_string(), + ) + })?; Ok(HttpResponseOk(components)) } diff --git a/sled-agent/src/updates.rs b/sled-agent/src/updates.rs index 3cdcead039..3fd810c253 100644 --- a/sled-agent/src/updates.rs +++ b/sled-agent/src/updates.rs @@ -12,21 +12,21 @@ use serde::{Deserialize, Serialize}; #[derive(thiserror::Error, Debug)] pub enum Error { - #[error("I/O Error: while accessing {path}: {err}")] + #[error("I/O Error: while accessing {path}")] Io { path: Utf8PathBuf, #[source] err: std::io::Error, }, - #[error("failed to read zone version from {path}: {err}")] + #[error("failed to read zone version from {path}")] ZoneVersion { path: Utf8PathBuf, #[source] err: std::io::Error, }, - #[error("Cannot parse semver in {path}: {err}")] + #[error("Cannot parse semver in {path}")] Semver { path: Utf8PathBuf, err: semver::Error }, }