implement refactor

This commit is contained in:
Stuart Axelbrooke 2025-11-25 13:31:31 +08:00
parent a7ac85917c
commit 375a15d9e9
7 changed files with 1177 additions and 301 deletions

View file

@ -49,6 +49,10 @@ crate.spec(
package = "uuid", package = "uuid",
version = "1.0", version = "1.0",
) )
crate.spec(
package = "sha2",
version = "0.10",
)
crate.spec( crate.spec(
features = ["bundled"], features = ["bundled"],
package = "rusqlite", package = "rusqlite",

File diff suppressed because one or more lines are too long

View file

@ -39,6 +39,7 @@ rust_library(
"@crates//:tower-http", "@crates//:tower-http",
"@crates//:tracing", "@crates//:tracing",
"@crates//:uuid", "@crates//:uuid",
"@crates//:sha2",
], ],
) )

View file

@ -3,12 +3,14 @@ use crate::data_deps::{WantTimestamps, missing_deps_to_want_events};
use crate::event_source::Source as EventSourceVariant; use crate::event_source::Source as EventSourceVariant;
use crate::job_run_state::{JobRun, JobRunWithState, QueuedState as JobQueuedState}; use crate::job_run_state::{JobRun, JobRunWithState, QueuedState as JobQueuedState};
use crate::partition_state::{ use crate::partition_state::{
BuildingPartitionRef, FailedPartitionRef, LivePartitionRef, MissingPartitionRef, MissingState, BuildingPartitionRef, BuildingState, FailedPartitionRef, LivePartitionRef, Partition,
Partition, PartitionWithState, TaintedPartitionRef, PartitionWithState, TaintedPartitionRef, UpForRetryPartitionRef, UpstreamBuildingPartitionRef,
UpstreamFailedPartitionRef,
}; };
use crate::util::current_timestamp; use crate::util::current_timestamp;
use crate::want_state::{ use crate::want_state::{
FailedWantId, IdleState as WantIdleState, SuccessfulWantId, Want, WantWithState, FailedWantId, IdleState as WantIdleState, NewState as WantNewState, SuccessfulWantId, Want,
WantWithState,
}; };
use crate::{ use crate::{
JobRunBufferEventV1, JobRunCancelEventV1, JobRunDetail, JobRunFailureEventV1, JobRunBufferEventV1, JobRunCancelEventV1, JobRunDetail, JobRunFailureEventV1,
@ -21,6 +23,7 @@ use crate::{
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
use std::collections::BTreeMap; use std::collections::BTreeMap;
use tracing; use tracing;
use uuid::Uuid;
/** /**
Design Notes Design Notes
@ -49,10 +52,18 @@ critical that these state machines, their states, and their transitions are type
/// state transitions (e.g. job run success resulting in partition going from Building to Live) /// state transitions (e.g. job run success resulting in partition going from Building to Live)
#[derive(Debug, Clone, Default)] #[derive(Debug, Clone, Default)]
pub struct BuildState { pub struct BuildState {
wants: BTreeMap<String, Want>, // Type-safe want storage // Core entity storage
wants: BTreeMap<String, Want>,
taints: BTreeMap<String, TaintDetail>, taints: BTreeMap<String, TaintDetail>,
partitions: BTreeMap<String, Partition>, // Type-safe partition storage job_runs: BTreeMap<String, JobRun>,
job_runs: BTreeMap<String, JobRun>, // Type-safe job run storage
// UUID-based partition indexing
partitions_by_uuid: BTreeMap<Uuid, Partition>,
canonical_partitions: BTreeMap<String, Uuid>, // partition ref → current UUID
// Inverted indexes
wants_for_partition: BTreeMap<String, Vec<String>>, // partition ref → want_ids
downstream_waiting: BTreeMap<String, Vec<Uuid>>, // upstream ref → partition UUIDs waiting for it
} }
impl BuildState { impl BuildState {
@ -74,23 +85,85 @@ impl BuildState {
self.job_runs.len() self.job_runs.len()
} }
/// Add want_id to partition's want_ids list // ===== New UUID-based partition access methods =====
fn add_want_to_partition(&mut self, pref: &PartitionRef, want_id: &str) {
// Create partition if it doesn't exist /// Get the canonical partition for a ref (the current/active partition instance)
if !self.partitions.contains_key(&pref.r#ref) { pub fn get_canonical_partition(&self, partition_ref: &str) -> Option<&Partition> {
let partition = Partition::new_missing(pref.clone()); self.canonical_partitions
self.partitions.insert(pref.r#ref.clone(), partition); .get(partition_ref)
.and_then(|uuid| self.partitions_by_uuid.get(uuid))
} }
// Add want_id /// Get the canonical partition UUID for a ref
if let Some(partition) = self.partitions.get_mut(&pref.r#ref) { pub fn get_canonical_partition_uuid(&self, partition_ref: &str) -> Option<Uuid> {
let want_ids = partition.want_ids_mut(); self.canonical_partitions.get(partition_ref).copied()
}
/// Get a partition by its UUID
pub fn get_partition_by_uuid(&self, uuid: Uuid) -> Option<&Partition> {
self.partitions_by_uuid.get(&uuid)
}
/// Take the canonical partition for a ref (removes from partitions_by_uuid for state transition)
/// The canonical_partitions mapping is NOT removed - caller must update it if creating a new partition
fn take_canonical_partition(&mut self, partition_ref: &str) -> Option<Partition> {
self.canonical_partitions
.get(partition_ref)
.copied()
.and_then(|uuid| self.partitions_by_uuid.remove(&uuid))
}
/// Get want IDs for a partition ref (from inverted index)
pub fn get_wants_for_partition(&self, partition_ref: &str) -> &[String] {
self.wants_for_partition
.get(partition_ref)
.map(|v| v.as_slice())
.unwrap_or(&[])
}
/// Register a want in the wants_for_partition inverted index
fn register_want_for_partitions(&mut self, want_id: &str, partition_refs: &[PartitionRef]) {
for pref in partition_refs {
let want_ids = self
.wants_for_partition
.entry(pref.r#ref.clone())
.or_insert_with(Vec::new);
if !want_ids.contains(&want_id.to_string()) { if !want_ids.contains(&want_id.to_string()) {
want_ids.push(want_id.to_string()); want_ids.push(want_id.to_string());
} }
} }
} }
/// Create a new partition in Building state and update indexes
fn create_partition_building(&mut self, job_run_id: &str, partition_ref: PartitionRef) -> Uuid {
let partition =
PartitionWithState::<BuildingState>::new(job_run_id.to_string(), partition_ref.clone());
let uuid = partition.uuid;
// Update indexes
self.partitions_by_uuid
.insert(uuid, Partition::Building(partition));
self.canonical_partitions
.insert(partition_ref.r#ref.clone(), uuid);
tracing::info!(
partition = %partition_ref.r#ref,
uuid = %uuid,
job_run_id = %job_run_id,
"Partition: Created in Building state"
);
uuid
}
/// Update a partition in the indexes (after state transition)
fn update_partition(&mut self, partition: Partition) {
let uuid = partition.uuid();
// Update in UUID map
self.partitions_by_uuid.insert(uuid, partition);
}
/// Handle creation of a derivative want (created due to job dep miss) /// Handle creation of a derivative want (created due to job dep miss)
/// ///
/// When a job reports missing dependencies, it returns WantCreateV1 events for those missing partitions. /// When a job reports missing dependencies, it returns WantCreateV1 events for those missing partitions.
@ -156,12 +229,10 @@ impl BuildState {
let mut impacted_want_ids: std::collections::HashSet<String> = let mut impacted_want_ids: std::collections::HashSet<String> =
std::collections::HashSet::new(); std::collections::HashSet::new();
for partition_ref in &impacted_partition_refs { for partition_ref in &impacted_partition_refs {
if let Some(partition) = self.partitions.get(partition_ref) { for want_id in self.get_wants_for_partition(partition_ref) {
for want_id in partition.want_ids() {
impacted_want_ids.insert(want_id.clone()); impacted_want_ids.insert(want_id.clone());
} }
} }
}
// Transition each impacted want to UpstreamBuilding, tracking this derivative want as an upstream // Transition each impacted want to UpstreamBuilding, tracking this derivative want as an upstream
for want_id in impacted_want_ids { for want_id in impacted_want_ids {
@ -208,49 +279,47 @@ impl BuildState {
} }
} }
/// Transition partitions from Missing to Building state /// Create partitions in Building state
/// Used when a job run starts building partitions /// Used when a job run starts building partitions.
/// Note: Partitions no longer have a Missing state - they start directly as Building.
fn transition_partitions_to_building( fn transition_partitions_to_building(
&mut self, &mut self,
partition_refs: &[BuildingPartitionRef], partition_refs: &[BuildingPartitionRef],
job_run_id: &str, job_run_id: &str,
) { ) {
for building_ref in partition_refs { for building_ref in partition_refs {
if let Some(partition) = self.partitions.remove(&building_ref.0.r#ref) { if let Some(partition) = self.get_canonical_partition(&building_ref.0.r#ref).cloned() {
// Partition exists - transition based on current state // Partition already exists - this is an error unless we're retrying from UpForRetry
let transitioned = match partition { match partition {
// Valid: Missing -> Building Partition::UpForRetry(_) => {
Partition::Missing(missing) => { // Valid: UpForRetry -> Building (retry after deps satisfied)
// Old partition stays in partitions_by_uuid as historical record
// Create new Building partition with fresh UUID
let uuid =
self.create_partition_building(job_run_id, building_ref.0.clone());
tracing::info!( tracing::info!(
partition = %building_ref.0.r#ref, partition = %building_ref.0.r#ref,
job_run_id = %job_run_id, job_run_id = %job_run_id,
"Partition: Missing → Building" uuid = %uuid,
"Partition: UpForRetry → Building (retry)"
); );
Partition::Building(missing.start_building(job_run_id.to_string()))
} }
// Invalid state: partition should not already be Building, Live, Failed, or Tainted
_ => { _ => {
panic!( panic!(
"BUG: Invalid state - partition {} cannot start building from state {:?}", "BUG: Invalid state - partition {} cannot start building from state {:?}",
building_ref.0.r#ref, partition building_ref.0.r#ref, partition
) )
} }
}; }
self.partitions
.insert(building_ref.0.r#ref.clone(), transitioned);
} else { } else {
// Partition doesn't exist yet - create in Missing then transition to Building // Partition doesn't exist yet - create directly in Building state
let missing = Partition::new_missing(building_ref.0.clone()); let uuid = self.create_partition_building(job_run_id, building_ref.0.clone());
if let Partition::Missing(m) = missing {
let building = m.start_building(job_run_id.to_string());
tracing::info!( tracing::info!(
partition = %building_ref.0.r#ref, partition = %building_ref.0.r#ref,
job_run_id = %job_run_id, job_run_id = %job_run_id,
"Partition: Missing → Building (created)" uuid = %uuid,
"Partition: (new) → Building"
); );
self.partitions
.insert(building_ref.0.r#ref.clone(), Partition::Building(building));
}
} }
} }
} }
@ -264,7 +333,9 @@ impl BuildState {
timestamp: u64, timestamp: u64,
) { ) {
for pref in partition_refs { for pref in partition_refs {
let partition = self.partitions.remove(&pref.0.r#ref).expect(&format!( let partition = self
.take_canonical_partition(&pref.0.r#ref)
.expect(&format!(
"BUG: Partition {} must exist and be in Building state before completion", "BUG: Partition {} must exist and be in Building state before completion",
pref.0.r#ref pref.0.r#ref
)); ));
@ -277,7 +348,7 @@ impl BuildState {
job_run_id = %job_run_id, job_run_id = %job_run_id,
"Partition: Building → Live" "Partition: Building → Live"
); );
Partition::Live(building.complete(job_run_id.to_string(), timestamp)) Partition::Live(building.complete(timestamp))
} }
// All other states are invalid // All other states are invalid
_ => { _ => {
@ -287,7 +358,7 @@ impl BuildState {
) )
} }
}; };
self.partitions.insert(pref.0.r#ref.clone(), transitioned); self.update_partition(transitioned);
} }
} }
@ -300,7 +371,9 @@ impl BuildState {
timestamp: u64, timestamp: u64,
) { ) {
for pref in partition_refs { for pref in partition_refs {
let partition = self.partitions.remove(&pref.0.r#ref).expect(&format!( let partition = self
.take_canonical_partition(&pref.0.r#ref)
.expect(&format!(
"BUG: Partition {} must exist and be in Building state before failure", "BUG: Partition {} must exist and be in Building state before failure",
pref.0.r#ref pref.0.r#ref
)); ));
@ -313,7 +386,7 @@ impl BuildState {
job_run_id = %job_run_id, job_run_id = %job_run_id,
"Partition: Building → Failed" "Partition: Building → Failed"
); );
Partition::Failed(building.fail(job_run_id.to_string(), timestamp)) Partition::Failed(building.fail(timestamp))
} }
// All other states are invalid // All other states are invalid
_ => { _ => {
@ -323,30 +396,46 @@ impl BuildState {
) )
} }
}; };
self.partitions.insert(pref.0.r#ref.clone(), transitioned); self.update_partition(transitioned);
} }
} }
/// Reset partitions from Building back to Missing state /// Transition partitions from Building to UpstreamBuilding state
/// Used when a job run encounters missing dependencies and cannot proceed /// Used when a job run encounters missing dependencies and cannot proceed.
fn reset_partitions_to_missing(&mut self, partition_refs: &[BuildingPartitionRef]) { /// The partition waits for its upstream deps to be built before becoming UpForRetry.
fn transition_partitions_to_upstream_building(
&mut self,
partition_refs: &[BuildingPartitionRef],
missing_deps: Vec<PartitionRef>,
) {
for building_ref in partition_refs { for building_ref in partition_refs {
let partition = self let partition = self
.partitions .take_canonical_partition(&building_ref.0.r#ref)
.remove(&building_ref.0.r#ref)
.expect(&format!( .expect(&format!(
"BUG: Partition {} must exist and be in Building state during dep_miss", "BUG: Partition {} must exist and be in Building state during dep_miss",
building_ref.0.r#ref building_ref.0.r#ref
)); ));
// Only valid transition: Building -> Missing // Only valid transition: Building -> UpstreamBuilding
let transitioned = match partition { let transitioned = match partition {
Partition::Building(building) => { Partition::Building(building) => {
let partition_uuid = building.uuid;
tracing::info!( tracing::info!(
partition = %building_ref.0.r#ref, partition = %building_ref.0.r#ref,
"Partition: Building → Missing (dep miss)" uuid = %partition_uuid,
missing_deps = ?missing_deps.iter().map(|p| &p.r#ref).collect::<Vec<_>>(),
"Partition: Building → UpstreamBuilding (dep miss)"
); );
Partition::Missing(building.reset_to_missing())
// Update downstream_waiting index: for each missing dep, record that this partition is waiting
for missing_dep in &missing_deps {
self.downstream_waiting
.entry(missing_dep.r#ref.clone())
.or_default()
.push(partition_uuid);
}
Partition::UpstreamBuilding(building.dep_miss(missing_deps.clone()))
} }
// All other states are invalid // All other states are invalid
_ => { _ => {
@ -356,8 +445,137 @@ impl BuildState {
) )
} }
}; };
self.partitions self.update_partition(transitioned);
.insert(building_ref.0.r#ref.clone(), transitioned); }
}
/// Transition partitions from UpstreamBuilding to UpForRetry when their upstream deps become Live.
/// This should be called when partitions become Live to check if any downstream partitions can now retry.
/// Uses the `downstream_waiting` index for O(1) lookup of affected partitions.
fn unblock_downstream_partitions(&mut self, newly_live_partition_refs: &[LivePartitionRef]) {
// Collect UUIDs of partitions that might be unblocked using the inverted index
let mut uuids_to_check: Vec<Uuid> = Vec::new();
for live_ref in newly_live_partition_refs {
if let Some(waiting_uuids) = self.downstream_waiting.get(&live_ref.0.r#ref) {
uuids_to_check.extend(waiting_uuids.iter().cloned());
}
}
// Deduplicate UUIDs (a partition might be waiting for multiple deps that all became live)
uuids_to_check.sort();
uuids_to_check.dedup();
for uuid in uuids_to_check {
// Get partition by UUID - it might have been transitioned already or no longer exist
let Some(partition) = self.partitions_by_uuid.get(&uuid).cloned() else {
continue;
};
let partition_ref = partition.partition_ref().r#ref.clone();
// Only process UpstreamBuilding partitions
if let Partition::UpstreamBuilding(mut upstream_building) = partition {
// Remove satisfied deps from missing_deps
for live_ref in newly_live_partition_refs {
upstream_building
.state
.missing_deps
.retain(|d| d.r#ref != live_ref.0.r#ref);
// Also remove from downstream_waiting index
if let Some(waiting) = self.downstream_waiting.get_mut(&live_ref.0.r#ref) {
waiting.retain(|u| *u != uuid);
}
}
let transitioned = if upstream_building.state.missing_deps.is_empty() {
// All deps satisfied, transition to UpForRetry
tracing::info!(
partition = %partition_ref,
uuid = %uuid,
"Partition: UpstreamBuilding → UpForRetry (all upstreams satisfied)"
);
Partition::UpForRetry(upstream_building.upstreams_satisfied())
} else {
// Still waiting for more deps
tracing::debug!(
partition = %partition_ref,
uuid = %uuid,
remaining_deps = ?upstream_building.state.missing_deps.iter().map(|d| &d.r#ref).collect::<Vec<_>>(),
"Partition remains in UpstreamBuilding (still waiting for deps)"
);
Partition::UpstreamBuilding(upstream_building)
};
self.update_partition(transitioned);
}
}
}
/// Cascade failures to downstream partitions when their upstream dependencies fail.
/// Transitions UpstreamBuilding → UpstreamFailed for partitions waiting on failed upstreams.
/// Uses the `downstream_waiting` index for O(1) lookup of affected partitions.
fn cascade_failures_to_downstream_partitions(
&mut self,
failed_partition_refs: &[FailedPartitionRef],
) {
// Collect UUIDs of partitions that are waiting for the failed partitions
let mut uuids_to_fail: Vec<Uuid> = Vec::new();
for failed_ref in failed_partition_refs {
if let Some(waiting_uuids) = self.downstream_waiting.get(&failed_ref.0.r#ref) {
uuids_to_fail.extend(waiting_uuids.iter().cloned());
}
}
// Deduplicate UUIDs
uuids_to_fail.sort();
uuids_to_fail.dedup();
for uuid in uuids_to_fail {
// Get partition by UUID
let Some(partition) = self.partitions_by_uuid.get(&uuid).cloned() else {
continue;
};
let partition_ref = partition.partition_ref().r#ref.clone();
// Only process UpstreamBuilding partitions
if let Partition::UpstreamBuilding(upstream_building) = partition {
// Collect which upstream refs failed
let failed_upstream_refs: Vec<PartitionRef> = failed_partition_refs
.iter()
.filter(|f| {
upstream_building
.state
.missing_deps
.iter()
.any(|d| d.r#ref == f.0.r#ref)
})
.map(|f| f.0.clone())
.collect();
if !failed_upstream_refs.is_empty() {
tracing::info!(
partition = %partition_ref,
uuid = %uuid,
failed_upstreams = ?failed_upstream_refs.iter().map(|p| &p.r#ref).collect::<Vec<_>>(),
"Partition: UpstreamBuilding → UpstreamFailed (upstream failed)"
);
// Remove from downstream_waiting index for all deps
for dep in &upstream_building.state.missing_deps {
if let Some(waiting) = self.downstream_waiting.get_mut(&dep.r#ref) {
waiting.retain(|u| *u != uuid);
}
}
// Transition to UpstreamFailed
let transitioned = Partition::UpstreamFailed(
upstream_building
.upstream_failed(failed_upstream_refs, current_timestamp()),
);
self.update_partition(transitioned);
}
}
} }
} }
@ -372,11 +590,7 @@ impl BuildState {
let mut newly_successful_wants: Vec<SuccessfulWantId> = Vec::new(); let mut newly_successful_wants: Vec<SuccessfulWantId> = Vec::new();
for pref in newly_live_partitions { for pref in newly_live_partitions {
let want_ids = self let want_ids: Vec<String> = self.get_wants_for_partition(&pref.0.r#ref).to_vec();
.partitions
.get(&pref.0.r#ref)
.map(|p| p.want_ids().clone())
.unwrap_or_default();
for want_id in want_ids { for want_id in want_ids {
let want = self.wants.remove(&want_id).expect(&format!( let want = self.wants.remove(&want_id).expect(&format!(
@ -388,8 +602,7 @@ impl BuildState {
Want::Building(building) => { Want::Building(building) => {
// Check if ALL partitions for this want are now Live // Check if ALL partitions for this want are now Live
let all_partitions_live = building.want.partitions.iter().all(|p| { let all_partitions_live = building.want.partitions.iter().all(|p| {
self.partitions self.get_canonical_partition(&p.r#ref)
.get(&p.r#ref)
.map(|partition| partition.is_live()) .map(|partition| partition.is_live())
.unwrap_or(false) .unwrap_or(false)
}); });
@ -433,11 +646,7 @@ impl BuildState {
let mut newly_failed_wants: Vec<FailedWantId> = Vec::new(); let mut newly_failed_wants: Vec<FailedWantId> = Vec::new();
for pref in failed_partitions { for pref in failed_partitions {
let want_ids = self let want_ids: Vec<String> = self.get_wants_for_partition(&pref.0.r#ref).to_vec();
.partitions
.get(&pref.0.r#ref)
.map(|p| p.want_ids().clone())
.unwrap_or_default();
for want_id in want_ids { for want_id in want_ids {
let want = self.wants.remove(&want_id).expect(&format!( let want = self.wants.remove(&want_id).expect(&format!(
@ -546,8 +755,7 @@ impl BuildState {
// But other jobs might still be building other partitions for this want // But other jobs might still be building other partitions for this want
let any_partition_building = let any_partition_building =
downstream_want.want.partitions.iter().any(|p| { downstream_want.want.partitions.iter().any(|p| {
self.partitions self.get_canonical_partition(&p.r#ref)
.get(&p.r#ref)
.map(|partition| matches!(partition, Partition::Building(_))) .map(|partition| matches!(partition, Partition::Building(_)))
.unwrap_or(false) .unwrap_or(false)
}); });
@ -669,11 +877,11 @@ impl BuildState {
} }
fn handle_want_create(&mut self, event: &WantCreateEventV1) -> Vec<Event> { fn handle_want_create(&mut self, event: &WantCreateEventV1) -> Vec<Event> {
// Use From impl to create want in Idle state // Create want in New state from event
let want_idle: WantWithState<WantIdleState> = event.clone().into(); let want_new: WantWithState<WantNewState> = event.clone().into();
// Log creation with derivative vs user-created distinction // Log creation with derivative vs user-created distinction
if let Some(source) = &event.source { let is_derivative = if let Some(source) = &event.source {
if let Some(EventSourceVariant::JobTriggered(job_triggered)) = &source.source { if let Some(EventSourceVariant::JobTriggered(job_triggered)) = &source.source {
tracing::info!( tracing::info!(
want_id = %event.want_id, want_id = %event.want_id,
@ -681,6 +889,9 @@ impl BuildState {
source_job_run_id = %job_triggered.job_run_id, source_job_run_id = %job_triggered.job_run_id,
"Want created (derivative - auto-created due to missing dependency)" "Want created (derivative - auto-created due to missing dependency)"
); );
true
} else {
false
} }
} else { } else {
tracing::info!( tracing::info!(
@ -688,17 +899,119 @@ impl BuildState {
partitions = ?event.partitions.iter().map(|p| &p.r#ref).collect::<Vec<_>>(), partitions = ?event.partitions.iter().map(|p| &p.r#ref).collect::<Vec<_>>(),
"Want created (user-requested)" "Want created (user-requested)"
); );
} false
};
self.wants // Register this want with all its partitions (via inverted index)
.insert(event.want_id.clone(), Want::Idle(want_idle)); self.register_want_for_partitions(&event.want_id, &event.partitions);
// Sense canonical partition states and determine initial want state
// Priority order: Failed > UpstreamFailed > AllLive > Building > UpstreamBuilding > UpForRetry > Idle
let mut failed_partitions: Vec<PartitionRef> = Vec::new();
let mut upstream_failed_wants: Vec<String> = Vec::new();
let mut all_live = true;
let mut any_building = false;
let mut any_upstream_building = false;
let mut any_up_for_retry = false;
let mut building_started_at: Option<u64> = None;
// Register this want with all its partitions
for pref in &event.partitions { for pref in &event.partitions {
self.add_want_to_partition(pref, &event.want_id); match self.get_canonical_partition(&pref.r#ref) {
Some(partition) => match partition {
Partition::Failed(_f) => {
failed_partitions.push(pref.clone());
all_live = false;
} }
Partition::UpstreamFailed(uf) => {
// Track which upstream refs failed
for failed_ref in &uf.state.failed_upstream_refs {
upstream_failed_wants.push(failed_ref.r#ref.clone());
}
all_live = false;
}
Partition::Live(_) => {
// Contributes to all_live check
}
Partition::Building(_b) => {
any_building = true;
all_live = false;
// Track when building started (use earliest if multiple)
if building_started_at.is_none() {
building_started_at = Some(current_timestamp());
}
}
Partition::UpstreamBuilding(_) => {
any_upstream_building = true;
all_live = false;
}
Partition::UpForRetry(_) => {
any_up_for_retry = true;
all_live = false;
}
Partition::Tainted(_) => {
// Tainted partitions need rebuild
all_live = false;
}
},
None => {
// Partition doesn't exist - needs to be built
all_live = false;
}
}
}
// Transition from New to appropriate state based on sensing
let final_want: Want = if !failed_partitions.is_empty() {
tracing::info!(
want_id = %event.want_id,
failed_partitions = ?failed_partitions.iter().map(|p| &p.r#ref).collect::<Vec<_>>(),
"Want: New → Failed (partition already failed)"
);
Want::Failed(
want_new.to_failed(failed_partitions, "Partition already failed".to_string()),
)
} else if !upstream_failed_wants.is_empty() {
tracing::info!(
want_id = %event.want_id,
upstream_failed = ?upstream_failed_wants,
"Want: New → UpstreamFailed (upstream already failed)"
);
Want::UpstreamFailed(want_new.to_upstream_failed(upstream_failed_wants))
} else if all_live && !event.partitions.is_empty() {
tracing::info!(
want_id = %event.want_id,
"Want: New → Successful (all partitions already live)"
);
Want::Successful(want_new.to_successful(current_timestamp()))
} else if any_building {
tracing::info!(
want_id = %event.want_id,
"Want: New → Building (partitions being built)"
);
Want::Building(
want_new.to_building(building_started_at.unwrap_or_else(current_timestamp)),
)
} else if any_upstream_building {
// For upstream building, we need the upstream want IDs
// For now, transition to Idle and let derivative want handling take care of it
tracing::info!(
want_id = %event.want_id,
"Want: New → Idle (upstream building - will be updated by derivative want handling)"
);
Want::Idle(want_new.to_idle())
} else {
// Partitions don't exist, or are UpForRetry - want is schedulable
tracing::info!(
want_id = %event.want_id,
"Want: New → Idle (ready to schedule)"
);
Want::Idle(want_new.to_idle())
};
self.wants.insert(event.want_id.clone(), final_want);
// If this is a derivative want (triggered by a job's dep miss), transition impacted wants to UpstreamBuilding // If this is a derivative want (triggered by a job's dep miss), transition impacted wants to UpstreamBuilding
if is_derivative {
if let Some(source) = &event.source { if let Some(source) = &event.source {
if let Some(EventSourceVariant::JobTriggered(job_triggered)) = &source.source { if let Some(EventSourceVariant::JobTriggered(job_triggered)) = &source.source {
self.handle_derivative_want_creation( self.handle_derivative_want_creation(
@ -708,6 +1021,7 @@ impl BuildState {
); );
} }
} }
}
vec![] vec![]
} }
@ -722,6 +1036,9 @@ impl BuildState {
)); ));
let canceled = match want { let canceled = match want {
Want::New(new_want) => {
Want::Canceled(new_want.cancel(event.source.clone(), event.comment.clone()))
}
Want::Idle(idle) => { Want::Idle(idle) => {
Want::Canceled(idle.cancel(event.source.clone(), event.comment.clone())) Want::Canceled(idle.cancel(event.source.clone(), event.comment.clone()))
} }
@ -774,6 +1091,16 @@ impl BuildState {
)); ));
let transitioned = match want { let transitioned = match want {
Want::New(new_want) => {
// Want was just created and hasn't fully sensed yet - transition to Building
// This can happen if want creation and job buffer happen in quick succession
tracing::info!(
want_id = %wap.want_id,
job_run_id = %event.job_run_id,
"Want: New → Building (job scheduled before sensing completed)"
);
Want::Building(new_want.to_building(current_timestamp()))
}
Want::Idle(idle) => { Want::Idle(idle) => {
// First job starting for this want // First job starting for this want
tracing::info!( tracing::info!(
@ -789,7 +1116,7 @@ impl BuildState {
} }
_ => { _ => {
panic!( panic!(
"BUG: Want {} in invalid state {:?} for job buffer. Only Idle or Building wants should be scheduled.", "BUG: Want {} in invalid state {:?} for job buffer. Only New, Idle or Building wants should be scheduled.",
wap.want_id, want wap.want_id, want
); );
} }
@ -876,6 +1203,9 @@ impl BuildState {
current_timestamp(), current_timestamp(),
); );
// UpstreamBuilding → UpForRetry (for downstream partitions waiting on newly live partitions)
self.unblock_downstream_partitions(&newly_live_partitions);
// Building → Successful (when all partitions Live) // Building → Successful (when all partitions Live)
let newly_successful_wants: Vec<SuccessfulWantId> = self.complete_successful_wants( let newly_successful_wants: Vec<SuccessfulWantId> = self.complete_successful_wants(
&newly_live_partitions, &newly_live_partitions,
@ -928,6 +1258,9 @@ impl BuildState {
current_timestamp(), current_timestamp(),
); );
// UpstreamBuilding → UpstreamFailed (for downstream partitions waiting on failed upstreams)
self.cascade_failures_to_downstream_partitions(&failed_partitions);
// Building → Failed (for wants directly building failed partitions) // Building → Failed (for wants directly building failed partitions)
let newly_failed_wants: Vec<FailedWantId> = let newly_failed_wants: Vec<FailedWantId> =
self.fail_directly_affected_wants(&failed_partitions); self.fail_directly_affected_wants(&failed_partitions);
@ -965,9 +1298,20 @@ impl BuildState {
} }
}; };
// Canceled job means building partitions should reset to Missing // Canceled job means building partitions should be removed (they never completed).
// In the new model without Missing state, partitions are only created when jobs
// start building them, and removed if the job is canceled before completion.
let building_refs_to_reset = canceled.get_building_partitions_to_reset(); let building_refs_to_reset = canceled.get_building_partitions_to_reset();
self.reset_partitions_to_missing(&building_refs_to_reset); for building_ref in &building_refs_to_reset {
// Remove from UUID map and canonical map
if let Some(uuid) = self.canonical_partitions.remove(&building_ref.0.r#ref) {
self.partitions_by_uuid.remove(&uuid);
}
tracing::info!(
partition = %building_ref.0.r#ref,
"Partition removed (job canceled)"
);
}
self.job_runs self.job_runs
.insert(event.job_run_id.clone(), JobRun::Canceled(canceled)); .insert(event.job_run_id.clone(), JobRun::Canceled(canceled));
@ -1012,9 +1356,16 @@ impl BuildState {
.reduce(|a: WantTimestamps, b: WantTimestamps| a.merge(b)) .reduce(|a: WantTimestamps, b: WantTimestamps| a.merge(b))
.expect("BUG: No servicing wants found"); .expect("BUG: No servicing wants found");
// Transition partitions back to Missing since this job can't build them yet // Collect all missing deps into a flat list of partition refs
let all_missing_deps: Vec<PartitionRef> = event
.missing_deps
.iter()
.flat_map(|md| md.missing.clone())
.collect();
// Transition partitions from Building to UpstreamBuilding since this job can't build them yet
let building_refs_to_reset = dep_miss.get_building_partitions_to_reset(); let building_refs_to_reset = dep_miss.get_building_partitions_to_reset();
self.reset_partitions_to_missing(&building_refs_to_reset); self.transition_partitions_to_upstream_building(&building_refs_to_reset, all_missing_deps);
// Generate WantCreateV1 events for the missing dependencies // Generate WantCreateV1 events for the missing dependencies
// These events will be returned and appended to the BEL by BuildEventLog.append_event() // These events will be returned and appended to the BEL by BuildEventLog.append_event()
@ -1039,12 +1390,85 @@ impl BuildState {
want_events want_events
} }
fn handle_taint_create(&mut self, _event: &TaintCreateEventV1) -> Vec<Event> { fn handle_taint_create(&mut self, event: &TaintCreateEventV1) -> Vec<Event> {
todo!("...?") // Store the taint detail
let taint_detail = TaintDetail {
taint_id: event.taint_id.clone(),
root_taint_id: event.root_taint_id.clone(),
parent_taint_id: event.parent_taint_id.clone(),
partitions: event.partitions.clone(),
source: event.source.clone(),
comment: event.comment.clone(),
};
self.taints.insert(event.taint_id.clone(), taint_detail);
// Transition affected partitions to Tainted state
for pref in &event.partitions {
if let Some(partition) = self.take_canonical_partition(&pref.r#ref) {
let transitioned = match partition {
Partition::Live(live) => {
tracing::info!(
partition = %pref.r#ref,
taint_id = %event.taint_id,
"Partition: Live → Tainted"
);
Partition::Tainted(live.taint(event.taint_id.clone(), current_timestamp()))
}
Partition::Tainted(tainted) => {
// Add additional taint to already-tainted partition
tracing::info!(
partition = %pref.r#ref,
taint_id = %event.taint_id,
"Partition: Tainted → Tainted (adding taint)"
);
Partition::Tainted(tainted.add_taint(event.taint_id.clone()))
}
other => {
// For non-Live/Tainted partitions (Building, UpstreamBuilding, etc.),
// we can't taint them - log a warning and skip
tracing::warn!(
partition = %pref.r#ref,
taint_id = %event.taint_id,
state = ?other,
"Cannot taint partition in non-Live state, skipping"
);
other
}
};
self.update_partition(transitioned);
} else {
// Partition doesn't exist yet - this is fine, taint will apply when it's built
tracing::debug!(
partition = %pref.r#ref,
taint_id = %event.taint_id,
"Taint targeting non-existent partition, will apply when built"
);
}
} }
fn handle_taint_delete(&mut self, _event: &TaintCancelEventV1) -> Vec<Event> { vec![]
todo!("...?") }
fn handle_taint_delete(&mut self, event: &TaintCancelEventV1) -> Vec<Event> {
// Remove the taint from our tracking
if let Some(taint) = self.taints.remove(&event.taint_id) {
tracing::info!(
taint_id = %event.taint_id,
partitions = ?taint.partitions.iter().map(|p| &p.r#ref).collect::<Vec<_>>(),
"Taint canceled/deleted"
);
// Note: We do NOT automatically un-taint partitions when a taint is canceled.
// Once tainted, partitions remain tainted until they are rebuilt.
// The taint_ids on the partition are historical records of why it was tainted.
} else {
tracing::warn!(
taint_id = %event.taint_id,
"Attempted to cancel non-existent taint"
);
}
vec![]
} }
fn with_wants(self, wants: BTreeMap<String, Want>) -> Self { fn with_wants(self, wants: BTreeMap<String, Want>) -> Self {
@ -1053,16 +1477,35 @@ impl BuildState {
#[cfg(test)] #[cfg(test)]
fn with_partitions(self, old_partitions: BTreeMap<String, PartitionDetail>) -> Self { fn with_partitions(self, old_partitions: BTreeMap<String, PartitionDetail>) -> Self {
// Convert PartitionDetail to Partition (for backfill scenarios) use crate::partition_state::PartitionWithState;
let partitions: BTreeMap<String, Partition> = old_partitions
.into_iter() let mut canonical_partitions: BTreeMap<String, Uuid> = BTreeMap::new();
.map(|(key, detail)| { let mut partitions_by_uuid: BTreeMap<Uuid, Partition> = BTreeMap::new();
// For now, just create in Missing state - real migration would be more sophisticated
let partition = Partition::new_missing(detail.r#ref.clone().unwrap_or_default()); // Convert PartitionDetail to Live partitions for testing
(key, partition) for (key, detail) in old_partitions {
}) let partition_ref = detail.r#ref.clone().unwrap_or_default();
.collect(); // Create a deterministic UUID for test data
Self { partitions, ..self } let uuid =
crate::partition_state::derive_partition_uuid("test_job_run", &partition_ref.r#ref);
let live_partition = Partition::Live(PartitionWithState {
uuid,
partition_ref,
state: crate::partition_state::LiveState {
built_at: 0,
built_by: "test_job_run".to_string(),
},
});
canonical_partitions.insert(key, uuid);
partitions_by_uuid.insert(uuid, live_partition);
}
Self {
canonical_partitions,
partitions_by_uuid,
..self
}
} }
pub fn get_want(&self, want_id: &str) -> Option<WantDetail> { pub fn get_want(&self, want_id: &str) -> Option<WantDetail> {
@ -1072,7 +1515,8 @@ impl BuildState {
self.taints.get(taint_id).cloned() self.taints.get(taint_id).cloned()
} }
pub fn get_partition(&self, partition_id: &str) -> Option<PartitionDetail> { pub fn get_partition(&self, partition_id: &str) -> Option<PartitionDetail> {
self.partitions.get(partition_id).map(|p| p.to_detail()) self.get_canonical_partition(partition_id)
.map(|p| p.to_detail())
} }
pub fn get_job_run(&self, job_run_id: &str) -> Option<JobRunDetail> { pub fn get_job_run(&self, job_run_id: &str) -> Option<JobRunDetail> {
self.job_runs.get(job_run_id).map(|jr| jr.to_detail()) self.job_runs.get(job_run_id).map(|jr| jr.to_detail())
@ -1115,15 +1559,19 @@ impl BuildState {
pub fn list_partitions(&self, request: &ListPartitionsRequest) -> ListPartitionsResponse { pub fn list_partitions(&self, request: &ListPartitionsRequest) -> ListPartitionsResponse {
let page = request.page.unwrap_or(0); let page = request.page.unwrap_or(0);
let page_size = request.page_size.unwrap_or(consts::DEFAULT_PAGE_SIZE); let page_size = request.page_size.unwrap_or(consts::DEFAULT_PAGE_SIZE);
// Convert Partition to PartitionDetail for API // Convert canonical partitions to PartitionDetail for API
let partition_details: BTreeMap<String, PartitionDetail> = self let partition_details: BTreeMap<String, PartitionDetail> = self
.partitions .canonical_partitions
.iter() .iter()
.map(|(k, v)| (k.clone(), v.to_detail())) .filter_map(|(k, uuid)| {
self.partitions_by_uuid
.get(uuid)
.map(|p| (k.clone(), p.to_detail()))
})
.collect(); .collect();
ListPartitionsResponse { ListPartitionsResponse {
data: list_state_items(&partition_details, page, page_size), data: list_state_items(&partition_details, page, page_size),
match_count: self.wants.len() as u64, match_count: self.canonical_partitions.len() as u64,
page, page,
page_size, page_size,
} }
@ -1157,23 +1605,21 @@ impl BuildState {
// Check upstream partition statuses (dependencies) // Check upstream partition statuses (dependencies)
let mut live: Vec<LivePartitionRef> = Vec::new(); let mut live: Vec<LivePartitionRef> = Vec::new();
let mut tainted: Vec<TaintedPartitionRef> = Vec::new(); let mut tainted: Vec<TaintedPartitionRef> = Vec::new();
let mut missing: Vec<MissingPartitionRef> = Vec::new(); let mut not_ready: Vec<PartitionRef> = Vec::new(); // Partitions that don't exist or aren't Live
for upstream_ref in &want.upstreams { for upstream_ref in &want.upstreams {
match self.partitions.get(&upstream_ref.r#ref) { match self.get_canonical_partition(&upstream_ref.r#ref) {
Some(partition) => { Some(partition) => {
match partition { match partition {
Partition::Live(p) => live.push(p.get_ref()), Partition::Live(p) => live.push(p.get_ref()),
Partition::Tainted(p) => tainted.push(p.get_ref()), Partition::Tainted(p) => tainted.push(p.get_ref()),
Partition::Missing(p) => missing.push(p.get_ref()), // All other states (Building, UpstreamBuilding, UpForRetry, Failed, UpstreamFailed) mean upstream is not ready
_ => (), // Other states (Missing, Building, Failed) don't add to any list _ => not_ready.push(upstream_ref.clone()),
} }
} }
None => { None => {
// TODO this definitely feels dirty, but we can't take a mutable ref of self to // Partition doesn't exist yet - it's not ready
// insert the missing partition here, and it feels a little over the top to not_ready.push(upstream_ref.clone());
// create a more elaborate way to mint a missing ref.
missing.push(MissingPartitionRef(upstream_ref.clone()));
} }
} }
} }
@ -1182,7 +1628,7 @@ impl BuildState {
// If any target partition is already Building, this want should wait // If any target partition is already Building, this want should wait
let mut building: Vec<BuildingPartitionRef> = Vec::new(); let mut building: Vec<BuildingPartitionRef> = Vec::new();
for target_ref in &want.partitions { for target_ref in &want.partitions {
if let Some(partition) = self.partitions.get(&target_ref.r#ref) { if let Some(partition) = self.get_canonical_partition(&target_ref.r#ref) {
if let Partition::Building(p) = partition { if let Partition::Building(p) = partition {
building.push(p.get_ref()); building.push(p.get_ref());
} }
@ -1194,7 +1640,7 @@ impl BuildState {
status: WantUpstreamStatus { status: WantUpstreamStatus {
live, live,
tainted, tainted,
missing, not_ready,
building, building,
}, },
} }
@ -1217,7 +1663,8 @@ impl BuildState {
pub struct WantUpstreamStatus { pub struct WantUpstreamStatus {
pub live: Vec<LivePartitionRef>, pub live: Vec<LivePartitionRef>,
pub tainted: Vec<TaintedPartitionRef>, pub tainted: Vec<TaintedPartitionRef>,
pub missing: Vec<MissingPartitionRef>, /// Upstream partitions that are not ready (don't exist, or are in Building/UpstreamBuilding/UpForRetry/Failed/UpstreamFailed states)
pub not_ready: Vec<PartitionRef>,
/// Target partitions that are currently being built by another job /// Target partitions that are currently being built by another job
pub building: Vec<BuildingPartitionRef>, pub building: Vec<BuildingPartitionRef>,
} }
@ -1246,10 +1693,10 @@ impl WantsSchedulability {
impl WantSchedulability { impl WantSchedulability {
pub fn is_schedulable(&self) -> bool { pub fn is_schedulable(&self) -> bool {
// Want is schedulable if: // Want is schedulable if:
// - No missing upstream dependencies // - No not-ready upstream dependencies (must all be Live or Tainted)
// - No tainted upstream dependencies // - No tainted upstream dependencies
// - No target partitions currently being built by another job // - No target partitions currently being built by another job
self.status.missing.is_empty() self.status.not_ready.is_empty()
&& self.status.tainted.is_empty() && self.status.tainted.is_empty()
&& self.status.building.is_empty() && self.status.building.is_empty()
} }

View file

@ -227,6 +227,9 @@ message PartitionDetail {
// Wants that reference this partition // Wants that reference this partition
repeated string want_ids = 5; repeated string want_ids = 5;
repeated string taint_ids = 6; repeated string taint_ids = 6;
// The unique identifier for this partition instance (UUID as string)
// Each time a partition is built, it gets a new UUID derived from the job_run_id
string uuid = 7;
} }
message PartitionStatus { message PartitionStatus {
PartitionStatusCode code = 1; PartitionStatusCode code = 1;
@ -248,7 +251,18 @@ enum PartitionStatusCode {
} }
message TaintDetail { message TaintDetail {
// TODO // The unique identifier for this taint
string taint_id = 1;
// The root taint ID (for taint hierarchies)
string root_taint_id = 2;
// The parent taint ID (for taint hierarchies)
string parent_taint_id = 3;
// The partitions affected by this taint
repeated PartitionRef partitions = 4;
// Source of the taint event
EventSource source = 5;
// Optional comment describing the taint
optional string comment = 6;
} }
message JobRunStatus { message JobRunStatus {

View file

@ -1,14 +1,35 @@
use crate::{PartitionDetail, PartitionRef, PartitionStatus, PartitionStatusCode}; use crate::{PartitionDetail, PartitionRef, PartitionStatus, PartitionStatusCode};
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
use sha2::{Digest, Sha256};
use uuid::Uuid;
/// State: Partition has been referenced but not yet built /// Derive a deterministic UUID from job_run_id and partition_ref.
#[derive(Debug, Clone)] /// This ensures replay produces the same UUIDs.
pub struct MissingState {} pub fn derive_partition_uuid(job_run_id: &str, partition_ref: &str) -> Uuid {
let mut hasher = Sha256::new();
hasher.update(job_run_id.as_bytes());
hasher.update(partition_ref.as_bytes());
let hash = hasher.finalize();
Uuid::from_slice(&hash[0..16]).expect("SHA256 produces at least 16 bytes")
}
/// State: Partition is currently being built by one or more jobs /// State: Partition is currently being built by a job
#[derive(Debug, Clone)] #[derive(Debug, Clone)]
pub struct BuildingState { pub struct BuildingState {
pub building_by: Vec<String>, // job_run_ids pub job_run_id: String,
}
/// State: Partition is waiting for upstream dependencies to be built
#[derive(Debug, Clone)]
pub struct UpstreamBuildingState {
pub job_run_id: String,
pub missing_deps: Vec<PartitionRef>, // partition refs that are missing
}
/// State: Upstream dependencies are satisfied, partition is ready to retry building
#[derive(Debug, Clone)]
pub struct UpForRetryState {
pub original_job_run_id: String, // job that had the dep miss
} }
/// State: Partition has been successfully built /// State: Partition has been successfully built
@ -18,13 +39,20 @@ pub struct LiveState {
pub built_by: String, // job_run_id pub built_by: String, // job_run_id
} }
/// State: Partition build failed /// State: Partition build failed (hard failure, not retryable)
#[derive(Debug, Clone)] #[derive(Debug, Clone)]
pub struct FailedState { pub struct FailedState {
pub failed_at: u64, pub failed_at: u64,
pub failed_by: String, // job_run_id pub failed_by: String, // job_run_id
} }
/// State: Partition failed because upstream dependencies failed (terminal)
#[derive(Debug, Clone)]
pub struct UpstreamFailedState {
pub failed_at: u64,
pub failed_upstream_refs: Vec<PartitionRef>, // which upstream partitions failed
}
/// State: Partition has been marked as invalid/tainted /// State: Partition has been marked as invalid/tainted
#[derive(Debug, Clone)] #[derive(Debug, Clone)]
pub struct TaintedState { pub struct TaintedState {
@ -32,21 +60,25 @@ pub struct TaintedState {
pub taint_ids: Vec<String>, pub taint_ids: Vec<String>,
} }
/// Generic partition struct parameterized by state /// Generic partition struct parameterized by state.
/// Each partition has a unique UUID derived from the job_run_id that created it.
#[derive(Debug, Clone)] #[derive(Debug, Clone)]
pub struct PartitionWithState<S> { pub struct PartitionWithState<S> {
pub uuid: Uuid,
pub partition_ref: PartitionRef, pub partition_ref: PartitionRef,
pub want_ids: Vec<String>,
pub state: S, pub state: S,
} }
/// Wrapper enum for storing partitions in collections /// Wrapper enum for storing partitions in collections.
/// Note: Missing state has been removed - partitions are only created when jobs start building them.
#[derive(Debug, Clone)] #[derive(Debug, Clone)]
pub enum Partition { pub enum Partition {
Missing(PartitionWithState<MissingState>),
Building(PartitionWithState<BuildingState>), Building(PartitionWithState<BuildingState>),
UpstreamBuilding(PartitionWithState<UpstreamBuildingState>),
UpForRetry(PartitionWithState<UpForRetryState>),
Live(PartitionWithState<LiveState>), Live(PartitionWithState<LiveState>),
Failed(PartitionWithState<FailedState>), Failed(PartitionWithState<FailedState>),
UpstreamFailed(PartitionWithState<UpstreamFailedState>),
Tainted(PartitionWithState<TaintedState>), Tainted(PartitionWithState<TaintedState>),
} }
@ -54,14 +86,6 @@ pub enum Partition {
/// is critical that these be treated with respect, not just summoned because it's convenient. /// is critical that these be treated with respect, not just summoned because it's convenient.
/// These should be created ephemerally from typestate objects via .get_ref() and used /// These should be created ephemerally from typestate objects via .get_ref() and used
/// immediately — never stored long-term, as partition state can change. /// immediately — never stored long-term, as partition state can change.
#[derive(Debug, Clone, Serialize, Deserialize, Default)]
pub struct MissingPartitionRef(pub PartitionRef);
impl PartitionWithState<MissingState> {
pub fn get_ref(&self) -> MissingPartitionRef {
MissingPartitionRef(self.partition_ref.clone())
}
}
#[derive(Debug, Clone, Serialize, Deserialize, Default)] #[derive(Debug, Clone, Serialize, Deserialize, Default)]
pub struct BuildingPartitionRef(pub PartitionRef); pub struct BuildingPartitionRef(pub PartitionRef);
impl PartitionWithState<BuildingState> { impl PartitionWithState<BuildingState> {
@ -70,6 +94,22 @@ impl PartitionWithState<BuildingState> {
} }
} }
#[derive(Debug, Clone, Serialize, Deserialize, Default)]
pub struct UpstreamBuildingPartitionRef(pub PartitionRef);
impl PartitionWithState<UpstreamBuildingState> {
pub fn get_ref(&self) -> UpstreamBuildingPartitionRef {
UpstreamBuildingPartitionRef(self.partition_ref.clone())
}
}
#[derive(Debug, Clone, Serialize, Deserialize, Default)]
pub struct UpForRetryPartitionRef(pub PartitionRef);
impl PartitionWithState<UpForRetryState> {
pub fn get_ref(&self) -> UpForRetryPartitionRef {
UpForRetryPartitionRef(self.partition_ref.clone())
}
}
#[derive(Debug, Clone, Serialize, Deserialize, Default)] #[derive(Debug, Clone, Serialize, Deserialize, Default)]
pub struct LivePartitionRef(pub PartitionRef); pub struct LivePartitionRef(pub PartitionRef);
impl PartitionWithState<LiveState> { impl PartitionWithState<LiveState> {
@ -86,6 +126,14 @@ impl PartitionWithState<FailedState> {
} }
} }
#[derive(Debug, Clone, Serialize, Deserialize, Default)]
pub struct UpstreamFailedPartitionRef(pub PartitionRef);
impl PartitionWithState<UpstreamFailedState> {
pub fn get_ref(&self) -> UpstreamFailedPartitionRef {
UpstreamFailedPartitionRef(self.partition_ref.clone())
}
}
#[derive(Debug, Clone, Serialize, Deserialize, Default)] #[derive(Debug, Clone, Serialize, Deserialize, Default)]
pub struct TaintedPartitionRef(pub PartitionRef); pub struct TaintedPartitionRef(pub PartitionRef);
impl PartitionWithState<TaintedState> { impl PartitionWithState<TaintedState> {
@ -94,62 +142,102 @@ impl PartitionWithState<TaintedState> {
} }
} }
// Type-safe transition methods for MissingState
impl PartitionWithState<MissingState> {
/// Transition from Missing to Building when a job starts building this partition
pub fn start_building(self, job_run_id: String) -> PartitionWithState<BuildingState> {
PartitionWithState {
partition_ref: self.partition_ref,
want_ids: self.want_ids,
state: BuildingState {
building_by: vec![job_run_id],
},
}
}
}
// Type-safe transition methods for BuildingState // Type-safe transition methods for BuildingState
impl PartitionWithState<BuildingState> { impl PartitionWithState<BuildingState> {
/// Transition from Building to Live when a job successfully completes /// Create a new partition directly in Building state.
pub fn complete(self, job_run_id: String, timestamp: u64) -> PartitionWithState<LiveState> { /// UUID is derived from job_run_id + partition_ref for deterministic replay.
pub fn new(job_run_id: String, partition_ref: PartitionRef) -> Self {
let uuid = derive_partition_uuid(&job_run_id, &partition_ref.r#ref);
PartitionWithState { PartitionWithState {
uuid,
partition_ref,
state: BuildingState { job_run_id },
}
}
/// Transition from Building to Live when a job successfully completes
pub fn complete(self, timestamp: u64) -> PartitionWithState<LiveState> {
PartitionWithState {
uuid: self.uuid,
partition_ref: self.partition_ref, partition_ref: self.partition_ref,
want_ids: self.want_ids,
state: LiveState { state: LiveState {
built_at: timestamp, built_at: timestamp,
built_by: job_run_id, built_by: self.state.job_run_id,
}, },
} }
} }
/// Transition from Building to Failed when a job fails /// Transition from Building to Failed when a job fails (hard failure)
pub fn fail(self, job_run_id: String, timestamp: u64) -> PartitionWithState<FailedState> { pub fn fail(self, timestamp: u64) -> PartitionWithState<FailedState> {
PartitionWithState { PartitionWithState {
uuid: self.uuid,
partition_ref: self.partition_ref, partition_ref: self.partition_ref,
want_ids: self.want_ids,
state: FailedState { state: FailedState {
failed_at: timestamp, failed_at: timestamp,
failed_by: job_run_id, failed_by: self.state.job_run_id,
}, },
} }
} }
/// Add another job to the list of jobs building this partition /// Transition from Building to UpstreamBuilding when job reports missing dependencies
pub fn add_building_job(mut self, job_run_id: String) -> Self { pub fn dep_miss(
if !self.state.building_by.contains(&job_run_id) { self,
self.state.building_by.push(job_run_id); missing_deps: Vec<PartitionRef>,
) -> PartitionWithState<UpstreamBuildingState> {
PartitionWithState {
uuid: self.uuid,
partition_ref: self.partition_ref,
state: UpstreamBuildingState {
job_run_id: self.state.job_run_id,
missing_deps,
},
}
} }
self
} }
/// Transition from Building back to Missing when a job discovers missing dependencies // Type-safe transition methods for UpstreamBuildingState
pub fn reset_to_missing(self) -> PartitionWithState<MissingState> { impl PartitionWithState<UpstreamBuildingState> {
/// Transition from UpstreamBuilding to UpForRetry when all upstream deps are satisfied
pub fn upstreams_satisfied(self) -> PartitionWithState<UpForRetryState> {
PartitionWithState { PartitionWithState {
uuid: self.uuid,
partition_ref: self.partition_ref, partition_ref: self.partition_ref,
want_ids: self.want_ids, state: UpForRetryState {
state: MissingState {}, original_job_run_id: self.state.job_run_id,
},
} }
} }
/// Transition from UpstreamBuilding to UpstreamFailed when an upstream dep fails
pub fn upstream_failed(
self,
failed_upstream_refs: Vec<PartitionRef>,
timestamp: u64,
) -> PartitionWithState<UpstreamFailedState> {
PartitionWithState {
uuid: self.uuid,
partition_ref: self.partition_ref,
state: UpstreamFailedState {
failed_at: timestamp,
failed_upstream_refs,
},
}
}
/// Check if a specific upstream ref is in our missing deps
pub fn is_waiting_for(&self, upstream_ref: &str) -> bool {
self.state
.missing_deps
.iter()
.any(|d| d.r#ref == upstream_ref)
}
/// Remove a satisfied upstream from missing deps. Returns remaining count.
pub fn satisfy_upstream(mut self, upstream_ref: &str) -> (Self, usize) {
self.state.missing_deps.retain(|r| r.r#ref != upstream_ref);
let remaining = self.state.missing_deps.len();
(self, remaining)
}
} }
// Type-safe transition methods for LiveState // Type-safe transition methods for LiveState
@ -157,8 +245,8 @@ impl PartitionWithState<LiveState> {
/// Transition from Live to Tainted when a taint is applied /// Transition from Live to Tainted when a taint is applied
pub fn taint(self, taint_id: String, timestamp: u64) -> PartitionWithState<TaintedState> { pub fn taint(self, taint_id: String, timestamp: u64) -> PartitionWithState<TaintedState> {
PartitionWithState { PartitionWithState {
uuid: self.uuid,
partition_ref: self.partition_ref, partition_ref: self.partition_ref,
want_ids: self.want_ids,
state: TaintedState { state: TaintedState {
tainted_at: timestamp, tainted_at: timestamp,
taint_ids: vec![taint_id], taint_ids: vec![taint_id],
@ -180,100 +268,115 @@ impl PartitionWithState<TaintedState> {
// Helper methods on the Partition enum // Helper methods on the Partition enum
impl Partition { impl Partition {
/// Create a new partition in the Missing state /// Get the UUID from any state
pub fn new_missing(partition_ref: PartitionRef) -> Self { pub fn uuid(&self) -> Uuid {
Partition::Missing(PartitionWithState { match self {
partition_ref, Partition::Building(p) => p.uuid,
want_ids: vec![], Partition::UpstreamBuilding(p) => p.uuid,
state: MissingState {}, Partition::UpForRetry(p) => p.uuid,
}) Partition::Live(p) => p.uuid,
Partition::Failed(p) => p.uuid,
Partition::UpstreamFailed(p) => p.uuid,
Partition::Tainted(p) => p.uuid,
}
} }
/// Get the partition reference from any state /// Get the partition reference from any state
pub fn partition_ref(&self) -> &PartitionRef { pub fn partition_ref(&self) -> &PartitionRef {
match self { match self {
Partition::Missing(p) => &p.partition_ref,
Partition::Building(p) => &p.partition_ref, Partition::Building(p) => &p.partition_ref,
Partition::UpstreamBuilding(p) => &p.partition_ref,
Partition::UpForRetry(p) => &p.partition_ref,
Partition::Live(p) => &p.partition_ref, Partition::Live(p) => &p.partition_ref,
Partition::Failed(p) => &p.partition_ref, Partition::Failed(p) => &p.partition_ref,
Partition::UpstreamFailed(p) => &p.partition_ref,
Partition::Tainted(p) => &p.partition_ref, Partition::Tainted(p) => &p.partition_ref,
} }
} }
/// Get want_ids from any state
pub fn want_ids(&self) -> &Vec<String> {
match self {
Partition::Missing(p) => &p.want_ids,
Partition::Building(p) => &p.want_ids,
Partition::Live(p) => &p.want_ids,
Partition::Failed(p) => &p.want_ids,
Partition::Tainted(p) => &p.want_ids,
}
}
/// Get mutable want_ids from any state
pub fn want_ids_mut(&mut self) -> &mut Vec<String> {
match self {
Partition::Missing(p) => &mut p.want_ids,
Partition::Building(p) => &mut p.want_ids,
Partition::Live(p) => &mut p.want_ids,
Partition::Failed(p) => &mut p.want_ids,
Partition::Tainted(p) => &mut p.want_ids,
}
}
/// Check if partition is in Live state /// Check if partition is in Live state
pub fn is_live(&self) -> bool { pub fn is_live(&self) -> bool {
matches!(self, Partition::Live(_)) matches!(self, Partition::Live(_))
} }
/// Check if partition is satisfied (Live or Tainted both count as "available") /// Check if partition is in a terminal state (Live, Failed, UpstreamFailed, or Tainted)
pub fn is_satisfied(&self) -> bool {
matches!(self, Partition::Live(_) | Partition::Tainted(_))
}
/// Check if partition is in a terminal state (Live, Failed, or Tainted)
pub fn is_terminal(&self) -> bool { pub fn is_terminal(&self) -> bool {
matches!( matches!(
self, self,
Partition::Live(_) | Partition::Failed(_) | Partition::Tainted(_) Partition::Live(_)
| Partition::Failed(_)
| Partition::UpstreamFailed(_)
| Partition::Tainted(_)
) )
} }
/// Check if partition is currently being built /// Check if partition is currently being built (includes UpstreamBuilding as it holds a "lease")
pub fn is_building(&self) -> bool { pub fn is_building(&self) -> bool {
matches!(self, Partition::Building(_)) matches!(
self,
Partition::Building(_) | Partition::UpstreamBuilding(_)
)
} }
/// Check if partition is missing (referenced but not built) /// Check if partition is in UpForRetry state (ready to be rebuilt)
pub fn is_missing(&self) -> bool { pub fn is_up_for_retry(&self) -> bool {
matches!(self, Partition::Missing(_)) matches!(self, Partition::UpForRetry(_))
} }
/// Convert to PartitionDetail for API responses and queries /// Check if partition is failed (hard failure)
pub fn is_failed(&self) -> bool {
matches!(self, Partition::Failed(_))
}
/// Check if partition is upstream failed
pub fn is_upstream_failed(&self) -> bool {
matches!(self, Partition::UpstreamFailed(_))
}
/// Check if partition is tainted
pub fn is_tainted(&self) -> bool {
matches!(self, Partition::Tainted(_))
}
/// Convert to PartitionDetail for API responses and queries.
/// Note: want_ids is now empty - this will be populated by BuildState from the inverted index.
pub fn to_detail(&self) -> PartitionDetail { pub fn to_detail(&self) -> PartitionDetail {
match self { match self {
Partition::Missing(p) => PartitionDetail {
r#ref: Some(p.partition_ref.clone()),
status: Some(PartitionStatus {
code: PartitionStatusCode::PartitionWanted as i32,
name: "PartitionWanted".to_string(),
}),
want_ids: p.want_ids.clone(),
job_run_ids: vec![],
taint_ids: vec![],
last_updated_timestamp: None,
},
Partition::Building(p) => PartitionDetail { Partition::Building(p) => PartitionDetail {
r#ref: Some(p.partition_ref.clone()), r#ref: Some(p.partition_ref.clone()),
status: Some(PartitionStatus { status: Some(PartitionStatus {
code: PartitionStatusCode::PartitionBuilding as i32, code: PartitionStatusCode::PartitionBuilding as i32,
name: "PartitionBuilding".to_string(), name: "PartitionBuilding".to_string(),
}), }),
want_ids: p.want_ids.clone(), want_ids: vec![], // Populated by BuildState
job_run_ids: p.state.building_by.clone(), job_run_ids: vec![p.state.job_run_id.clone()],
taint_ids: vec![], taint_ids: vec![],
last_updated_timestamp: None, last_updated_timestamp: None,
uuid: p.uuid.to_string(),
},
Partition::UpstreamBuilding(p) => PartitionDetail {
r#ref: Some(p.partition_ref.clone()),
status: Some(PartitionStatus {
code: PartitionStatusCode::PartitionBuilding as i32, // Use Building status for UpstreamBuilding
name: "PartitionUpstreamBuilding".to_string(),
}),
want_ids: vec![], // Populated by BuildState
job_run_ids: vec![p.state.job_run_id.clone()],
taint_ids: vec![],
last_updated_timestamp: None,
uuid: p.uuid.to_string(),
},
Partition::UpForRetry(p) => PartitionDetail {
r#ref: Some(p.partition_ref.clone()),
status: Some(PartitionStatus {
code: PartitionStatusCode::PartitionBuilding as i32, // Still "building" conceptually
name: "PartitionUpForRetry".to_string(),
}),
want_ids: vec![], // Populated by BuildState
job_run_ids: vec![p.state.original_job_run_id.clone()],
taint_ids: vec![],
last_updated_timestamp: None,
uuid: p.uuid.to_string(),
}, },
Partition::Live(p) => PartitionDetail { Partition::Live(p) => PartitionDetail {
r#ref: Some(p.partition_ref.clone()), r#ref: Some(p.partition_ref.clone()),
@ -281,10 +384,11 @@ impl Partition {
code: PartitionStatusCode::PartitionLive as i32, code: PartitionStatusCode::PartitionLive as i32,
name: "PartitionLive".to_string(), name: "PartitionLive".to_string(),
}), }),
want_ids: p.want_ids.clone(), want_ids: vec![], // Populated by BuildState
job_run_ids: vec![p.state.built_by.clone()], job_run_ids: vec![p.state.built_by.clone()],
taint_ids: vec![], taint_ids: vec![],
last_updated_timestamp: Some(p.state.built_at), last_updated_timestamp: Some(p.state.built_at),
uuid: p.uuid.to_string(),
}, },
Partition::Failed(p) => PartitionDetail { Partition::Failed(p) => PartitionDetail {
r#ref: Some(p.partition_ref.clone()), r#ref: Some(p.partition_ref.clone()),
@ -292,10 +396,23 @@ impl Partition {
code: PartitionStatusCode::PartitionFailed as i32, code: PartitionStatusCode::PartitionFailed as i32,
name: "PartitionFailed".to_string(), name: "PartitionFailed".to_string(),
}), }),
want_ids: p.want_ids.clone(), want_ids: vec![], // Populated by BuildState
job_run_ids: vec![p.state.failed_by.clone()], job_run_ids: vec![p.state.failed_by.clone()],
taint_ids: vec![], taint_ids: vec![],
last_updated_timestamp: Some(p.state.failed_at), last_updated_timestamp: Some(p.state.failed_at),
uuid: p.uuid.to_string(),
},
Partition::UpstreamFailed(p) => PartitionDetail {
r#ref: Some(p.partition_ref.clone()),
status: Some(PartitionStatus {
code: PartitionStatusCode::PartitionFailed as i32, // Use Failed status
name: "PartitionUpstreamFailed".to_string(),
}),
want_ids: vec![], // Populated by BuildState
job_run_ids: vec![],
taint_ids: vec![],
last_updated_timestamp: Some(p.state.failed_at),
uuid: p.uuid.to_string(),
}, },
Partition::Tainted(p) => PartitionDetail { Partition::Tainted(p) => PartitionDetail {
r#ref: Some(p.partition_ref.clone()), r#ref: Some(p.partition_ref.clone()),
@ -303,11 +420,92 @@ impl Partition {
code: PartitionStatusCode::PartitionTainted as i32, code: PartitionStatusCode::PartitionTainted as i32,
name: "PartitionTainted".to_string(), name: "PartitionTainted".to_string(),
}), }),
want_ids: p.want_ids.clone(), want_ids: vec![], // Populated by BuildState
job_run_ids: vec![], job_run_ids: vec![],
taint_ids: p.state.taint_ids.clone(), taint_ids: p.state.taint_ids.clone(),
last_updated_timestamp: Some(p.state.tainted_at), last_updated_timestamp: Some(p.state.tainted_at),
uuid: p.uuid.to_string(),
}, },
} }
} }
} }
#[cfg(test)]
mod tests {
use super::*;
#[test]
fn test_derive_partition_uuid_deterministic() {
let uuid1 = derive_partition_uuid("job-123", "data/beta");
let uuid2 = derive_partition_uuid("job-123", "data/beta");
assert_eq!(uuid1, uuid2);
}
#[test]
fn test_derive_partition_uuid_different_inputs() {
let uuid1 = derive_partition_uuid("job-123", "data/beta");
let uuid2 = derive_partition_uuid("job-456", "data/beta");
let uuid3 = derive_partition_uuid("job-123", "data/alpha");
assert_ne!(uuid1, uuid2);
assert_ne!(uuid1, uuid3);
assert_ne!(uuid2, uuid3);
}
#[test]
fn test_partition_building_transitions() {
let partition = PartitionWithState::<BuildingState>::new(
"job-123".to_string(),
PartitionRef {
r#ref: "data/beta".to_string(),
},
);
// Can transition to Live
let live = partition.clone().complete(1000);
assert_eq!(live.state.built_at, 1000);
assert_eq!(live.state.built_by, "job-123");
// Can transition to Failed
let failed = partition.clone().fail(2000);
assert_eq!(failed.state.failed_at, 2000);
assert_eq!(failed.state.failed_by, "job-123");
// Can transition to UpstreamBuilding (dep miss)
let upstream_building = partition.dep_miss(vec![PartitionRef {
r#ref: "data/alpha".to_string(),
}]);
assert_eq!(upstream_building.state.missing_deps.len(), 1);
assert_eq!(upstream_building.state.missing_deps[0].r#ref, "data/alpha");
}
#[test]
fn test_upstream_building_transitions() {
let building = PartitionWithState::<BuildingState>::new(
"job-123".to_string(),
PartitionRef {
r#ref: "data/beta".to_string(),
},
);
let upstream_building = building.dep_miss(vec![PartitionRef {
r#ref: "data/alpha".to_string(),
}]);
// Can transition to UpForRetry
let up_for_retry = upstream_building.clone().upstreams_satisfied();
assert_eq!(up_for_retry.state.original_job_run_id, "job-123");
// Can transition to UpstreamFailed
let upstream_failed = upstream_building.upstream_failed(
vec![PartitionRef {
r#ref: "data/alpha".to_string(),
}],
3000,
);
assert_eq!(upstream_failed.state.failed_at, 3000);
assert_eq!(upstream_failed.state.failed_upstream_refs.len(), 1);
assert_eq!(
upstream_failed.state.failed_upstream_refs[0].r#ref,
"data/alpha"
);
}
}

View file

@ -4,6 +4,10 @@ use crate::{EventSource, PartitionRef, WantCreateEventV1, WantDetail, WantStatus
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
use uuid::Uuid; use uuid::Uuid;
/// State: Want has just been created, state not yet determined by sensing partition states
#[derive(Debug, Clone)]
pub struct NewState {}
/// State: Want has been created and is ready to be scheduled /// State: Want has been created and is ready to be scheduled
#[derive(Debug, Clone)] #[derive(Debug, Clone)]
pub struct IdleState {} pub struct IdleState {}
@ -95,6 +99,7 @@ pub struct WantWithState<S> {
/// Wrapper enum for storing wants in collections /// Wrapper enum for storing wants in collections
#[derive(Debug, Clone)] #[derive(Debug, Clone)]
pub enum Want { pub enum Want {
New(WantWithState<NewState>),
Idle(WantWithState<IdleState>), Idle(WantWithState<IdleState>),
Building(WantWithState<BuildingState>), Building(WantWithState<BuildingState>),
UpstreamBuilding(WantWithState<UpstreamBuildingState>), UpstreamBuilding(WantWithState<UpstreamBuildingState>),
@ -108,6 +113,14 @@ pub enum Want {
/// is critical that these be treated with respect, not just summoned because it's convenient. /// is critical that these be treated with respect, not just summoned because it's convenient.
/// These should be created ephemerally from typestate objects via .get_ref() and used /// These should be created ephemerally from typestate objects via .get_ref() and used
/// immediately — never stored long-term, as partition state can change. /// immediately — never stored long-term, as partition state can change.
#[derive(Debug, Clone, Serialize, Deserialize, Default)]
pub struct NewWantId(pub String);
impl WantWithState<NewState> {
pub fn get_id(&self) -> NewWantId {
NewWantId(self.want.want_id.clone())
}
}
#[derive(Debug, Clone, Serialize, Deserialize, Default)] #[derive(Debug, Clone, Serialize, Deserialize, Default)]
pub struct IdleWantId(pub String); pub struct IdleWantId(pub String);
impl WantWithState<IdleState> { impl WantWithState<IdleState> {
@ -164,8 +177,8 @@ impl WantWithState<CanceledState> {
} }
} }
// From impl for creating want from event // From impl for creating want from event - creates in New state for sensing
impl From<WantCreateEventV1> for WantWithState<IdleState> { impl From<WantCreateEventV1> for WantWithState<NewState> {
fn from(event: WantCreateEventV1) -> Self { fn from(event: WantCreateEventV1) -> Self {
WantWithState { WantWithState {
want: WantInfo { want: WantInfo {
@ -178,9 +191,94 @@ impl From<WantCreateEventV1> for WantWithState<IdleState> {
comment: event.comment, comment: event.comment,
last_updated_at: current_timestamp(), last_updated_at: current_timestamp(),
}, },
state: NewState {},
}
}
}
// Type-safe transition methods for NewState
impl WantWithState<NewState> {
/// Transition from New to Idle when partitions don't exist or are ready to schedule
pub fn to_idle(self) -> WantWithState<IdleState> {
WantWithState {
want: self.want.updated_timestamp(),
state: IdleState {}, state: IdleState {},
} }
} }
/// Transition from New to Building when partitions are currently being built
pub fn to_building(self, started_at: u64) -> WantWithState<BuildingState> {
WantWithState {
want: self.want.updated_timestamp(),
state: BuildingState { started_at },
}
}
/// Transition from New to Successful when all partitions are already Live
pub fn to_successful(self, completed_at: u64) -> WantWithState<SuccessfulState> {
WantWithState {
want: self.want.updated_timestamp(),
state: SuccessfulState { completed_at },
}
}
/// Transition from New to Failed when any partition has failed
pub fn to_failed(
self,
failed_partition_refs: Vec<PartitionRef>,
reason: String,
) -> WantWithState<FailedState> {
WantWithState {
want: self.want.updated_timestamp(),
state: FailedState {
failed_at: current_timestamp(),
failed_partition_refs,
failure_reason: reason,
},
}
}
/// Transition from New to UpstreamBuilding when partitions are waiting for upstream deps
pub fn to_upstream_building(
self,
upstream_want_ids: Vec<String>,
) -> WantWithState<UpstreamBuildingState> {
WantWithState {
want: self.want.updated_timestamp(),
state: UpstreamBuildingState { upstream_want_ids },
}
}
/// Transition from New to UpstreamFailed when upstream dependencies have failed
pub fn to_upstream_failed(
self,
failed_wants: Vec<String>,
) -> WantWithState<UpstreamFailedState> {
WantWithState {
want: self.want.updated_timestamp(),
state: UpstreamFailedState {
failed_at: current_timestamp(),
failed_wants,
},
}
}
/// Transition from New to Canceled when want is explicitly canceled
/// (Rarely used - wants are typically transitioned before cancel can arrive)
pub fn cancel(
self,
canceled_by: Option<EventSource>,
comment: Option<String>,
) -> WantWithState<CanceledState> {
WantWithState {
want: self.want.updated_timestamp(),
state: CanceledState {
canceled_at: current_timestamp(),
canceled_by,
comment,
},
}
}
} }
// Type-safe transition methods for IdleState // Type-safe transition methods for IdleState
@ -413,6 +511,7 @@ impl Want {
pub fn want(&self) -> &WantInfo { pub fn want(&self) -> &WantInfo {
match self { match self {
Want::New(w) => &w.want,
Want::Idle(w) => &w.want, Want::Idle(w) => &w.want,
Want::Building(w) => &w.want, Want::Building(w) => &w.want,
Want::UpstreamBuilding(w) => &w.want, Want::UpstreamBuilding(w) => &w.want,
@ -436,6 +535,7 @@ impl Want {
comment: self.want().comment.clone(), comment: self.want().comment.clone(),
last_updated_timestamp: self.want().last_updated_at, last_updated_timestamp: self.want().last_updated_at,
status: match self { status: match self {
Want::New(_) => Some(WantStatusCode::WantNew.into()),
Want::Idle(_) => Some(WantStatusCode::WantIdle.into()), Want::Idle(_) => Some(WantStatusCode::WantIdle.into()),
Want::Building(_) => Some(WantStatusCode::WantBuilding.into()), Want::Building(_) => Some(WantStatusCode::WantBuilding.into()),
Want::UpstreamBuilding(_) => Some(WantStatusCode::WantUpstreamBuilding.into()), Want::UpstreamBuilding(_) => Some(WantStatusCode::WantUpstreamBuilding.into()),