diff --git a/docs/docs/hydroflow_plus/aggregations.mdx b/docs/docs/hydroflow_plus/aggregations.mdx index ae61ad9611ea..2951c1d097f5 100644 --- a/docs/docs/hydroflow_plus/aggregations.mdx +++ b/docs/docs/hydroflow_plus/aggregations.mdx @@ -21,7 +21,7 @@ To specify this **window**, Hydroflow+ offers two operators, `tick_batch()` and For example, consider a pipelined aggregation across two processes. We can sum up elements on the first process in a batched manner using `tick_batch()`, then sum up the results on the second process in an unbounded manner using `all_ticks()`: ```rust -let root_stream = flow.source_stream(&process, q!(1..=10)); +let root_stream = process.source_stream(q!(1..=10)); root_stream .tick_batch() .fold(q!(|| 0), q!(|acc, x| *acc += x)) diff --git a/docs/docs/hydroflow_plus/clusters.mdx b/docs/docs/hydroflow_plus/clusters.mdx index bfe1e0cac258..63380857ca5b 100644 --- a/docs/docs/hydroflow_plus/clusters.mdx +++ b/docs/docs/hydroflow_plus/clusters.mdx @@ -23,7 +23,7 @@ This API follows the same pattern as processes, where a cluster spec represents Instantiating streams on clusters uses the same APIs as streams: `source_iter` and `source_stream` are both available. But when using these APIs, the root streams will be instantiated on _all_ instances in the cluster. ```rust -let stream = flow.source_iter(&cluster, q!(vec![1, 2, 3])); +let stream = cluster.source_iter(q!(vec![1, 2, 3])); stream.for_each(q!(|x| println!("{}", x))) // will print 1, 2, 3 on **each** instance @@ -36,7 +36,7 @@ Elements in a cluster are identified by a **cluster ID** (a `u32`). To get the I This can then be passed into `source_iter` to load the IDs into the graph. ```rust -let stream = flow.source_iter(&process, cluster.members()).cloned(); +let stream = process.source_iter(cluster.members()).cloned(); ``` ### One-to-Many @@ -45,20 +45,20 @@ When sending data from a process to a cluster, the source must be a stream of tu This is useful for partitioning data across instances. For example, we can partition a stream of elements in a round-robin fashion by using `enumerate` to add a sequence number to each element, then using `send_bincode` to send each element to the instance with the matching sequence number: ```rust let cluster_ids = cluster.members(); -let stream = flow.source_iter(&process, q!(vec![123, 456, 789])) +let stream = process.source_iter(q!(vec![123, 456, 789])) .enumerate() .map(q!(|(i, x)| ( i % cluster_ids.len() as u32, x ))) - .send_bincode(cluster); + .send_bincode(&cluster); ``` To broadcast data to all instances in a cluster, use `broadcast_{bincode,bytes}`, which acts as a shortcut for the cross product. ```rust -let stream = flow.source_iter(&process, q!(vec![123, 456, 789])) - .broadcast_bincode(cluster); +let stream = process.source_iter(q!(vec![123, 456, 789])) + .broadcast_bincode(&cluster); ``` ### Many-to-One @@ -66,15 +66,15 @@ In the other direction, sending data from a cluster to a process, we have a stre This is useful for aggregating data from multiple instances into a single stream. For example, we can use `send_bincode` to send data from all instances to a single process, and then print them all out: ```rust -let stream = flow.source_iter(&cluster, q!(vec![123, 456, 789])) - .send_bincode(process) +let stream = cluster.source_iter(q!(vec![123, 456, 789])) + .send_bincode(&process) .for_each(q!(|(id, x)| println!("{}: {}", id, x))); ``` If you don't care which instance sent the data, you can use `send_{bincode,bytes}_interleaved`, where the recipient receives a stream of `T` elements, but the elements received from different instances will be interleaved. ```rust -let stream = flow.source_iter(&cluster, q!(vec![123, 456, 789])) - .send_bincode_interleaved(process) +let stream = cluster.source_iter(q!(vec![123, 456, 789])) + .send_bincode_interleaved(&process) .for_each(q!(|x| println!("{}", x))); ``` @@ -83,7 +83,7 @@ Finally, when sending data from one cluster to another (or to itself as in distr We can use the same shortcuts as before. For example, we can use `broadcast_bincode_interleaved` to send data from all instances in a cluster to all instances in another cluster, and then print them all out: ```rust -let stream = flow.source_iter(&cluster1, q!(vec![123, 456, 789])) - .broadcast_bincode_interleaved(cluster2) +let stream = cluster1.source_iter(q!(vec![123, 456, 789])) + .broadcast_bincode_interleaved(&cluster2) .for_each(q!(|x| println!("{}", x))); ``` diff --git a/docs/docs/hydroflow_plus/cycles.mdx b/docs/docs/hydroflow_plus/cycles.mdx index 2682cc01dd8d..21d1b78fcf52 100644 --- a/docs/docs/hydroflow_plus/cycles.mdx +++ b/docs/docs/hydroflow_plus/cycles.mdx @@ -10,7 +10,7 @@ Because streams are represented as values when constructing a Hydroflow+ graph, We can create a cycle by using the `cycle` method on flow with a process or cluster. This returns a tuple of two values: a `HfCycle` value that can be used to complete the cycle later and the placeholder stream. ```rust -let (complete_cycle, cycle_placeholder) = flow.cycle(&process); +let (complete_cycle, cycle_placeholder) = process.cycle(); ``` For example, consider the classic graph reachability problem, which computes the nodes reachable from a given set of roots in a directed graph. This can be modeled as an iterative fixpoint computation where we start with the roots, then repeatedly add the children of each node to the set of reachable nodes until we reach a fixpoint. @@ -18,10 +18,10 @@ For example, consider the classic graph reachability problem, which computes the In Hydroflow+, we can implement this using cycles: ```rust -let roots = flow.source_stream(&process, roots); -let edges = flow.source_stream(&process, edges); +let roots = process.source_stream(roots); +let edges = process.source_stream(edges); -let (complete_reached_nodes, reached_nodes) = flow.cycle(&process); +let (complete_reached_nodes, reached_nodes) = process.cycle(); let reach_iteration = roots .union(&reached_nodes) diff --git a/docs/docs/hydroflow_plus/process_streams.mdx b/docs/docs/hydroflow_plus/process_streams.mdx index e99c6b49971e..75559052b43a 100644 --- a/docs/docs/hydroflow_plus/process_streams.mdx +++ b/docs/docs/hydroflow_plus/process_streams.mdx @@ -41,7 +41,7 @@ Root streams are created using methods available on an an instantiated process. To create a stream from a Rust iterator, use `source_iter`. This is useful for loading static data into the graph. Each element of the iterator will be emitted _exactly once_ in the _first tick_ of execution (see [Aggregations and Ticks](./aggregations.mdx)). ```rust -let stream = flow.source_iter(&process, q!(vec![1, 2, 3])); +let stream = process.source_iter(q!(vec![1, 2, 3])); ``` #### `source_stream` @@ -52,7 +52,7 @@ pub fn my_flow<'a, D: Deploy<'a>>( ..., my_stream: RuntimeData> ) { - let stream = flow.source_stream(&process, my_stream); + let stream = process.source_stream(my_stream); ... } ``` @@ -66,13 +66,13 @@ If sending a type that supports serialization using `serde`, use `send_bincode`, let process0 = flow.process(process_spec); let process1 = flow.process(process_spec); -let stream0 = flow.source_iter(&process0, ...); -let stream1 = stream0.send_bincode(process1); +let stream0 = process0.source_iter(...); +let stream1 = stream0.send_bincode(&process1); ``` To use custom serializers, you can use the `send_bytes` method to send a stream of `Bytes` values. ```rust -let stream0 = flow.source_iter(&process0, ...); -let stream1 = stream0.send_bytes(process1); +let stream0 = process0.source_iter(...); +let stream1 = stream0.send_bytes(&process1); ``` diff --git a/docs/docs/hydroflow_plus/quickstart/clusters.mdx b/docs/docs/hydroflow_plus/quickstart/clusters.mdx index 240b8fd5e153..4155c13e68a0 100644 --- a/docs/docs/hydroflow_plus/quickstart/clusters.mdx +++ b/docs/docs/hydroflow_plus/quickstart/clusters.mdx @@ -44,7 +44,7 @@ pub fn broadcast( When sending data between individual processes, we used the `send_bincode` operator. When sending data from a process to a cluster, we can use the `broadcast_bincode` operator instead. ```rust -let data = flow.source_iter(&leader, q!(0..10)); +let data = leader.source_iter(q!(0..10)); data .broadcast_bincode(&workers) .for_each(q!(|n| println!("{}", n))); diff --git a/hydroflow_plus/src/builder/built.rs b/hydroflow_plus/src/builder/built.rs index d0d749965507..fd8659a0c12a 100644 --- a/hydroflow_plus/src/builder/built.rs +++ b/hydroflow_plus/src/builder/built.rs @@ -10,7 +10,7 @@ use crate::location::{Cluster, ExternalProcess, Process}; use crate::HfCompiled; pub struct BuiltFlow<'a> { - pub(super) ir: Vec>, + pub(super) ir: Vec, pub(super) processes: Vec, pub(super) clusters: Vec, pub(super) used: bool, @@ -27,13 +27,13 @@ impl<'a> Drop for BuiltFlow<'a> { } impl<'a> BuiltFlow<'a> { - pub fn ir(&self) -> &Vec> { + pub fn ir(&self) -> &Vec { &self.ir } pub fn optimize_with( mut self, - f: impl FnOnce(Vec>) -> Vec>, + f: impl FnOnce(Vec) -> Vec, ) -> BuiltFlow<'a> { self.used = true; BuiltFlow { diff --git a/hydroflow_plus/src/builder/deploy.rs b/hydroflow_plus/src/builder/deploy.rs index 1c165ddd00f0..bc70ab3a2aa9 100644 --- a/hydroflow_plus/src/builder/deploy.rs +++ b/hydroflow_plus/src/builder/deploy.rs @@ -20,7 +20,7 @@ use crate::location::{ use crate::{Cluster, ClusterSpec, Deploy, HfCompiled, Process, ProcessSpec}; pub struct DeployFlow<'a, D: LocalDeploy<'a>> { - pub(super) ir: Vec>, + pub(super) ir: Vec, pub(super) nodes: HashMap, pub(super) externals: HashMap, pub(super) clusters: HashMap, @@ -69,7 +69,7 @@ impl<'a, D: Deploy<'a>> DeployFlow<'a, D> { self.used = true; let mut seen_tees: HashMap<_, _> = HashMap::new(); - let mut ir_leaves_networked: Vec = std::mem::take(&mut self.ir) + let mut flow_state_networked: Vec = std::mem::take(&mut self.ir) .into_iter() .map(|leaf| { leaf.compile_network::( @@ -85,7 +85,7 @@ impl<'a, D: Deploy<'a>> DeployFlow<'a, D> { let extra_stmts = self.extra_stmts(env); HfCompiled { - hydroflow_ir: build_inner(&mut ir_leaves_networked), + hydroflow_ir: build_inner(&mut flow_state_networked), extra_stmts, _phantom: PhantomData, } @@ -132,7 +132,7 @@ impl<'a, D: Deploy<'a, CompileEnv = ()>> DeployFlow<'a, D> { self.used = true; let mut seen_tees_instantiate: HashMap<_, _> = HashMap::new(); - let mut ir_leaves_networked: Vec = std::mem::take(&mut self.ir) + let mut flow_state_networked: Vec = std::mem::take(&mut self.ir) .into_iter() .map(|leaf| { leaf.compile_network::( @@ -145,7 +145,7 @@ impl<'a, D: Deploy<'a, CompileEnv = ()>> DeployFlow<'a, D> { }) .collect(); - let mut compiled = build_inner(&mut ir_leaves_networked); + let mut compiled = build_inner(&mut flow_state_networked); let mut extra_stmts = self.extra_stmts(&()); let mut meta = D::Meta::default(); @@ -201,7 +201,7 @@ impl<'a, D: Deploy<'a, CompileEnv = ()>> DeployFlow<'a, D> { } let mut seen_tees_connect = HashMap::new(); - for leaf in ir_leaves_networked { + for leaf in flow_state_networked { leaf.connect_network(&mut seen_tees_connect); } @@ -229,7 +229,7 @@ impl<'a, D: Deploy<'a>> DeployResult<'a, D> { self.processes.get(&id).unwrap() } - pub fn get_cluster(&self, c: &Cluster) -> &D::Cluster { + pub fn get_cluster(&self, c: &Cluster<'a, C>) -> &D::Cluster { let id = match c.id() { LocationId::Cluster(id) => id, _ => panic!("Cluster ID expected"), diff --git a/hydroflow_plus/src/builder/mod.rs b/hydroflow_plus/src/builder/mod.rs index 16dbe420f3f1..3950a08a183e 100644 --- a/hydroflow_plus/src/builder/mod.rs +++ b/hydroflow_plus/src/builder/mod.rs @@ -2,39 +2,34 @@ use std::cell::RefCell; use std::collections::HashMap; use std::marker::PhantomData; use std::rc::Rc; -use std::time::Duration; -use hydroflow::bytes::Bytes; -use hydroflow::futures::stream::Stream as FuturesStream; -use hydroflow::{tokio, tokio_stream}; use internal::TokenStream; use proc_macro2::Span; use quote::quote; use runtime_support::FreeVariable; -use serde::de::DeserializeOwned; -use serde::Serialize; use stageleft::*; -use crate::cycle::{CycleCollection, CycleCollectionWithInitial}; -use crate::ir::{HfPlusLeaf, HfPlusNode, HfPlusSource}; -use crate::location::{ - Cluster, ExternalBincodeSink, ExternalBytesPort, ExternalProcess, Location, LocationId, Process, -}; -use crate::stream::{Bounded, NoTick, Tick, Unbounded}; -use crate::{HfCycle, Optional, RuntimeContext, Singleton, Stream}; +use crate::ir::HfPlusLeaf; +use crate::location::{Cluster, ExternalProcess, Process}; +use crate::RuntimeContext; pub mod built; pub mod deploy; -/// Tracks the leaves of the dataflow IR. This is referenced by -/// `Stream` and `HfCycle` to build the IR. The inner option will -/// be set to `None` when this builder is finalized. -/// -/// The second `usize` is used to generate unique identifiers for external -/// outputs of the dataflow. -pub type FlowLeaves<'a> = Rc>>, usize)>>; +pub struct FlowStateInner { + /// Tracks the leaves of the dataflow IR. This is referenced by + /// `Stream` and `HfCycle` to build the IR. The inner option will + /// be set to `None` when this builder is finalized. + pub(crate) leaves: Option>, -pub type ExternalPortCounter = Rc>; + /// Counter for generating unique external output identifiers. + pub(crate) next_external_out: usize, + + /// Counters for generating identifiers for cycles. + pub(crate) cycle_counts: HashMap, +} + +pub type FlowState = Rc>; #[derive(Copy, Clone)] pub struct ClusterIds<'a> { @@ -79,10 +74,9 @@ impl<'a> FreeVariable for ClusterSelfId<'a> { impl<'a> Quoted<'a, u32> for ClusterSelfId<'a> {} pub struct FlowBuilder<'a> { - ir_leaves: FlowLeaves<'a>, + flow_state: FlowState, nodes: RefCell>, clusters: RefCell>, - cycle_ids: RefCell>, next_node_id: RefCell, @@ -118,10 +112,13 @@ impl<'a> FlowBuilder<'a> { )] pub fn new() -> FlowBuilder<'a> { FlowBuilder { - ir_leaves: Rc::new(RefCell::new((Some(Vec::new()), 0))), + flow_state: Rc::new(RefCell::new(FlowStateInner { + leaves: Some(vec![]), + next_external_out: 0, + cycle_counts: HashMap::new(), + })), nodes: RefCell::new(vec![]), clusters: RefCell::new(vec![]), - cycle_ids: RefCell::new(HashMap::new()), next_node_id: RefCell::new(0), finalized: false, _phantom: PhantomData, @@ -132,7 +129,7 @@ impl<'a> FlowBuilder<'a> { self.finalized = true; built::BuiltFlow { - ir: self.ir_leaves.borrow_mut().0.take().unwrap(), + ir: self.flow_state.borrow_mut().leaves.take().unwrap(), processes: self.nodes.replace(vec![]), clusters: self.clusters.replace(vec![]), used: false, @@ -146,16 +143,16 @@ impl<'a> FlowBuilder<'a> { pub fn optimize_with( self, - f: impl FnOnce(Vec>) -> Vec>, + f: impl FnOnce(Vec) -> Vec, ) -> built::BuiltFlow<'a> { self.finalize().optimize_with(f) } - pub fn ir_leaves(&self) -> &FlowLeaves<'a> { - &self.ir_leaves + pub fn flow_state(&self) -> &FlowState { + &self.flow_state } - pub fn process

(&self) -> Process

{ + pub fn process

(&self) -> Process<'a, P> { let mut next_node_id = self.next_node_id.borrow_mut(); let id = *next_node_id; *next_node_id += 1; @@ -164,11 +161,12 @@ impl<'a> FlowBuilder<'a> { Process { id, + flow_state: self.flow_state().clone(), _phantom: PhantomData, } } - pub fn external_process

(&self) -> ExternalProcess

{ + pub fn external_process

(&self) -> ExternalProcess<'a, P> { let mut next_node_id = self.next_node_id.borrow_mut(); let id = *next_node_id; *next_node_id += 1; @@ -177,11 +175,12 @@ impl<'a> FlowBuilder<'a> { ExternalProcess { id, + flow_state: self.flow_state().clone(), _phantom: PhantomData, } } - pub fn cluster(&self) -> Cluster { + pub fn cluster(&self) -> Cluster<'a, C> { let mut next_node_id = self.next_node_id.borrow_mut(); let id = *next_node_id; *next_node_id += 1; @@ -190,6 +189,7 @@ impl<'a> FlowBuilder<'a> { Cluster { id, + flow_state: self.flow_state().clone(), _phantom: PhantomData, } } @@ -199,301 +199,4 @@ impl<'a> FlowBuilder<'a> { _phantom: PhantomData, } } - - pub fn spin(&self, on: &L) -> Stream<'a, (), Unbounded, NoTick, L> { - Stream::new( - on.id(), - self.ir_leaves().clone(), - HfPlusNode::Persist(Box::new(HfPlusNode::Source { - source: HfPlusSource::Spin(), - location_kind: on.id(), - })), - ) - } - - pub fn spin_batch( - &self, - on: &L, - batch_size: impl Quoted<'a, usize> + Copy + 'a, - ) -> Stream<'a, (), Bounded, Tick, L> { - self.spin(on) - .flat_map(q!(move |_| 0..batch_size)) - .map(q!(|_| ())) - .tick_batch() - } - - pub fn source_external_bytes( - &self, - from: &ExternalProcess

, - to: &L, - ) -> (ExternalBytesPort, Stream<'a, Bytes, Unbounded, NoTick, L>) { - let next_external_port_id = { - let mut ir_leaves = self.ir_leaves.borrow_mut(); - let id = ir_leaves.1; - ir_leaves.1 += 1; - id - }; - - ( - ExternalBytesPort { - process_id: from.id, - port_id: next_external_port_id, - }, - Stream::new( - to.id(), - self.ir_leaves().clone(), - HfPlusNode::Persist(Box::new(HfPlusNode::Network { - from_location: LocationId::ExternalProcess(from.id), - from_key: Some(next_external_port_id), - to_location: to.id(), - to_key: None, - serialize_pipeline: None, - instantiate_fn: crate::ir::DebugInstantiate::Building(), - deserialize_pipeline: Some(syn::parse_quote!(map(|b| b.unwrap().freeze()))), - input: Box::new(HfPlusNode::Source { - source: HfPlusSource::ExternalNetwork(), - location_kind: LocationId::ExternalProcess(from.id), - }), - })), - ), - ) - } - - pub fn source_external_bincode( - &self, - from: &ExternalProcess

, - to: &L, - ) -> (ExternalBincodeSink, Stream<'a, T, Unbounded, NoTick, L>) { - let next_external_port_id = { - let mut ir_leaves = self.ir_leaves.borrow_mut(); - let id = ir_leaves.1; - ir_leaves.1 += 1; - id - }; - - ( - ExternalBincodeSink { - process_id: from.id, - port_id: next_external_port_id, - _phantom: PhantomData, - }, - Stream::new( - to.id(), - self.ir_leaves().clone(), - HfPlusNode::Persist(Box::new(HfPlusNode::Network { - from_location: LocationId::ExternalProcess(from.id), - from_key: Some(next_external_port_id), - to_location: to.id(), - to_key: None, - serialize_pipeline: None, - instantiate_fn: crate::ir::DebugInstantiate::Building(), - deserialize_pipeline: Some(crate::stream::deserialize_bincode::(false)), - input: Box::new(HfPlusNode::Source { - source: HfPlusSource::ExternalNetwork(), - location_kind: LocationId::ExternalProcess(from.id), - }), - })), - ), - ) - } - - pub fn source_stream + Unpin, L: Location>( - &self, - on: &L, - e: impl Quoted<'a, E>, - ) -> Stream<'a, T, Unbounded, NoTick, L> { - let e = e.splice_untyped(); - - Stream::new( - on.id(), - self.ir_leaves().clone(), - HfPlusNode::Persist(Box::new(HfPlusNode::Source { - source: HfPlusSource::Stream(e.into()), - location_kind: on.id(), - })), - ) - } - - pub fn source_iter, L: Location>( - &self, - on: &L, - e: impl Quoted<'a, E>, - ) -> Stream<'a, T, Bounded, NoTick, L> { - let e = e.splice_untyped(); - - Stream::new( - on.id(), - self.ir_leaves().clone(), - HfPlusNode::Persist(Box::new(HfPlusNode::Source { - source: HfPlusSource::Iter(e.into()), - location_kind: on.id(), - })), - ) - } - - pub fn singleton( - &self, - on: &L, - e: impl Quoted<'a, T>, - ) -> Singleton<'a, T, Bounded, NoTick, L> { - let e_arr = q!([e]); - let e = e_arr.splice_untyped(); - - // we do a double persist here because if the singleton shows up on every tick, - // we first persist the source so that we store that value and then persist again - // so that it grows every tick - Singleton::new( - on.id(), - self.ir_leaves().clone(), - HfPlusNode::Persist(Box::new(HfPlusNode::Persist(Box::new( - HfPlusNode::Source { - source: HfPlusSource::Iter(e.into()), - location_kind: on.id(), - }, - )))), - ) - } - - pub fn singleton_first_tick( - &self, - on: &L, - e: impl Quoted<'a, T>, - ) -> Optional<'a, T, Bounded, Tick, L> { - let e_arr = q!([e]); - let e = e_arr.splice_untyped(); - - Optional::new( - on.id(), - self.ir_leaves().clone(), - HfPlusNode::Source { - source: HfPlusSource::Iter(e.into()), - location_kind: on.id(), - }, - ) - } - - pub fn source_interval( - &self, - on: &L, - interval: impl Quoted<'a, Duration> + Copy + 'a, - ) -> Optional<'a, (), Unbounded, NoTick, L> { - let interval = interval.splice_untyped(); - - Optional::new( - on.id(), - self.ir_leaves().clone(), - HfPlusNode::Persist(Box::new(HfPlusNode::Source { - source: HfPlusSource::Interval(interval.into()), - location_kind: on.id(), - })), - ) - } - - pub fn source_interval_delayed( - &self, - on: &L, - delay: impl Quoted<'a, Duration> + Copy + 'a, - interval: impl Quoted<'a, Duration> + Copy + 'a, - ) -> Optional<'a, tokio::time::Instant, Unbounded, NoTick, L> { - self.source_stream( - on, - q!(tokio_stream::wrappers::IntervalStream::new( - tokio::time::interval_at(tokio::time::Instant::now() + delay, interval) - )), - ) - .tick_batch() - .first() - .latest() - } - - pub fn tick_cycle>( - &self, - on: &S::Location, - ) -> (HfCycle<'a, Tick, S>, S) { - let next_id = { - let on_id = match on.id() { - LocationId::Process(id) => id, - LocationId::Cluster(id) => id, - LocationId::ExternalProcess(_) => panic!(), - }; - - let mut cycle_ids = self.cycle_ids.borrow_mut(); - let next_id_entry = cycle_ids.entry(on_id).or_default(); - - let id = *next_id_entry; - *next_id_entry += 1; - id - }; - - let ident = syn::Ident::new(&format!("cycle_{}", next_id), Span::call_site()); - - ( - HfCycle { - ident: ident.clone(), - _phantom: PhantomData, - }, - S::create_source(ident, self.ir_leaves.clone(), on.id()), - ) - } - - pub fn cycle>( - &self, - on: &S::Location, - ) -> (HfCycle<'a, NoTick, S>, S) { - let next_id = { - let on_id = match on.id() { - LocationId::Process(id) => id, - LocationId::Cluster(id) => id, - LocationId::ExternalProcess(_) => panic!(), - }; - - let mut cycle_ids = self.cycle_ids.borrow_mut(); - let next_id_entry = cycle_ids.entry(on_id).or_default(); - - let id = *next_id_entry; - *next_id_entry += 1; - id - }; - - let ident = syn::Ident::new(&format!("cycle_{}", next_id), Span::call_site()); - - ( - HfCycle { - ident: ident.clone(), - _phantom: PhantomData, - }, - S::create_source(ident, self.ir_leaves.clone(), on.id()), - ) - } - - pub fn tick_cycle_with_initial>( - &self, - on: &S::Location, - initial: S, - ) -> (HfCycle<'a, Tick, S>, S) { - let next_id = { - let on_id = match on.id() { - LocationId::Process(id) => id, - LocationId::Cluster(id) => id, - LocationId::ExternalProcess(_) => panic!(), - }; - - let mut cycle_ids = self.cycle_ids.borrow_mut(); - let next_id_entry = cycle_ids.entry(on_id).or_default(); - - let id = *next_id_entry; - *next_id_entry += 1; - id - }; - - let ident = syn::Ident::new(&format!("cycle_{}", next_id), Span::call_site()); - - ( - HfCycle { - ident: ident.clone(), - _phantom: PhantomData, - }, - S::create_source(ident, self.ir_leaves.clone(), initial, on.id()), - ) - } } diff --git a/hydroflow_plus/src/cycle.rs b/hydroflow_plus/src/cycle.rs index a4ba3384336c..d8a240e56548 100644 --- a/hydroflow_plus/src/cycle.rs +++ b/hydroflow_plus/src/cycle.rs @@ -1,6 +1,6 @@ use std::marker::PhantomData; -use crate::builder::FlowLeaves; +use crate::builder::FlowState; use crate::location::{Location, LocationId}; use crate::{NoTick, Tick}; @@ -9,17 +9,17 @@ pub trait CycleComplete<'a, T> { } pub trait CycleCollection<'a, T>: CycleComplete<'a, T> { - type Location: Location; + type Location: Location<'a>; - fn create_source(ident: syn::Ident, ir_leaves: FlowLeaves<'a>, l: LocationId) -> Self; + fn create_source(ident: syn::Ident, flow_state: FlowState, l: LocationId) -> Self; } pub trait CycleCollectionWithInitial<'a, T>: CycleComplete<'a, T> { - type Location: Location; + type Location: Location<'a>; fn create_source( ident: syn::Ident, - ir_leaves: FlowLeaves<'a>, + flow_state: FlowState, initial: Self, l: LocationId, ) -> Self; diff --git a/hydroflow_plus/src/deploy/deploy_graph.rs b/hydroflow_plus/src/deploy/deploy_graph.rs index 7118f95dc16d..e7fecc6e7167 100644 --- a/hydroflow_plus/src/deploy/deploy_graph.rs +++ b/hydroflow_plus/src/deploy/deploy_graph.rs @@ -77,22 +77,29 @@ impl<'a> Deploy<'a> for HydroDeploy { p1_port: &Self::Port, p2: &Self::Process, p2_port: &Self::Port, - ) { - let self_underlying_borrow = p1.underlying.borrow(); - let self_underlying = self_underlying_borrow.as_ref().unwrap(); - let source_port = self_underlying - .try_read() - .unwrap() - .get_port(p1_port.clone(), self_underlying); + ) -> Box { + let p1 = p1.clone(); + let p1_port = p1_port.clone(); + let p2 = p2.clone(); + let p2_port = p2_port.clone(); + + Box::new(move || { + let self_underlying_borrow = p1.underlying.borrow(); + let self_underlying = self_underlying_borrow.as_ref().unwrap(); + let source_port = self_underlying + .try_read() + .unwrap() + .get_port(p1_port.clone(), self_underlying); - let other_underlying_borrow = p2.underlying.borrow(); - let other_underlying = other_underlying_borrow.as_ref().unwrap(); - let recipient_port = other_underlying - .try_read() - .unwrap() - .get_port(p2_port.clone(), other_underlying); + let other_underlying_borrow = p2.underlying.borrow(); + let other_underlying = other_underlying_borrow.as_ref().unwrap(); + let recipient_port = other_underlying + .try_read() + .unwrap() + .get_port(p2_port.clone(), other_underlying); - source_port.send_to(&recipient_port); + source_port.send_to(&recipient_port) + }) } fn o2m_sink_source( @@ -116,32 +123,39 @@ impl<'a> Deploy<'a> for HydroDeploy { p1_port: &Self::Port, c2: &Self::Cluster, c2_port: &Self::Port, - ) { - let self_underlying_borrow = p1.underlying.borrow(); - let self_underlying = self_underlying_borrow.as_ref().unwrap(); - let source_port = self_underlying - .try_read() - .unwrap() - .get_port(p1_port.clone(), self_underlying); - - let recipient_port = DemuxSink { - demux: c2 - .members - .borrow() - .iter() - .enumerate() - .map(|(id, c)| { - let n = c.underlying.try_read().unwrap(); - ( - id as u32, - Arc::new(n.get_port(c2_port.clone(), &c.underlying)) - as Arc, - ) - }) - .collect(), - }; + ) -> Box { + let p1 = p1.clone(); + let p1_port = p1_port.clone(); + let c2 = c2.clone(); + let c2_port = c2_port.clone(); + + Box::new(move || { + let self_underlying_borrow = p1.underlying.borrow(); + let self_underlying = self_underlying_borrow.as_ref().unwrap(); + let source_port = self_underlying + .try_read() + .unwrap() + .get_port(p1_port.clone(), self_underlying); - source_port.send_to(&recipient_port); + let recipient_port = DemuxSink { + demux: c2 + .members + .borrow() + .iter() + .enumerate() + .map(|(id, c)| { + let n = c.underlying.try_read().unwrap(); + ( + id as u32, + Arc::new(n.get_port(c2_port.clone(), &c.underlying)) + as Arc, + ) + }) + .collect(), + }; + + source_port.send_to(&recipient_port) + }) } fn m2o_sink_source( @@ -165,28 +179,35 @@ impl<'a> Deploy<'a> for HydroDeploy { c1_port: &Self::Port, p2: &Self::Process, p2_port: &Self::Port, - ) { - let other_underlying_borrow = p2.underlying.borrow(); - let other_underlying = other_underlying_borrow.as_ref().unwrap(); - let recipient_port = other_underlying - .try_read() - .unwrap() - .get_port(p2_port.clone(), other_underlying) - .merge(); - - for (i, node) in c1.members.borrow().iter().enumerate() { - let source_port = node - .underlying + ) -> Box { + let c1 = c1.clone(); + let c1_port = c1_port.clone(); + let p2 = p2.clone(); + let p2_port = p2_port.clone(); + + Box::new(move || { + let other_underlying_borrow = p2.underlying.borrow(); + let other_underlying = other_underlying_borrow.as_ref().unwrap(); + let recipient_port = other_underlying .try_read() .unwrap() - .get_port(c1_port.clone(), &node.underlying); - - TaggedSource { - source: Arc::new(source_port), - tag: i as u32, + .get_port(p2_port.clone(), other_underlying) + .merge(); + + for (i, node) in c1.members.borrow().iter().enumerate() { + let source_port = node + .underlying + .try_read() + .unwrap() + .get_port(c1_port.clone(), &node.underlying); + + TaggedSource { + source: Arc::new(source_port), + tag: i as u32, + } + .send_to(&recipient_port); } - .send_to(&recipient_port); - } + }) } fn m2m_sink_source( @@ -210,37 +231,44 @@ impl<'a> Deploy<'a> for HydroDeploy { c1_port: &Self::Port, c2: &Self::Cluster, c2_port: &Self::Port, - ) { - for (i, sender) in c1.members.borrow().iter().enumerate() { - let source_port = sender - .underlying - .try_read() - .unwrap() - .get_port(c1_port.clone(), &sender.underlying); - - let recipient_port = DemuxSink { - demux: c2 - .members - .borrow() - .iter() - .enumerate() - .map(|(id, c)| { - let n = c.underlying.try_read().unwrap(); - ( - id as u32, - Arc::new(n.get_port(c2_port.clone(), &c.underlying).merge()) - as Arc, - ) - }) - .collect(), - }; + ) -> Box { + let c1 = c1.clone(); + let c1_port = c1_port.clone(); + let c2 = c2.clone(); + let c2_port = c2_port.clone(); + + Box::new(move || { + for (i, sender) in c1.members.borrow().iter().enumerate() { + let source_port = sender + .underlying + .try_read() + .unwrap() + .get_port(c1_port.clone(), &sender.underlying); + + let recipient_port = DemuxSink { + demux: c2 + .members + .borrow() + .iter() + .enumerate() + .map(|(id, c)| { + let n = c.underlying.try_read().unwrap(); + ( + id as u32, + Arc::new(n.get_port(c2_port.clone(), &c.underlying).merge()) + as Arc, + ) + }) + .collect(), + }; - TaggedSource { - source: Arc::new(source_port), - tag: i as u32, + TaggedSource { + source: Arc::new(source_port), + tag: i as u32, + } + .send_to(&recipient_port); } - .send_to(&recipient_port); - } + }) } fn e2o_source( @@ -264,26 +292,33 @@ impl<'a> Deploy<'a> for HydroDeploy { p1_port: &Self::Port, p2: &Self::Process, p2_port: &Self::Port, - ) { - let self_underlying_borrow = p1.underlying.borrow(); - let self_underlying = self_underlying_borrow.as_ref().unwrap(); - let source_port = self_underlying - .try_read() - .unwrap() - .declare_client(self_underlying); + ) -> Box { + let p1 = p1.clone(); + let p1_port = p1_port.clone(); + let p2 = p2.clone(); + let p2_port = p2_port.clone(); + + Box::new(move || { + let self_underlying_borrow = p1.underlying.borrow(); + let self_underlying = self_underlying_borrow.as_ref().unwrap(); + let source_port = self_underlying + .try_read() + .unwrap() + .declare_client(self_underlying); - let other_underlying_borrow = p2.underlying.borrow(); - let other_underlying = other_underlying_borrow.as_ref().unwrap(); - let recipient_port = other_underlying - .try_read() - .unwrap() - .get_port(p2_port.clone(), other_underlying); + let other_underlying_borrow = p2.underlying.borrow(); + let other_underlying = other_underlying_borrow.as_ref().unwrap(); + let recipient_port = other_underlying + .try_read() + .unwrap() + .get_port(p2_port.clone(), other_underlying); - source_port.send_to(&recipient_port); + source_port.send_to(&recipient_port); - p1.client_ports - .borrow_mut() - .insert(p1_port.clone(), source_port); + p1.client_ports + .borrow_mut() + .insert(p1_port.clone(), source_port); + }) } fn o2e_sink( @@ -307,26 +342,33 @@ impl<'a> Deploy<'a> for HydroDeploy { p1_port: &Self::Port, p2: &Self::ExternalProcess, p2_port: &Self::Port, - ) { - let self_underlying_borrow = p1.underlying.borrow(); - let self_underlying = self_underlying_borrow.as_ref().unwrap(); - let source_port = self_underlying - .try_read() - .unwrap() - .get_port(p1_port.clone(), self_underlying); + ) -> Box { + let p1 = p1.clone(); + let p1_port = p1_port.clone(); + let p2 = p2.clone(); + let p2_port = p2_port.clone(); + + Box::new(move || { + let self_underlying_borrow = p1.underlying.borrow(); + let self_underlying = self_underlying_borrow.as_ref().unwrap(); + let source_port = self_underlying + .try_read() + .unwrap() + .get_port(p1_port.clone(), self_underlying); - let other_underlying_borrow = p2.underlying.borrow(); - let other_underlying = other_underlying_borrow.as_ref().unwrap(); - let recipient_port = other_underlying - .try_read() - .unwrap() - .declare_client(other_underlying); + let other_underlying_borrow = p2.underlying.borrow(); + let other_underlying = other_underlying_borrow.as_ref().unwrap(); + let recipient_port = other_underlying + .try_read() + .unwrap() + .declare_client(other_underlying); - source_port.send_to(&recipient_port); + source_port.send_to(&recipient_port); - p2.client_ports - .borrow_mut() - .insert(p2_port.clone(), recipient_port); + p2.client_ports + .borrow_mut() + .insert(p2_port.clone(), recipient_port); + }) } fn cluster_ids( diff --git a/hydroflow_plus/src/deploy/macro_runtime.rs b/hydroflow_plus/src/deploy/macro_runtime.rs index 28c8c0b057fe..4f4e395f45da 100644 --- a/hydroflow_plus/src/deploy/macro_runtime.rs +++ b/hydroflow_plus/src/deploy/macro_runtime.rs @@ -65,8 +65,8 @@ impl<'a> Deploy<'a> for DeployRuntime { _p1_port: &Self::Port, _p2: &Self::Process, _p2_port: &Self::Port, - ) { - panic!() + ) -> Box { + Box::new(|| panic!()) } fn o2m_sink_source( @@ -84,8 +84,8 @@ impl<'a> Deploy<'a> for DeployRuntime { _p1_port: &Self::Port, _c2: &Self::Cluster, _c2_port: &Self::Port, - ) { - panic!() + ) -> Box { + Box::new(|| panic!()) } fn m2o_sink_source( @@ -103,8 +103,8 @@ impl<'a> Deploy<'a> for DeployRuntime { _c1_port: &Self::Port, _p2: &Self::Process, _p2_port: &Self::Port, - ) { - panic!() + ) -> Box { + Box::new(|| panic!()) } fn m2m_sink_source( @@ -122,8 +122,8 @@ impl<'a> Deploy<'a> for DeployRuntime { _c1_port: &Self::Port, _c2: &Self::Cluster, _c2_port: &Self::Port, - ) { - panic!() + ) -> Box { + Box::new(|| panic!()) } fn e2o_source( @@ -141,7 +141,7 @@ impl<'a> Deploy<'a> for DeployRuntime { _p1_port: &Self::Port, _p2: &Self::Process, _p2_port: &Self::Port, - ) { + ) -> Box { panic!() } @@ -160,7 +160,7 @@ impl<'a> Deploy<'a> for DeployRuntime { _p1_port: &Self::Port, _p2: &Self::ExternalProcess, _p2_port: &Self::Port, - ) { + ) -> Box { panic!() } diff --git a/hydroflow_plus/src/deploy/mod.rs b/hydroflow_plus/src/deploy/mod.rs index 934c18ff9659..995f2f0f69cc 100644 --- a/hydroflow_plus/src/deploy/mod.rs +++ b/hydroflow_plus/src/deploy/mod.rs @@ -96,7 +96,7 @@ pub trait Deploy<'a> { p1_port: &Self::Port, p2: &Self::Process, p2_port: &Self::Port, - ); + ) -> Box; fn o2m_sink_source( compile_env: &Self::CompileEnv, @@ -110,7 +110,7 @@ pub trait Deploy<'a> { p1_port: &Self::Port, c2: &Self::Cluster, c2_port: &Self::Port, - ); + ) -> Box; fn m2o_sink_source( compile_env: &Self::CompileEnv, @@ -124,7 +124,7 @@ pub trait Deploy<'a> { c1_port: &Self::Port, p2: &Self::Process, p2_port: &Self::Port, - ); + ) -> Box; fn m2m_sink_source( compile_env: &Self::CompileEnv, @@ -138,7 +138,7 @@ pub trait Deploy<'a> { c1_port: &Self::Port, c2: &Self::Cluster, c2_port: &Self::Port, - ); + ) -> Box; fn e2o_source( compile_env: &Self::CompileEnv, @@ -147,13 +147,12 @@ pub trait Deploy<'a> { p2: &Self::Process, p2_port: &Self::Port, ) -> syn::Expr; - fn e2o_connect( p1: &Self::ExternalProcess, p1_port: &Self::Port, p2: &Self::Process, p2_port: &Self::Port, - ); + ) -> Box; fn o2e_sink( compile_env: &Self::CompileEnv, @@ -162,13 +161,12 @@ pub trait Deploy<'a> { p2: &Self::ExternalProcess, p2_port: &Self::Port, ) -> syn::Expr; - fn o2e_connect( p1: &Self::Process, p1_port: &Self::Port, p2: &Self::ExternalProcess, p2_port: &Self::Port, - ); + ) -> Box; fn cluster_ids( env: &Self::CompileEnv, diff --git a/hydroflow_plus/src/ir.rs b/hydroflow_plus/src/ir.rs index c765bc93e9f4..5630e01ce800 100644 --- a/hydroflow_plus/src/ir.rs +++ b/hydroflow_plus/src/ir.rs @@ -42,12 +42,12 @@ impl std::fmt::Debug for DebugExpr { } } -pub enum DebugInstantiate<'a> { +pub enum DebugInstantiate { Building(), - Finalized(syn::Expr, syn::Expr, Box), + Finalized(syn::Expr, syn::Expr, Option>), } -impl<'a> std::fmt::Debug for DebugInstantiate<'a> { +impl std::fmt::Debug for DebugInstantiate { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { write!(f, "") } @@ -76,31 +76,31 @@ pub enum HfPlusSource { /// any downstream values. Traversals over the dataflow graph and /// generating Hydroflow IR start from leaves. #[derive(Debug)] -pub enum HfPlusLeaf<'a> { +pub enum HfPlusLeaf { ForEach { f: DebugExpr, - input: Box>, + input: Box, }, DestSink { sink: DebugExpr, - input: Box>, + input: Box, }, CycleSink { ident: syn::Ident, location_kind: LocationId, - input: Box>, + input: Box, }, } -impl<'a> HfPlusLeaf<'a> { - pub fn compile_network + 'a>( +impl HfPlusLeaf { + pub fn compile_network<'a, D: Deploy<'a> + 'a>( self, compile_env: &D::CompileEnv, - seen_tees: &mut SeenTees<'a>, + seen_tees: &mut SeenTees, nodes: &HashMap, clusters: &HashMap, externals: &HashMap, - ) -> HfPlusLeaf<'a> { + ) -> HfPlusLeaf { self.transform_children( |n, s| { n.compile_network::(compile_env, s, nodes, clusters, externals); @@ -109,7 +109,7 @@ impl<'a> HfPlusLeaf<'a> { ) } - pub fn connect_network(self, seen_tees: &mut SeenTees<'a>) -> HfPlusLeaf<'a> { + pub fn connect_network(self, seen_tees: &mut SeenTees) -> HfPlusLeaf { self.transform_children( |n, s| { n.connect_network(s); @@ -120,9 +120,9 @@ impl<'a> HfPlusLeaf<'a> { pub fn transform_children( self, - mut transform: impl FnMut(&mut HfPlusNode<'a>, &mut SeenTees<'a>), - seen_tees: &mut SeenTees<'a>, - ) -> HfPlusLeaf<'a> { + mut transform: impl FnMut(&mut HfPlusNode, &mut SeenTees), + seen_tees: &mut SeenTees, + ) -> HfPlusLeaf { match self { HfPlusLeaf::ForEach { f, mut input } => { transform(&mut input, seen_tees); @@ -150,7 +150,7 @@ impl<'a> HfPlusLeaf<'a> { pub fn emit( &self, graph_builders: &mut BTreeMap, - built_tees: &mut HashMap<*const RefCell>, (syn::Ident, usize)>, + built_tees: &mut HashMap<*const RefCell, (syn::Ident, usize)>, next_stmt_id: &mut usize, ) { match self { @@ -211,7 +211,7 @@ impl<'a> HfPlusLeaf<'a> { /// An intermediate node in a Hydroflow+ graph, which consumes data /// from upstream nodes and emits data to downstream nodes. #[derive(Debug)] -pub enum HfPlusNode<'a> { +pub enum HfPlusNode { Placeholder, Source { @@ -225,65 +225,65 @@ pub enum HfPlusNode<'a> { }, Tee { - inner: Rc>>, + inner: Rc>, }, - Persist(Box>), - Unpersist(Box>), - Delta(Box>), + Persist(Box), + Unpersist(Box), + Delta(Box), - Union(Box>, Box>), - CrossProduct(Box>, Box>), - CrossSingleton(Box>, Box>), - Join(Box>, Box>), - Difference(Box>, Box>), - AntiJoin(Box>, Box>), + Union(Box, Box), + CrossProduct(Box, Box), + CrossSingleton(Box, Box), + Join(Box, Box), + Difference(Box, Box), + AntiJoin(Box, Box), Map { f: DebugExpr, - input: Box>, + input: Box, }, FlatMap { f: DebugExpr, - input: Box>, + input: Box, }, Filter { f: DebugExpr, - input: Box>, + input: Box, }, FilterMap { f: DebugExpr, - input: Box>, + input: Box, }, - DeferTick(Box>), - Enumerate(Box>), + DeferTick(Box), + Enumerate(Box), Inspect { f: DebugExpr, - input: Box>, + input: Box, }, - Unique(Box>), + Unique(Box), - Sort(Box>), + Sort(Box), Fold { init: DebugExpr, acc: DebugExpr, - input: Box>, + input: Box, }, FoldKeyed { init: DebugExpr, acc: DebugExpr, - input: Box>, + input: Box, }, Reduce { f: DebugExpr, - input: Box>, + input: Box, }, ReduceKeyed { f: DebugExpr, - input: Box>, + input: Box, }, Network { @@ -292,19 +292,19 @@ pub enum HfPlusNode<'a> { to_location: LocationId, to_key: Option, serialize_pipeline: Option, - instantiate_fn: DebugInstantiate<'a>, + instantiate_fn: DebugInstantiate, deserialize_pipeline: Option, - input: Box>, + input: Box, }, } -pub type SeenTees<'a> = HashMap<*const RefCell>, Rc>>>; +pub type SeenTees = HashMap<*const RefCell, Rc>>; -impl<'a> HfPlusNode<'a> { +impl<'a> HfPlusNode { pub fn compile_network + 'a>( &mut self, compile_env: &D::CompileEnv, - seen_tees: &mut SeenTees<'a>, + seen_tees: &mut SeenTees, nodes: &HashMap, clusters: &HashMap, externals: &HashMap, @@ -338,18 +338,18 @@ impl<'a> HfPlusNode<'a> { DebugInstantiate::Finalized(_, _, _) => panic!("network already finalized"), }; - *instantiate_fn = DebugInstantiate::Finalized(sink_expr, source_expr, connect_fn); + *instantiate_fn = DebugInstantiate::Finalized(sink_expr, source_expr, Some(connect_fn)); } } - pub fn connect_network(&mut self, seen_tees: &mut SeenTees<'a>) { + pub fn connect_network(&mut self, seen_tees: &mut SeenTees) { self.transform_children(|n, s| n.connect_network(s), seen_tees); if let HfPlusNode::Network { instantiate_fn, .. } = self { match instantiate_fn { DebugInstantiate::Building() => panic!("network not built"), - DebugInstantiate::Finalized(_, _, ref connect_fn) => { - connect_fn(); + DebugInstantiate::Finalized(_, _, connect_fn) => { + connect_fn.take().unwrap()(); } } } @@ -357,8 +357,8 @@ impl<'a> HfPlusNode<'a> { pub fn transform_bottom_up( &mut self, - mut transform: impl FnMut(&mut HfPlusNode<'a>, &mut C) + Copy, - seen_tees: &mut SeenTees<'a>, + mut transform: impl FnMut(&mut HfPlusNode, &mut C) + Copy, + seen_tees: &mut SeenTees, ctx: &mut C, ) { self.transform_children(|n, s| n.transform_bottom_up(transform, s, ctx), seen_tees); @@ -369,8 +369,8 @@ impl<'a> HfPlusNode<'a> { #[inline(always)] pub fn transform_children( &mut self, - mut transform: impl FnMut(&mut HfPlusNode<'a>, &mut SeenTees<'a>), - seen_tees: &mut SeenTees<'a>, + mut transform: impl FnMut(&mut HfPlusNode, &mut SeenTees), + seen_tees: &mut SeenTees, ) { match self { HfPlusNode::Placeholder => { @@ -480,7 +480,7 @@ impl<'a> HfPlusNode<'a> { pub fn emit( &self, graph_builders: &mut BTreeMap, - built_tees: &mut HashMap<*const RefCell>, (syn::Ident, usize)>, + built_tees: &mut HashMap<*const RefCell, (syn::Ident, usize)>, next_stmt_id: &mut usize, ) -> (syn::Ident, usize) { match self { @@ -1148,7 +1148,7 @@ fn instantiate_network<'a, D: Deploy<'a> + 'a>( clusters: &HashMap, externals: &HashMap, compile_env: &D::CompileEnv, -) -> (syn::Expr, syn::Expr, Box) { +) -> (syn::Expr, syn::Expr, Box) { let ((sink, source), connect_fn) = match (from_location, to_location) { (LocationId::Process(from), LocationId::Process(to)) => { let from_node = nodes @@ -1169,8 +1169,7 @@ fn instantiate_network<'a, D: Deploy<'a> + 'a>( ( D::o2o_sink_source(compile_env, &from_node, &sink_port, &to_node, &source_port), - Box::new(move || D::o2o_connect(&from_node, &sink_port, &to_node, &source_port)) - as Box, + D::o2o_connect(&from_node, &sink_port, &to_node, &source_port), ) } (LocationId::Process(from), LocationId::Cluster(to)) => { @@ -1192,8 +1191,7 @@ fn instantiate_network<'a, D: Deploy<'a> + 'a>( ( D::o2m_sink_source(compile_env, &from_node, &sink_port, &to_node, &source_port), - Box::new(move || D::o2m_connect(&from_node, &sink_port, &to_node, &source_port)) - as Box, + D::o2m_connect(&from_node, &sink_port, &to_node, &source_port), ) } (LocationId::Cluster(from), LocationId::Process(to)) => { @@ -1215,8 +1213,7 @@ fn instantiate_network<'a, D: Deploy<'a> + 'a>( ( D::m2o_sink_source(compile_env, &from_node, &sink_port, &to_node, &source_port), - Box::new(move || D::m2o_connect(&from_node, &sink_port, &to_node, &source_port)) - as Box, + D::m2o_connect(&from_node, &sink_port, &to_node, &source_port), ) } (LocationId::Cluster(from), LocationId::Cluster(to)) => { @@ -1238,8 +1235,7 @@ fn instantiate_network<'a, D: Deploy<'a> + 'a>( ( D::m2m_sink_source(compile_env, &from_node, &sink_port, &to_node, &source_port), - Box::new(move || D::m2m_connect(&from_node, &sink_port, &to_node, &source_port)) - as Box, + D::m2m_connect(&from_node, &sink_port, &to_node, &source_port), ) } (LocationId::ExternalProcess(from), LocationId::Process(to)) => { @@ -1270,8 +1266,7 @@ fn instantiate_network<'a, D: Deploy<'a> + 'a>( parse_quote!(DUMMY), D::e2o_source(compile_env, &from_node, &sink_port, &to_node, &source_port), ), - Box::new(move || D::e2o_connect(&from_node, &sink_port, &to_node, &source_port)) - as Box, + D::e2o_connect(&from_node, &sink_port, &to_node, &source_port), ) } (LocationId::ExternalProcess(_from), LocationId::Cluster(_to)) => { @@ -1305,8 +1300,7 @@ fn instantiate_network<'a, D: Deploy<'a> + 'a>( D::o2e_sink(compile_env, &from_node, &sink_port, &to_node, &source_port), parse_quote!(DUMMY), ), - Box::new(move || D::o2e_connect(&from_node, &sink_port, &to_node, &source_port)) - as Box, + D::o2e_connect(&from_node, &sink_port, &to_node, &source_port), ) } (LocationId::Cluster(_from), LocationId::ExternalProcess(_to)) => { diff --git a/hydroflow_plus/src/lib.rs b/hydroflow_plus/src/lib.rs index e214ec5ac6c6..7c986128bd8f 100644 --- a/hydroflow_plus/src/lib.rs +++ b/hydroflow_plus/src/lib.rs @@ -25,7 +25,7 @@ pub mod singleton; pub use singleton::{Optional, Singleton}; pub mod location; -pub use location::{Cluster, Process}; +pub use location::{Cluster, Location, Process}; pub mod deploy; pub use deploy::{ClusterSpec, Deploy, ProcessSpec}; diff --git a/hydroflow_plus/src/location.rs b/hydroflow_plus/src/location.rs index 73ba8cfebf01..b4a6b7a9ef5f 100644 --- a/hydroflow_plus/src/location.rs +++ b/hydroflow_plus/src/location.rs @@ -1,10 +1,18 @@ use std::marker::PhantomData; +use std::time::Duration; +use hydroflow::bytes::Bytes; +use hydroflow::futures::stream::Stream as FuturesStream; +use hydroflow::{tokio, tokio_stream}; +use proc_macro2::Span; use serde::de::DeserializeOwned; use serde::Serialize; -use stageleft::Quoted; +use stageleft::{q, Quoted}; -use super::builder::{ClusterIds, ClusterSelfId}; +use super::builder::{ClusterIds, ClusterSelfId, FlowState}; +use crate::cycle::{CycleCollection, CycleCollectionWithInitial}; +use crate::ir::{HfPlusNode, HfPlusSource}; +use crate::{Bounded, HfCycle, NoTick, Optional, Singleton, Stream, Tick, Unbounded}; #[derive(PartialEq, Eq, Clone, Copy, Debug)] pub enum LocationId { @@ -13,8 +21,240 @@ pub enum LocationId { ExternalProcess(usize), } -pub trait Location { +pub trait Location<'a> { fn id(&self) -> LocationId; + + fn flow_state(&self) -> &FlowState; + + fn spin(&self) -> Stream<(), Unbounded, NoTick, Self> + where + Self: Sized, + { + Stream::new( + self.id(), + self.flow_state().clone(), + HfPlusNode::Persist(Box::new(HfPlusNode::Source { + source: HfPlusSource::Spin(), + location_kind: self.id(), + })), + ) + } + + fn spin_batch( + &self, + batch_size: impl Quoted<'a, usize> + Copy + 'a, + ) -> Stream<(), Bounded, Tick, Self> + where + Self: Sized, + { + self.spin() + .flat_map(q!(move |_| 0..batch_size)) + .map(q!(|_| ())) + .tick_batch() + } + + fn source_stream + Unpin>( + &self, + e: impl Quoted<'a, E>, + ) -> Stream + where + Self: Sized, + { + let e = e.splice_untyped(); + + Stream::new( + self.id(), + self.flow_state().clone(), + HfPlusNode::Persist(Box::new(HfPlusNode::Source { + source: HfPlusSource::Stream(e.into()), + location_kind: self.id(), + })), + ) + } + + fn source_iter>( + &self, + e: impl Quoted<'a, E>, + ) -> Stream + where + Self: Sized, + { + let e = e.splice_untyped(); + + Stream::new( + self.id(), + self.flow_state().clone(), + HfPlusNode::Persist(Box::new(HfPlusNode::Source { + source: HfPlusSource::Iter(e.into()), + location_kind: self.id(), + })), + ) + } + + fn singleton(&self, e: impl Quoted<'a, T>) -> Singleton + where + Self: Sized, + { + let e_arr = q!([e]); + let e = e_arr.splice_untyped(); + + // we do a double persist here because if the singleton shows up on every tick, + // we first persist the source so that we store that value and then persist again + // so that it grows every tick + Singleton::new( + self.id(), + self.flow_state().clone(), + HfPlusNode::Persist(Box::new(HfPlusNode::Persist(Box::new( + HfPlusNode::Source { + source: HfPlusSource::Iter(e.into()), + location_kind: self.id(), + }, + )))), + ) + } + + fn singleton_first_tick( + &self, + e: impl Quoted<'a, T>, + ) -> Optional + where + Self: Sized, + { + let e_arr = q!([e]); + let e = e_arr.splice_untyped(); + + Optional::new( + self.id(), + self.flow_state().clone(), + HfPlusNode::Source { + source: HfPlusSource::Iter(e.into()), + location_kind: self.id(), + }, + ) + } + + fn source_interval( + &self, + interval: impl Quoted<'a, Duration> + Copy + 'a, + ) -> Optional<(), Unbounded, NoTick, Self> + where + Self: Sized, + { + let interval = interval.splice_untyped(); + + Optional::new( + self.id(), + self.flow_state().clone(), + HfPlusNode::Persist(Box::new(HfPlusNode::Source { + source: HfPlusSource::Interval(interval.into()), + location_kind: self.id(), + })), + ) + } + + fn source_interval_delayed( + &self, + delay: impl Quoted<'a, Duration> + Copy + 'a, + interval: impl Quoted<'a, Duration> + Copy + 'a, + ) -> Optional + where + Self: Sized, + { + self.source_stream(q!(tokio_stream::wrappers::IntervalStream::new( + tokio::time::interval_at(tokio::time::Instant::now() + delay, interval) + ))) + .tick_batch() + .first() + .latest() + } + + fn tick_cycle>( + &self, + ) -> (HfCycle<'a, Tick, S>, S) { + let next_id = { + let on_id = match self.id() { + LocationId::Process(id) => id, + LocationId::Cluster(id) => id, + LocationId::ExternalProcess(_) => panic!(), + }; + + let mut flow_state = self.flow_state().borrow_mut(); + let next_id_entry = flow_state.cycle_counts.entry(on_id).or_default(); + + let id = *next_id_entry; + *next_id_entry += 1; + id + }; + + let ident = syn::Ident::new(&format!("cycle_{}", next_id), Span::call_site()); + + ( + HfCycle { + ident: ident.clone(), + _phantom: PhantomData, + }, + S::create_source(ident, self.flow_state().clone(), self.id()), + ) + } + + fn cycle>( + &self, + ) -> (HfCycle<'a, NoTick, S>, S) { + let next_id = { + let on_id = match self.id() { + LocationId::Process(id) => id, + LocationId::Cluster(id) => id, + LocationId::ExternalProcess(_) => panic!(), + }; + + let mut flow_state = self.flow_state().borrow_mut(); + let next_id_entry = flow_state.cycle_counts.entry(on_id).or_default(); + + let id = *next_id_entry; + *next_id_entry += 1; + id + }; + + let ident = syn::Ident::new(&format!("cycle_{}", next_id), Span::call_site()); + + ( + HfCycle { + ident: ident.clone(), + _phantom: PhantomData, + }, + S::create_source(ident, self.flow_state().clone(), self.id()), + ) + } + + fn tick_cycle_with_initial>( + &self, + initial: S, + ) -> (HfCycle<'a, Tick, S>, S) { + let next_id = { + let on_id = match self.id() { + LocationId::Process(id) => id, + LocationId::Cluster(id) => id, + LocationId::ExternalProcess(_) => panic!(), + }; + + let mut flow_state = self.flow_state().borrow_mut(); + let next_id_entry = flow_state.cycle_counts.entry(on_id).or_default(); + + let id = *next_id_entry; + *next_id_entry += 1; + id + }; + + let ident = syn::Ident::new(&format!("cycle_{}", next_id), Span::call_site()); + + ( + HfCycle { + ident: ident.clone(), + _phantom: PhantomData, + }, + S::create_source(ident, self.flow_state().clone(), initial, self.id()), + ) + } } pub struct ExternalBytesPort { @@ -34,60 +274,150 @@ pub struct ExternalBincodeStream { pub(crate) _phantom: PhantomData, } -pub struct ExternalProcess

{ +pub struct ExternalProcess<'a, P> { pub(crate) id: usize, - pub(crate) _phantom: PhantomData

, + + pub(crate) flow_state: FlowState, + + pub(crate) _phantom: PhantomData<&'a &'a mut P>, } -impl

Clone for ExternalProcess

{ +impl<'a, P> Clone for ExternalProcess<'a, P> { fn clone(&self) -> Self { ExternalProcess { id: self.id, + flow_state: self.flow_state.clone(), _phantom: PhantomData, } } } -impl

Location for ExternalProcess

{ +impl<'a, P> Location<'a> for ExternalProcess<'a, P> { fn id(&self) -> LocationId { LocationId::ExternalProcess(self.id) } + + fn flow_state(&self) -> &FlowState { + &self.flow_state + } } -pub struct Process

{ +impl<'a, P> ExternalProcess<'a, P> { + pub fn source_external_bytes>( + &self, + to: &L, + ) -> (ExternalBytesPort, Stream) { + let next_external_port_id = { + let mut flow_state = self.flow_state.borrow_mut(); + let id = flow_state.next_external_out; + flow_state.next_external_out += 1; + id + }; + + ( + ExternalBytesPort { + process_id: self.id, + port_id: next_external_port_id, + }, + Stream::new( + to.id(), + self.flow_state().clone(), + HfPlusNode::Persist(Box::new(HfPlusNode::Network { + from_location: LocationId::ExternalProcess(self.id), + from_key: Some(next_external_port_id), + to_location: to.id(), + to_key: None, + serialize_pipeline: None, + instantiate_fn: crate::ir::DebugInstantiate::Building(), + deserialize_pipeline: Some(syn::parse_quote!(map(|b| b.unwrap().freeze()))), + input: Box::new(HfPlusNode::Source { + source: HfPlusSource::ExternalNetwork(), + location_kind: LocationId::ExternalProcess(self.id), + }), + })), + ), + ) + } + + pub fn source_external_bincode, T: Serialize + DeserializeOwned>( + &self, + to: &L, + ) -> (ExternalBincodeSink, Stream) { + let next_external_port_id = { + let mut flow_state = self.flow_state.borrow_mut(); + let id = flow_state.next_external_out; + flow_state.next_external_out += 1; + id + }; + + ( + ExternalBincodeSink { + process_id: self.id, + port_id: next_external_port_id, + _phantom: PhantomData, + }, + Stream::new( + to.id(), + self.flow_state().clone(), + HfPlusNode::Persist(Box::new(HfPlusNode::Network { + from_location: LocationId::ExternalProcess(self.id), + from_key: Some(next_external_port_id), + to_location: to.id(), + to_key: None, + serialize_pipeline: None, + instantiate_fn: crate::ir::DebugInstantiate::Building(), + deserialize_pipeline: Some(crate::stream::deserialize_bincode::(false)), + input: Box::new(HfPlusNode::Source { + source: HfPlusSource::ExternalNetwork(), + location_kind: LocationId::ExternalProcess(self.id), + }), + })), + ), + ) + } +} + +pub struct Process<'a, P> { pub(crate) id: usize, - pub(crate) _phantom: PhantomData

, + pub(crate) flow_state: FlowState, + pub(crate) _phantom: PhantomData<&'a &'a mut P>, } -impl

Clone for Process

{ +impl<'a, P> Clone for Process<'a, P> { fn clone(&self) -> Self { Process { id: self.id, + flow_state: self.flow_state.clone(), _phantom: PhantomData, } } } -impl

Location for Process

{ +impl<'a, P> Location<'a> for Process<'a, P> { fn id(&self) -> LocationId { LocationId::Process(self.id) } + + fn flow_state(&self) -> &FlowState { + &self.flow_state + } } -pub struct Cluster { +pub struct Cluster<'a, C> { pub(crate) id: usize, - pub(crate) _phantom: PhantomData, + pub(crate) flow_state: FlowState, + pub(crate) _phantom: PhantomData<&'a &'a mut C>, } -impl Cluster { - pub fn self_id<'a>(&self) -> impl Quoted<'a, u32> + Copy + 'a { +impl<'a, C> Cluster<'a, C> { + pub fn self_id(&self) -> impl Quoted<'a, u32> + Copy + 'a { ClusterSelfId { id: self.id, _phantom: PhantomData, } } - pub fn members<'a>(&self) -> impl Quoted<'a, &'a Vec> + Copy + 'a { + pub fn members(&self) -> impl Quoted<'a, &'a Vec> + Copy + 'a { ClusterIds { id: self.id, _phantom: PhantomData, @@ -95,22 +425,27 @@ impl Cluster { } } -impl Clone for Cluster { +impl<'a, C> Clone for Cluster<'a, C> { fn clone(&self) -> Self { Cluster { id: self.id, + flow_state: self.flow_state.clone(), _phantom: PhantomData, } } } -impl Location for Cluster { +impl<'a, C> Location<'a> for Cluster<'a, C> { fn id(&self) -> LocationId { LocationId::Cluster(self.id) } + + fn flow_state(&self) -> &FlowState { + &self.flow_state + } } -pub trait CanSend: Location { +pub trait CanSend<'a, To: Location<'a>>: Location<'a> { type In; type Out; @@ -118,7 +453,7 @@ pub trait CanSend: Location { fn is_tagged() -> bool; } -impl CanSend> for Process { +impl<'a, P1, P2> CanSend<'a, Process<'a, P2>> for Process<'a, P1> { type In = T; type Out = T; @@ -131,7 +466,7 @@ impl CanSend> for Process { } } -impl CanSend> for Process { +impl<'a, P1, C2> CanSend<'a, Cluster<'a, C2>> for Process<'a, P1> { type In = (u32, T); type Out = T; @@ -144,7 +479,7 @@ impl CanSend> for Process { } } -impl CanSend> for Cluster { +impl<'a, C1, P2> CanSend<'a, Process<'a, P2>> for Cluster<'a, C1> { type In = T; type Out = (u32, T); @@ -157,7 +492,7 @@ impl CanSend> for Cluster { } } -impl CanSend> for Cluster { +impl<'a, C1, C2> CanSend<'a, Cluster<'a, C2>> for Cluster<'a, C1> { type In = (u32, T); type Out = (u32, T); @@ -170,7 +505,7 @@ impl CanSend> for Cluster { } } -impl CanSend> for Process { +impl<'a, P1, E2> CanSend<'a, ExternalProcess<'a, E2>> for Process<'a, P1> { type In = T; type Out = T; diff --git a/hydroflow_plus/src/persist_pullup.rs b/hydroflow_plus/src/persist_pullup.rs index 4a394eda8238..ffe4f8d30e8d 100644 --- a/hydroflow_plus/src/persist_pullup.rs +++ b/hydroflow_plus/src/persist_pullup.rs @@ -3,9 +3,9 @@ use std::collections::HashSet; use crate::ir::*; -fn persist_pullup_node<'a>( - node: &mut HfPlusNode<'a>, - persist_pulled_tees: &mut HashSet<*const RefCell>>, +fn persist_pullup_node( + node: &mut HfPlusNode, + persist_pulled_tees: &mut HashSet<*const RefCell>, ) { *node = match std::mem::replace(node, HfPlusNode::Placeholder) { HfPlusNode::Unpersist(box HfPlusNode::Persist(box behind_persist)) => behind_persist, @@ -13,12 +13,12 @@ fn persist_pullup_node<'a>( HfPlusNode::Delta(box HfPlusNode::Persist(box behind_persist)) => behind_persist, HfPlusNode::Tee { inner } => { - if persist_pulled_tees.contains(&(inner.as_ref() as *const RefCell>)) { + if persist_pulled_tees.contains(&(inner.as_ref() as *const RefCell)) { HfPlusNode::Persist(Box::new(HfPlusNode::Tee { inner: inner.clone(), })) } else if matches!(*inner.borrow(), HfPlusNode::Persist(_)) { - persist_pulled_tees.insert(inner.as_ref() as *const RefCell>); + persist_pulled_tees.insert(inner.as_ref() as *const RefCell); if let HfPlusNode::Persist(box behind_persist) = inner.replace(HfPlusNode::Placeholder) { @@ -128,13 +128,15 @@ mod tests { use stageleft::*; use crate::deploy::MultiGraph; + use crate::location::Location; #[test] fn persist_pullup_through_map() { let flow = crate::builder::FlowBuilder::new(); let process = flow.process::<()>(); - flow.source_iter(&process, q!(0..10)) + process + .source_iter(q!(0..10)) .map(q!(|v| v + 1)) .for_each(q!(|n| println!("{}", n))); @@ -157,7 +159,7 @@ mod tests { let flow = crate::builder::FlowBuilder::new(); let process = flow.process::<()>(); - let before_tee = flow.source_iter(&process, q!(0..10)).tick_batch().persist(); + let before_tee = process.source_iter(q!(0..10)).tick_batch().persist(); before_tee .clone() diff --git a/hydroflow_plus/src/profiler.rs b/hydroflow_plus/src/profiler.rs index 85409f7ec4ee..4e4ded0833cc 100644 --- a/hydroflow_plus/src/profiler.rs +++ b/hydroflow_plus/src/profiler.rs @@ -16,12 +16,12 @@ fn quoted_any_fn<'a, F: Fn(&usize) + 'a, Q: IntoQuotedMut<'a, F>>(q: Q) -> Q { /// Add a profiling node before each node to count the cardinality of its input fn add_profiling_node<'a>( - node: &mut HfPlusNode<'a>, + node: &mut HfPlusNode, _context: RuntimeContext<'a>, counters: RuntimeData<&'a RefCell>>, counter_queue: RuntimeData<&'a RefCell>>, id: &mut u32, - seen_tees: &mut SeenTees<'a>, + seen_tees: &mut SeenTees, ) { let my_id = *id; *id += 1; @@ -53,11 +53,11 @@ fn add_profiling_node<'a>( /// Count the cardinality of each input and periodically output to a file pub fn profiling<'a>( - ir: Vec>, + ir: Vec, context: RuntimeContext<'a>, counters: RuntimeData<&'a RefCell>>, counter_queue: RuntimeData<&'a RefCell>>, -) -> Vec> { +) -> Vec { let mut id = 0; let mut seen_tees = Default::default(); ir.into_iter() @@ -77,13 +77,15 @@ mod tests { use stageleft::*; use crate::deploy::MultiGraph; + use crate::location::Location; #[test] fn profiler_wrapping_all_operators() { let flow = crate::builder::FlowBuilder::new(); let process = flow.process::<()>(); - flow.source_iter(&process, q!(0..10)) + process + .source_iter(q!(0..10)) .map(q!(|v| v + 1)) .for_each(q!(|n| println!("{}", n))); diff --git a/hydroflow_plus/src/properties.rs b/hydroflow_plus/src/properties.rs index 82ff2ebd89ae..ca557c0ef67c 100644 --- a/hydroflow_plus/src/properties.rs +++ b/hydroflow_plus/src/properties.rs @@ -51,10 +51,10 @@ impl PropertyDatabase { // Dataflow graph optimization rewrite rules based on algebraic property tags // TODO add a test that verifies the space of possible graphs after rewrites is correct for each property -fn properties_optimize_node<'a>( - node: &mut HfPlusNode<'a>, +fn properties_optimize_node( + node: &mut HfPlusNode, db: &PropertyDatabase, - seen_tees: &mut SeenTees<'a>, + seen_tees: &mut SeenTees, ) { node.transform_children( |node, seen_tees| properties_optimize_node(node, db, seen_tees), @@ -68,10 +68,7 @@ fn properties_optimize_node<'a>( } } -pub fn properties_optimize<'a>( - ir: Vec>, - db: &PropertyDatabase, -) -> Vec> { +pub fn properties_optimize(ir: Vec, db: &PropertyDatabase) -> Vec { let mut seen_tees = Default::default(); ir.into_iter() .map(|l| { @@ -87,6 +84,7 @@ pub fn properties_optimize<'a>( mod tests { use super::*; use crate::deploy::SingleProcessGraph; + use crate::location::Location; use crate::FlowBuilder; #[test] @@ -110,7 +108,8 @@ mod tests { let counter_func = q!(|count: &mut i32, _| *count += 1); let _ = database.add_commutative_tag(counter_func); - flow.source_iter(&process, q!(vec![])) + process + .source_iter(q!(vec![])) .map(q!(|string: String| (string, ()))) .tick_batch() .fold_keyed(q!(|| 0), counter_func) diff --git a/hydroflow_plus/src/singleton.rs b/hydroflow_plus/src/singleton.rs index 535c7e0b4548..64d5e8f64f2d 100644 --- a/hydroflow_plus/src/singleton.rs +++ b/hydroflow_plus/src/singleton.rs @@ -5,7 +5,7 @@ use std::rc::Rc; use stageleft::{q, IntoQuotedMut, Quoted}; -use crate::builder::FlowLeaves; +use crate::builder::FlowState; use crate::cycle::{CycleCollection, CycleCollectionWithInitial, CycleComplete}; use crate::ir::{HfPlusLeaf, HfPlusNode, HfPlusSource}; use crate::location::{Location, LocationId}; @@ -15,142 +15,122 @@ use crate::Stream; pub trait CrossResult<'a, Other> { type Out; fn other_location(other: &Other) -> LocationId; - fn other_ir_node(other: Other) -> HfPlusNode<'a>; + fn other_ir_node(other: Other) -> HfPlusNode; - fn make( - location_kind: LocationId, - ir_leaves: FlowLeaves<'a>, - ir_node: HfPlusNode<'a>, - ) -> Self::Out; + fn make(location_kind: LocationId, flow_state: FlowState, ir_node: HfPlusNode) -> Self::Out; } -impl<'a, T, U: Clone, W, C, N: Location> CrossResult<'a, Singleton<'a, U, W, C, N>> - for Singleton<'a, T, W, C, N> +impl<'a, T, U: Clone, W, C, N: Location<'a>> CrossResult<'a, Singleton> + for Singleton { - type Out = Singleton<'a, (T, U), W, C, N>; + type Out = Singleton<(T, U), W, C, N>; - fn other_location(other: &Singleton<'a, U, W, C, N>) -> LocationId { + fn other_location(other: &Singleton) -> LocationId { other.location_kind } - fn other_ir_node(other: Singleton<'a, U, W, C, N>) -> HfPlusNode<'a> { + fn other_ir_node(other: Singleton) -> HfPlusNode { other.ir_node.into_inner() } - fn make( - location_kind: LocationId, - ir_leaves: FlowLeaves<'a>, - ir_node: HfPlusNode<'a>, - ) -> Self::Out { - Singleton::new(location_kind, ir_leaves, ir_node) + fn make(location_kind: LocationId, flow_state: FlowState, ir_node: HfPlusNode) -> Self::Out { + Singleton::new(location_kind, flow_state, ir_node) } } -impl<'a, T, U: Clone, W, C, N: Location> CrossResult<'a, Optional<'a, U, W, C, N>> - for Singleton<'a, T, W, C, N> +impl<'a, T, U: Clone, W, C, N: Location<'a>> CrossResult<'a, Optional> + for Singleton { - type Out = Optional<'a, (T, U), W, C, N>; + type Out = Optional<(T, U), W, C, N>; - fn other_location(other: &Optional<'a, U, W, C, N>) -> LocationId { + fn other_location(other: &Optional) -> LocationId { other.location_kind } - fn other_ir_node(other: Optional<'a, U, W, C, N>) -> HfPlusNode<'a> { + fn other_ir_node(other: Optional) -> HfPlusNode { other.ir_node.into_inner() } - fn make( - location_kind: LocationId, - ir_leaves: FlowLeaves<'a>, - ir_node: HfPlusNode<'a>, - ) -> Self::Out { - Optional::new(location_kind, ir_leaves, ir_node) + fn make(location_kind: LocationId, flow_state: FlowState, ir_node: HfPlusNode) -> Self::Out { + Optional::new(location_kind, flow_state, ir_node) } } -impl<'a, T, U: Clone, W, C, N: Location> CrossResult<'a, Optional<'a, U, W, C, N>> - for Optional<'a, T, W, C, N> +impl<'a, T, U: Clone, W, C, N: Location<'a>> CrossResult<'a, Optional> + for Optional { - type Out = Optional<'a, (T, U), W, C, N>; + type Out = Optional<(T, U), W, C, N>; - fn other_location(other: &Optional<'a, U, W, C, N>) -> LocationId { + fn other_location(other: &Optional) -> LocationId { other.location_kind } - fn other_ir_node(other: Optional<'a, U, W, C, N>) -> HfPlusNode<'a> { + fn other_ir_node(other: Optional) -> HfPlusNode { other.ir_node.into_inner() } - fn make( - location_kind: LocationId, - ir_leaves: FlowLeaves<'a>, - ir_node: HfPlusNode<'a>, - ) -> Self::Out { - Optional::new(location_kind, ir_leaves, ir_node) + fn make(location_kind: LocationId, flow_state: FlowState, ir_node: HfPlusNode) -> Self::Out { + Optional::new(location_kind, flow_state, ir_node) } } -impl<'a, T, U: Clone, W, C, N: Location> CrossResult<'a, Singleton<'a, U, W, C, N>> - for Optional<'a, T, W, C, N> +impl<'a, T, U: Clone, W, C, N: Location<'a>> CrossResult<'a, Singleton> + for Optional { - type Out = Optional<'a, (T, U), W, C, N>; + type Out = Optional<(T, U), W, C, N>; - fn other_location(other: &Singleton<'a, U, W, C, N>) -> LocationId { + fn other_location(other: &Singleton) -> LocationId { other.location_kind } - fn other_ir_node(other: Singleton<'a, U, W, C, N>) -> HfPlusNode<'a> { + fn other_ir_node(other: Singleton) -> HfPlusNode { other.ir_node.into_inner() } - fn make( - location_kind: LocationId, - ir_leaves: FlowLeaves<'a>, - ir_node: HfPlusNode<'a>, - ) -> Self::Out { - Optional::new(location_kind, ir_leaves, ir_node) + fn make(location_kind: LocationId, flow_state: FlowState, ir_node: HfPlusNode) -> Self::Out { + Optional::new(location_kind, flow_state, ir_node) } } -pub struct Singleton<'a, T, W, C, N: Location> { +pub struct Singleton { pub(crate) location_kind: LocationId, - ir_leaves: FlowLeaves<'a>, - pub(crate) ir_node: RefCell>, + flow_state: FlowState, + pub(crate) ir_node: RefCell, - _phantom: PhantomData<(&'a mut &'a (), T, N, W, C)>, + _phantom: PhantomData<(T, N, W, C)>, } -impl<'a, T, W, C, N: Location> Singleton<'a, T, W, C, N> { +impl<'a, T, W, C, N: Location<'a>> Singleton { pub(crate) fn new( location_kind: LocationId, - ir_leaves: FlowLeaves<'a>, - ir_node: HfPlusNode<'a>, + flow_state: FlowState, + ir_node: HfPlusNode, ) -> Self { Singleton { location_kind, - ir_leaves, + flow_state, ir_node: RefCell::new(ir_node), _phantom: PhantomData, } } } -impl<'a, T, C, N: Location> From> - for Singleton<'a, T, Unbounded, C, N> +impl<'a, T, C, N: Location<'a>> From> + for Singleton { - fn from(singleton: Singleton<'a, T, Bounded, C, N>) -> Self { + fn from(singleton: Singleton) -> Self { Singleton::new( singleton.location_kind, - singleton.ir_leaves, + singleton.flow_state, singleton.ir_node.into_inner(), ) } } -impl<'a, T, N: Location> CycleComplete<'a, Tick> for Singleton<'a, T, Bounded, Tick, N> { +impl<'a, T, N: Location<'a>> CycleComplete<'a, Tick> for Singleton { fn complete(self, ident: syn::Ident) { - self.ir_leaves.borrow_mut().0.as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled.").push(HfPlusLeaf::CycleSink { + self.flow_state.borrow_mut().leaves.as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled.").push(HfPlusLeaf::CycleSink { ident, location_kind: self.location_kind, input: Box::new(self.ir_node.into_inner()), @@ -158,20 +138,20 @@ impl<'a, T, N: Location> CycleComplete<'a, Tick> for Singleton<'a, T, Bounded, T } } -impl<'a, T, N: Location> CycleCollectionWithInitial<'a, Tick> - for Singleton<'a, T, Bounded, Tick, N> +impl<'a, T, N: Location<'a>> CycleCollectionWithInitial<'a, Tick> + for Singleton { type Location = N; fn create_source( ident: syn::Ident, - ir_leaves: FlowLeaves<'a>, + flow_state: FlowState, initial: Self, l: LocationId, ) -> Self { Singleton::new( l, - ir_leaves, + flow_state, HfPlusNode::Union( Box::new(HfPlusNode::CycleSource { ident, @@ -183,7 +163,7 @@ impl<'a, T, N: Location> CycleCollectionWithInitial<'a, Tick> } } -impl<'a, T: Clone, W, C, N: Location> Clone for Singleton<'a, T, W, C, N> { +impl<'a, T: Clone, W, C, N: Location<'a>> Clone for Singleton { fn clone(&self) -> Self { if !matches!(self.ir_node.borrow().deref(), HfPlusNode::Tee { .. }) { let orig_ir_node = self.ir_node.replace(HfPlusNode::Placeholder); @@ -195,7 +175,7 @@ impl<'a, T: Clone, W, C, N: Location> Clone for Singleton<'a, T, W, C, N> { if let HfPlusNode::Tee { inner } = self.ir_node.borrow().deref() { Singleton { location_kind: self.location_kind, - ir_leaves: self.ir_leaves.clone(), + flow_state: self.flow_state.clone(), ir_node: HfPlusNode::Tee { inner: inner.clone(), } @@ -208,14 +188,11 @@ impl<'a, T: Clone, W, C, N: Location> Clone for Singleton<'a, T, W, C, N> { } } -impl<'a, T, W, C, N: Location> Singleton<'a, T, W, C, N> { - pub fn map U + 'a>( - self, - f: impl IntoQuotedMut<'a, F>, - ) -> Singleton<'a, U, W, C, N> { +impl<'a, T, W, C, N: Location<'a>> Singleton { + pub fn map U + 'a>(self, f: impl IntoQuotedMut<'a, F>) -> Singleton { Singleton::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Map { f: f.splice_fn1().into(), input: Box::new(self.ir_node.into_inner()), @@ -226,10 +203,10 @@ impl<'a, T, W, C, N: Location> Singleton<'a, T, W, C, N> { pub fn flat_map, F: Fn(T) -> I + 'a>( self, f: impl IntoQuotedMut<'a, F>, - ) -> Stream<'a, U, W, C, N> { + ) -> Stream { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::FlatMap { f: f.splice_fn1().into(), input: Box::new(self.ir_node.into_inner()), @@ -240,10 +217,10 @@ impl<'a, T, W, C, N: Location> Singleton<'a, T, W, C, N> { pub fn filter bool + 'a>( self, f: impl IntoQuotedMut<'a, F>, - ) -> Optional<'a, T, W, C, N> { + ) -> Optional { Optional::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Filter { f: f.splice_fn1_borrow().into(), input: Box::new(self.ir_node.into_inner()), @@ -254,10 +231,10 @@ impl<'a, T, W, C, N: Location> Singleton<'a, T, W, C, N> { pub fn filter_map Option + 'a>( self, f: impl IntoQuotedMut<'a, F>, - ) -> Optional<'a, U, W, C, N> { + ) -> Optional { Optional::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::FilterMap { f: f.splice_fn1().into(), input: Box::new(self.ir_node.into_inner()), @@ -266,7 +243,7 @@ impl<'a, T, W, C, N: Location> Singleton<'a, T, W, C, N> { } } -impl<'a, T, N: Location> Singleton<'a, T, Bounded, Tick, N> { +impl<'a, T, N: Location<'a>> Singleton { pub fn cross_singleton(self, other: Other) -> >::Out where Self: CrossResult<'a, Other>, @@ -277,7 +254,7 @@ impl<'a, T, N: Location> Singleton<'a, T, Bounded, Tick, N> { Self::make( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::CrossSingleton( Box::new(self.ir_node.into_inner()), Box::new(Self::other_ir_node(other)), @@ -287,67 +264,67 @@ impl<'a, T, N: Location> Singleton<'a, T, Bounded, Tick, N> { pub fn continue_if( self, - signal: Optional<'a, U, Bounded, Tick, N>, - ) -> Optional<'a, T, Bounded, Tick, N> { + signal: Optional, + ) -> Optional { self.cross_singleton(signal.map(q!(|_u| ()))) .map(q!(|(d, _signal)| d)) } pub fn continue_unless( self, - other: Optional<'a, U, Bounded, Tick, N>, - ) -> Optional<'a, T, Bounded, Tick, N> { + other: Optional, + ) -> Optional { self.continue_if(other.into_stream().count().filter(q!(|c| *c == 0))) } } -impl<'a, T, N: Location> Singleton<'a, T, Bounded, Tick, N> { - pub fn all_ticks(self) -> Stream<'a, T, Unbounded, NoTick, N> { +impl<'a, T, N: Location<'a>> Singleton { + pub fn all_ticks(self) -> Stream { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Persist(Box::new(self.ir_node.into_inner())), ) } - pub fn latest(self) -> Singleton<'a, T, Unbounded, NoTick, N> { + pub fn latest(self) -> Singleton { Singleton::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Persist(Box::new(self.ir_node.into_inner())), ) } - pub fn defer_tick(self) -> Singleton<'a, T, Bounded, Tick, N> { + pub fn defer_tick(self) -> Singleton { Singleton::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::DeferTick(Box::new(self.ir_node.into_inner())), ) } - pub fn persist(self) -> Stream<'a, T, Bounded, Tick, N> { + pub fn persist(self) -> Stream { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Persist(Box::new(self.ir_node.into_inner())), ) } - pub fn delta(self) -> Optional<'a, T, Bounded, Tick, N> { + pub fn delta(self) -> Optional { Optional::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Delta(Box::new(self.ir_node.into_inner())), ) } } -impl<'a, T, B, N: Location> Singleton<'a, T, B, NoTick, N> { - pub fn latest_tick(self) -> Singleton<'a, T, Bounded, Tick, N> { +impl<'a, T, B, N: Location<'a>> Singleton { + pub fn latest_tick(self) -> Singleton { Singleton::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Unpersist(Box::new(self.ir_node.into_inner())), ) } @@ -355,12 +332,12 @@ impl<'a, T, B, N: Location> Singleton<'a, T, B, NoTick, N> { pub fn sample_every( self, duration: impl Quoted<'a, std::time::Duration> + Copy + 'a, - ) -> Stream<'a, T, Unbounded, NoTick, N> { + ) -> Stream { let interval = duration.splice_typed(); - let samples = Stream::<'a, (), Bounded, Tick, N>::new( + let samples = Stream::<(), Bounded, Tick, N>::new( self.location_kind, - self.ir_leaves.clone(), + self.flow_state.clone(), HfPlusNode::Source { source: HfPlusSource::Interval(interval.into()), location_kind: self.location_kind, @@ -374,7 +351,7 @@ impl<'a, T, B, N: Location> Singleton<'a, T, B, NoTick, N> { } } -impl<'a, T, N: Location> Singleton<'a, T, Unbounded, NoTick, N> { +impl<'a, T, N: Location<'a>> Singleton { pub fn cross_singleton(self, other: Other) -> >::Out where Self: CrossResult<'a, Other>, @@ -385,7 +362,7 @@ impl<'a, T, N: Location> Singleton<'a, T, Unbounded, NoTick, N> { Self::make( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Persist(Box::new(HfPlusNode::CrossSingleton( Box::new(HfPlusNode::Unpersist(Box::new(self.ir_node.into_inner()))), Box::new(HfPlusNode::Unpersist(Box::new(Self::other_ir_node(other)))), @@ -394,42 +371,42 @@ impl<'a, T, N: Location> Singleton<'a, T, Unbounded, NoTick, N> { } } -pub struct Optional<'a, T, W, C, N: Location> { +pub struct Optional { pub(crate) location_kind: LocationId, - ir_leaves: FlowLeaves<'a>, - pub(crate) ir_node: RefCell>, + flow_state: FlowState, + pub(crate) ir_node: RefCell, - _phantom: PhantomData<(&'a mut &'a (), T, N, W, C)>, + _phantom: PhantomData<(T, N, W, C)>, } -impl<'a, T, W, C, N: Location> Optional<'a, T, W, C, N> { +impl<'a, T, W, C, N: Location<'a>> Optional { pub(crate) fn new( location_kind: LocationId, - ir_leaves: FlowLeaves<'a>, - ir_node: HfPlusNode<'a>, + flow_state: FlowState, + ir_node: HfPlusNode, ) -> Self { Optional { location_kind, - ir_leaves, + flow_state, ir_node: RefCell::new(ir_node), _phantom: PhantomData, } } - pub fn some(singleton: Singleton<'a, T, W, C, N>) -> Self { + pub fn some(singleton: Singleton) -> Self { Optional::new( singleton.location_kind, - singleton.ir_leaves, + singleton.flow_state, singleton.ir_node.into_inner(), ) } } -impl<'a, T, N: Location> CycleComplete<'a, Tick> for Optional<'a, T, Bounded, Tick, N> { +impl<'a, T, N: Location<'a>> CycleComplete<'a, Tick> for Optional { fn complete(self, ident: syn::Ident) { let me = self.defer_tick(); - me.ir_leaves.borrow_mut().0.as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled.").push(HfPlusLeaf::CycleSink { + me.flow_state.borrow_mut().leaves.as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled.").push(HfPlusLeaf::CycleSink { ident, location_kind: me.location_kind, input: Box::new(me.ir_node.into_inner()), @@ -437,13 +414,13 @@ impl<'a, T, N: Location> CycleComplete<'a, Tick> for Optional<'a, T, Bounded, Ti } } -impl<'a, T, N: Location> CycleCollection<'a, Tick> for Optional<'a, T, Bounded, Tick, N> { +impl<'a, T, N: Location<'a>> CycleCollection<'a, Tick> for Optional { type Location = N; - fn create_source(ident: syn::Ident, ir_leaves: FlowLeaves<'a>, l: LocationId) -> Self { + fn create_source(ident: syn::Ident, flow_state: FlowState, l: LocationId) -> Self { Optional::new( l, - ir_leaves, + flow_state, HfPlusNode::CycleSource { ident, location_kind: l, @@ -452,9 +429,9 @@ impl<'a, T, N: Location> CycleCollection<'a, Tick> for Optional<'a, T, Bounded, } } -impl<'a, T, W, N: Location> CycleComplete<'a, NoTick> for Optional<'a, T, W, NoTick, N> { +impl<'a, T, W, N: Location<'a>> CycleComplete<'a, NoTick> for Optional { fn complete(self, ident: syn::Ident) { - self.ir_leaves.borrow_mut().0.as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled.").push(HfPlusLeaf::CycleSink { + self.flow_state.borrow_mut().leaves.as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled.").push(HfPlusLeaf::CycleSink { ident, location_kind: self.location_kind, input: Box::new(HfPlusNode::Unpersist(Box::new(self.ir_node.into_inner()))), @@ -462,13 +439,13 @@ impl<'a, T, W, N: Location> CycleComplete<'a, NoTick> for Optional<'a, T, W, NoT } } -impl<'a, T, W, N: Location> CycleCollection<'a, NoTick> for Optional<'a, T, W, NoTick, N> { +impl<'a, T, W, N: Location<'a>> CycleCollection<'a, NoTick> for Optional { type Location = N; - fn create_source(ident: syn::Ident, ir_leaves: FlowLeaves<'a>, l: LocationId) -> Self { + fn create_source(ident: syn::Ident, flow_state: FlowState, l: LocationId) -> Self { Optional::new( l, - ir_leaves, + flow_state, HfPlusNode::Persist(Box::new(HfPlusNode::CycleSource { ident, location_kind: l, @@ -477,13 +454,13 @@ impl<'a, T, W, N: Location> CycleCollection<'a, NoTick> for Optional<'a, T, W, N } } -impl<'a, T, W, C, N: Location> From> for Optional<'a, T, W, C, N> { - fn from(singleton: Singleton<'a, T, W, C, N>) -> Self { +impl<'a, T, W, C, N: Location<'a>> From> for Optional { + fn from(singleton: Singleton) -> Self { Optional::some(singleton) } } -impl<'a, T: Clone, W, C, N: Location> Clone for Optional<'a, T, W, C, N> { +impl<'a, T: Clone, W, C, N: Location<'a>> Clone for Optional { fn clone(&self) -> Self { if !matches!(self.ir_node.borrow().deref(), HfPlusNode::Tee { .. }) { let orig_ir_node = self.ir_node.replace(HfPlusNode::Placeholder); @@ -495,7 +472,7 @@ impl<'a, T: Clone, W, C, N: Location> Clone for Optional<'a, T, W, C, N> { if let HfPlusNode::Tee { inner } = self.ir_node.borrow().deref() { Optional { location_kind: self.location_kind, - ir_leaves: self.ir_leaves.clone(), + flow_state: self.flow_state.clone(), ir_node: HfPlusNode::Tee { inner: inner.clone(), } @@ -508,14 +485,11 @@ impl<'a, T: Clone, W, C, N: Location> Clone for Optional<'a, T, W, C, N> { } } -impl<'a, T, W, C, N: Location> Optional<'a, T, W, C, N> { - pub fn map U + 'a>( - self, - f: impl IntoQuotedMut<'a, F>, - ) -> Optional<'a, U, W, C, N> { +impl<'a, T, W, C, N: Location<'a>> Optional { + pub fn map U + 'a>(self, f: impl IntoQuotedMut<'a, F>) -> Optional { Optional::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Map { f: f.splice_fn1().into(), input: Box::new(self.ir_node.into_inner()), @@ -526,10 +500,10 @@ impl<'a, T, W, C, N: Location> Optional<'a, T, W, C, N> { pub fn flat_map, F: Fn(T) -> I + 'a>( self, f: impl IntoQuotedMut<'a, F>, - ) -> Stream<'a, U, W, C, N> { + ) -> Stream { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::FlatMap { f: f.splice_fn1().into(), input: Box::new(self.ir_node.into_inner()), @@ -540,10 +514,10 @@ impl<'a, T, W, C, N: Location> Optional<'a, T, W, C, N> { pub fn filter bool + 'a>( self, f: impl IntoQuotedMut<'a, F>, - ) -> Optional<'a, T, W, C, N> { + ) -> Optional { Optional::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Filter { f: f.splice_fn1_borrow().into(), input: Box::new(self.ir_node.into_inner()), @@ -554,10 +528,10 @@ impl<'a, T, W, C, N: Location> Optional<'a, T, W, C, N> { pub fn filter_map Option + 'a>( self, f: impl IntoQuotedMut<'a, F>, - ) -> Optional<'a, U, W, C, N> { + ) -> Optional { Optional::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::FilterMap { f: f.splice_fn1().into(), input: Box::new(self.ir_node.into_inner()), @@ -566,31 +540,31 @@ impl<'a, T, W, C, N: Location> Optional<'a, T, W, C, N> { } } -impl<'a, T, N: Location> Optional<'a, T, Bounded, Tick, N> { +impl<'a, T, N: Location<'a>> Optional { // TODO(shadaj): this is technically incorrect; we should only return the first element of the stream - pub fn into_stream(self) -> Stream<'a, T, Bounded, Tick, N> { + pub fn into_stream(self) -> Stream { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, self.ir_node.into_inner(), ) } pub fn cross_singleton( self, - other: impl Into>, - ) -> Optional<'a, (T, O), Bounded, Tick, N> + other: impl Into>, + ) -> Optional<(T, O), Bounded, Tick, N> where O: Clone, { - let other: Optional<'a, O, Bounded, Tick, N> = other.into(); + let other: Optional = other.into(); if self.location_kind != other.location_kind { panic!("cross_singleton must be called on streams on the same node"); } Optional::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::CrossSingleton( Box::new(self.ir_node.into_inner()), Box::new(other.ir_node.into_inner()), @@ -600,30 +574,27 @@ impl<'a, T, N: Location> Optional<'a, T, Bounded, Tick, N> { pub fn continue_if( self, - signal: Optional<'a, U, Bounded, Tick, N>, - ) -> Optional<'a, T, Bounded, Tick, N> { + signal: Optional, + ) -> Optional { self.cross_singleton(signal.map(q!(|_u| ()))) .map(q!(|(d, _signal)| d)) } pub fn continue_unless( self, - other: Optional<'a, U, Bounded, Tick, N>, - ) -> Optional<'a, T, Bounded, Tick, N> { + other: Optional, + ) -> Optional { self.continue_if(other.into_stream().count().filter(q!(|c| *c == 0))) } - pub fn union( - self, - other: Optional<'a, T, Bounded, Tick, N>, - ) -> Optional<'a, T, Bounded, Tick, N> { + pub fn union(self, other: Optional) -> Optional { if self.location_kind != other.location_kind { panic!("union must be called on streams on the same node"); } Optional::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Union( Box::new(self.ir_node.into_inner()), Box::new(other.ir_node.into_inner()), @@ -633,15 +604,15 @@ impl<'a, T, N: Location> Optional<'a, T, Bounded, Tick, N> { pub fn unwrap_or( self, - other: Singleton<'a, T, Bounded, Tick, N>, - ) -> Singleton<'a, T, Bounded, Tick, N> { + other: Singleton, + ) -> Singleton { if self.location_kind != other.location_kind { panic!("or_else must be called on streams on the same node"); } Singleton::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Union( Box::new(self.ir_node.into_inner()), Box::new(other.ir_node.into_inner()), @@ -650,70 +621,70 @@ impl<'a, T, N: Location> Optional<'a, T, Bounded, Tick, N> { } } -impl<'a, T, N: Location> Optional<'a, T, Bounded, Tick, N> { - pub fn all_ticks(self) -> Stream<'a, T, Unbounded, NoTick, N> { +impl<'a, T, N: Location<'a>> Optional { + pub fn all_ticks(self) -> Stream { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Persist(Box::new(self.ir_node.into_inner())), ) } - pub fn latest(self) -> Optional<'a, T, Unbounded, NoTick, N> { + pub fn latest(self) -> Optional { Optional::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Persist(Box::new(self.ir_node.into_inner())), ) } - pub fn defer_tick(self) -> Optional<'a, T, Bounded, Tick, N> { + pub fn defer_tick(self) -> Optional { Optional::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::DeferTick(Box::new(self.ir_node.into_inner())), ) } - pub fn persist(self) -> Stream<'a, T, Bounded, Tick, N> { + pub fn persist(self) -> Stream { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Persist(Box::new(self.ir_node.into_inner())), ) } - pub fn delta(self) -> Optional<'a, T, Bounded, Tick, N> { + pub fn delta(self) -> Optional { Optional::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Delta(Box::new(self.ir_node.into_inner())), ) } } -impl<'a, T, B, N: Location> Optional<'a, T, B, NoTick, N> { - pub fn latest_tick(self) -> Optional<'a, T, Bounded, Tick, N> { +impl<'a, T, B, N: Location<'a>> Optional { + pub fn latest_tick(self) -> Optional { Optional::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Unpersist(Box::new(self.ir_node.into_inner())), ) } - pub fn tick_samples(self) -> Stream<'a, T, Unbounded, NoTick, N> { + pub fn tick_samples(self) -> Stream { self.latest_tick().all_ticks() } pub fn sample_every( self, duration: impl Quoted<'a, std::time::Duration> + Copy + 'a, - ) -> Stream<'a, T, Unbounded, NoTick, N> { + ) -> Stream { let interval = duration.splice_typed(); - let samples = Stream::<'a, (), Bounded, Tick, N>::new( + let samples = Stream::<(), Bounded, Tick, N>::new( self.location_kind, - self.ir_leaves.clone(), + self.flow_state.clone(), HfPlusNode::Source { source: HfPlusSource::Interval(interval.into()), location_kind: self.location_kind, @@ -728,8 +699,8 @@ impl<'a, T, B, N: Location> Optional<'a, T, B, NoTick, N> { pub fn unwrap_or( self, - other: impl Into>, - ) -> Singleton<'a, T, Unbounded, NoTick, N> { + other: impl Into>, + ) -> Singleton { let other = other.into(); if self.location_kind != other.location_kind { panic!("or_else must be called on streams on the same node"); @@ -739,15 +710,15 @@ impl<'a, T, B, N: Location> Optional<'a, T, B, NoTick, N> { } } -impl<'a, T, N: Location> Optional<'a, T, Unbounded, NoTick, N> { +impl<'a, T, N: Location<'a>> Optional { pub fn cross_singleton( self, - other: impl Into>, - ) -> Optional<'a, (T, O), Unbounded, NoTick, N> + other: impl Into>, + ) -> Optional<(T, O), Unbounded, NoTick, N> where O: Clone, { - let other: Optional<'a, O, Unbounded, NoTick, N> = other.into(); + let other: Optional = other.into(); if self.location_kind != other.location_kind { panic!("cross_singleton must be called on streams on the same node"); } diff --git a/hydroflow_plus/src/stream.rs b/hydroflow_plus/src/stream.rs index 1021b9010187..159c1814ae1b 100644 --- a/hydroflow_plus/src/stream.rs +++ b/hydroflow_plus/src/stream.rs @@ -14,7 +14,7 @@ use serde::Serialize; use stageleft::{q, IntoQuotedMut, Quoted}; use syn::parse_quote; -use crate::builder::FlowLeaves; +use crate::builder::FlowState; use crate::cycle::{CycleCollection, CycleComplete}; use crate::ir::{DebugInstantiate, HfPlusLeaf, HfPlusNode, HfPlusSource}; use crate::location::{ @@ -46,19 +46,19 @@ pub enum Tick {} /// - `C`: the tick domain of the stream, which is either [`Tick`] or /// [`NoTick`] /// - `N`: the type of the node that the stream is materialized on -pub struct Stream<'a, T, W, C, N: Location> { +pub struct Stream { location_kind: LocationId, - ir_leaves: FlowLeaves<'a>, - pub(crate) ir_node: RefCell>, + flow_state: FlowState, + pub(crate) ir_node: RefCell, - _phantom: PhantomData<(&'a mut &'a (), T, N, W, C)>, + _phantom: PhantomData<(T, N, W, C)>, } -impl<'a, T, N: Location> CycleComplete<'a, Tick> for Stream<'a, T, Bounded, Tick, N> { +impl<'a, T, N: Location<'a>> CycleComplete<'a, Tick> for Stream { fn complete(self, ident: syn::Ident) { let me = self.defer_tick(); - me.ir_leaves.borrow_mut().0.as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled.").push(HfPlusLeaf::CycleSink { + me.flow_state.borrow_mut().leaves.as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled.").push(HfPlusLeaf::CycleSink { ident, location_kind: me.location_kind, input: Box::new(me.ir_node.into_inner()), @@ -66,13 +66,13 @@ impl<'a, T, N: Location> CycleComplete<'a, Tick> for Stream<'a, T, Bounded, Tick } } -impl<'a, T, N: Location> CycleCollection<'a, Tick> for Stream<'a, T, Bounded, Tick, N> { +impl<'a, T, N: Location<'a>> CycleCollection<'a, Tick> for Stream { type Location = N; - fn create_source(ident: syn::Ident, ir_leaves: FlowLeaves<'a>, l: LocationId) -> Self { + fn create_source(ident: syn::Ident, flow_state: FlowState, l: LocationId) -> Self { Stream::new( l, - ir_leaves, + flow_state, HfPlusNode::CycleSource { ident, location_kind: l, @@ -81,9 +81,9 @@ impl<'a, T, N: Location> CycleCollection<'a, Tick> for Stream<'a, T, Bounded, Ti } } -impl<'a, T, W, N: Location> CycleComplete<'a, NoTick> for Stream<'a, T, W, NoTick, N> { +impl<'a, T, W, N: Location<'a>> CycleComplete<'a, NoTick> for Stream { fn complete(self, ident: syn::Ident) { - self.ir_leaves.borrow_mut().0.as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled.").push(HfPlusLeaf::CycleSink { + self.flow_state.borrow_mut().leaves.as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled.").push(HfPlusLeaf::CycleSink { ident, location_kind: self.location_kind, input: Box::new(HfPlusNode::Unpersist(Box::new(self.ir_node.into_inner()))), @@ -91,13 +91,13 @@ impl<'a, T, W, N: Location> CycleComplete<'a, NoTick> for Stream<'a, T, W, NoTic } } -impl<'a, T, W, N: Location> CycleCollection<'a, NoTick> for Stream<'a, T, W, NoTick, N> { +impl<'a, T, W, N: Location<'a>> CycleCollection<'a, NoTick> for Stream { type Location = N; - fn create_source(ident: syn::Ident, ir_leaves: FlowLeaves<'a>, l: LocationId) -> Self { + fn create_source(ident: syn::Ident, flow_state: FlowState, l: LocationId) -> Self { Stream::new( l, - ir_leaves, + flow_state, HfPlusNode::Persist(Box::new(HfPlusNode::CycleSource { ident, location_kind: l, @@ -106,22 +106,22 @@ impl<'a, T, W, N: Location> CycleCollection<'a, NoTick> for Stream<'a, T, W, NoT } } -impl<'a, T, W, C, N: Location> Stream<'a, T, W, C, N> { +impl<'a, T, W, C, N: Location<'a>> Stream { pub(crate) fn new( location_kind: LocationId, - ir_leaves: FlowLeaves<'a>, - ir_node: HfPlusNode<'a>, + flow_state: FlowState, + ir_node: HfPlusNode, ) -> Self { Stream { location_kind, - ir_leaves, + flow_state, ir_node: RefCell::new(ir_node), _phantom: PhantomData, } } } -impl<'a, T: Clone, W, C, N: Location> Clone for Stream<'a, T, W, C, N> { +impl<'a, T: Clone, W, C, N: Location<'a>> Clone for Stream { fn clone(&self) -> Self { if !matches!(self.ir_node.borrow().deref(), HfPlusNode::Tee { .. }) { let orig_ir_node = self.ir_node.replace(HfPlusNode::Placeholder); @@ -133,7 +133,7 @@ impl<'a, T: Clone, W, C, N: Location> Clone for Stream<'a, T, W, C, N> { if let HfPlusNode::Tee { inner } = self.ir_node.borrow().deref() { Stream { location_kind: self.location_kind, - ir_leaves: self.ir_leaves.clone(), + flow_state: self.flow_state.clone(), ir_node: HfPlusNode::Tee { inner: inner.clone(), } @@ -146,14 +146,11 @@ impl<'a, T: Clone, W, C, N: Location> Clone for Stream<'a, T, W, C, N> { } } -impl<'a, T, W, C, N: Location> Stream<'a, T, W, C, N> { - pub fn map U + 'a>( - self, - f: impl IntoQuotedMut<'a, F>, - ) -> Stream<'a, U, W, C, N> { +impl<'a, T, W, C, N: Location<'a>> Stream { + pub fn map U + 'a>(self, f: impl IntoQuotedMut<'a, F>) -> Stream { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Map { f: f.splice_fn1().into(), input: Box::new(self.ir_node.into_inner()), @@ -164,10 +161,10 @@ impl<'a, T, W, C, N: Location> Stream<'a, T, W, C, N> { pub fn flat_map, F: Fn(T) -> I + 'a>( self, f: impl IntoQuotedMut<'a, F>, - ) -> Stream<'a, U, W, C, N> { + ) -> Stream { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::FlatMap { f: f.splice_fn1().into(), input: Box::new(self.ir_node.into_inner()), @@ -178,10 +175,10 @@ impl<'a, T, W, C, N: Location> Stream<'a, T, W, C, N> { pub fn filter bool + 'a>( self, f: impl IntoQuotedMut<'a, F>, - ) -> Stream<'a, T, W, C, N> { + ) -> Stream { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Filter { f: f.splice_fn1_borrow().into(), input: Box::new(self.ir_node.into_inner()), @@ -192,10 +189,10 @@ impl<'a, T, W, C, N: Location> Stream<'a, T, W, C, N> { pub fn filter_map Option + 'a>( self, f: impl IntoQuotedMut<'a, F>, - ) -> Stream<'a, U, W, C, N> { + ) -> Stream { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::FilterMap { f: f.splice_fn1().into(), input: Box::new(self.ir_node.into_inner()), @@ -205,19 +202,19 @@ impl<'a, T, W, C, N: Location> Stream<'a, T, W, C, N> { pub fn cross_singleton( self, - other: impl Into>, - ) -> Stream<'a, (T, O), W, C, N> + other: impl Into>, + ) -> Stream<(T, O), W, C, N> where O: Clone, { - let other: Optional<'a, O, Bounded, C, N> = other.into(); + let other: Optional = other.into(); if self.location_kind != other.location_kind { panic!("cross_singleton must be called on streams on the same node"); } Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::CrossSingleton( Box::new(self.ir_node.into_inner()), Box::new(other.ir_node.into_inner()), @@ -226,7 +223,7 @@ impl<'a, T, W, C, N: Location> Stream<'a, T, W, C, N> { } // TODO(shadaj): should allow for differing windows, using strongest one - pub fn cross_product(self, other: Stream<'a, O, W, C, N>) -> Stream<'a, (T, O), W, C, N> + pub fn cross_product(self, other: Stream) -> Stream<(T, O), W, C, N> where T: Clone, O: Clone, @@ -237,7 +234,7 @@ impl<'a, T, W, C, N: Location> Stream<'a, T, W, C, N> { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::CrossProduct( Box::new(self.ir_node.into_inner()), Box::new(other.ir_node.into_inner()), @@ -245,14 +242,14 @@ impl<'a, T, W, C, N: Location> Stream<'a, T, W, C, N> { ) } - pub fn union(self, other: Stream<'a, T, W, C, N>) -> Stream<'a, T, W, C, N> { + pub fn union(self, other: Stream) -> Stream { if self.location_kind != other.location_kind { panic!("union must be called on streams on the same node"); } Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Union( Box::new(self.ir_node.into_inner()), Box::new(other.ir_node.into_inner()), @@ -260,49 +257,49 @@ impl<'a, T, W, C, N: Location> Stream<'a, T, W, C, N> { ) } - pub fn unique(self) -> Stream<'a, T, W, C, N> + pub fn unique(self) -> Stream where T: Eq + Hash, { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Unique(Box::new(self.ir_node.into_inner())), ) } pub fn dest_sink + 'a>(self, sink: impl Quoted<'a, S>) { - self.ir_leaves.borrow_mut().0.as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled.").push(HfPlusLeaf::DestSink { + self.flow_state.borrow_mut().leaves.as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled.").push(HfPlusLeaf::DestSink { sink: sink.splice_typed().into(), input: Box::new(self.ir_node.into_inner()), }); } } -impl<'a, T, N: Location> Stream<'a, T, Bounded, Tick, N> { - pub fn all_ticks(self) -> Stream<'a, T, Unbounded, NoTick, N> { +impl<'a, T, N: Location<'a>> Stream { + pub fn all_ticks(self) -> Stream { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Persist(Box::new(self.ir_node.into_inner())), ) } - pub fn persist(self) -> Stream<'a, T, Bounded, Tick, N> + pub fn persist(self) -> Stream where T: Clone, { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Persist(Box::new(self.ir_node.into_inner())), ) } - pub fn defer_tick(self) -> Stream<'a, T, Bounded, Tick, N> { + pub fn defer_tick(self) -> Stream { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::DeferTick(Box::new(self.ir_node.into_inner())), ) } @@ -310,10 +307,10 @@ impl<'a, T, N: Location> Stream<'a, T, Bounded, Tick, N> { pub fn inspect( self, f: impl IntoQuotedMut<'a, F>, - ) -> Stream<'a, T, Bounded, Tick, N> { + ) -> Stream { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Inspect { f: f.splice_fn1_borrow().into(), input: Box::new(self.ir_node.into_inner()), @@ -321,10 +318,10 @@ impl<'a, T, N: Location> Stream<'a, T, Bounded, Tick, N> { ) } - pub fn first(self) -> Optional<'a, T, Bounded, Tick, N> { + pub fn first(self) -> Optional { Optional::new( self.location_kind, - self.ir_leaves, + self.flow_state, self.ir_node.into_inner(), ) } @@ -332,8 +329,8 @@ impl<'a, T, N: Location> Stream<'a, T, Bounded, Tick, N> { /// Allow this stream through if the other stream has elements, otherwise the output is empty. pub fn continue_if( self, - signal: Optional<'a, U, Bounded, Tick, N>, - ) -> Stream<'a, T, Bounded, Tick, N> { + signal: Optional, + ) -> Stream { self.cross_singleton(signal.map(q!(|_u| ()))) .map(q!(|(d, _signal)| d)) } @@ -341,15 +338,15 @@ impl<'a, T, N: Location> Stream<'a, T, Bounded, Tick, N> { /// Allow this stream through if the other stream is empty, otherwise the output is empty. pub fn continue_unless( self, - other: Optional<'a, U, Bounded, Tick, N>, - ) -> Stream<'a, T, Bounded, Tick, N> { + other: Optional, + ) -> Stream { self.continue_if(other.into_stream().count().filter(q!(|c| *c == 0))) } - pub fn enumerate(self) -> Stream<'a, (usize, T), Bounded, Tick, N> { + pub fn enumerate(self) -> Stream<(usize, T), Bounded, Tick, N> { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Enumerate(Box::new(self.ir_node.into_inner())), ) } @@ -358,10 +355,10 @@ impl<'a, T, N: Location> Stream<'a, T, Bounded, Tick, N> { self, init: impl IntoQuotedMut<'a, I>, comb: impl IntoQuotedMut<'a, F>, - ) -> Singleton<'a, A, Bounded, Tick, N> { + ) -> Singleton { Singleton::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Fold { init: init.splice_fn0().into(), acc: comb.splice_fn2_borrow_mut().into(), @@ -373,10 +370,10 @@ impl<'a, T, N: Location> Stream<'a, T, Bounded, Tick, N> { pub fn reduce( self, comb: impl IntoQuotedMut<'a, F>, - ) -> Optional<'a, T, Bounded, Tick, N> { + ) -> Optional { Optional::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Reduce { f: comb.splice_fn2_borrow_mut().into(), input: Box::new(self.ir_node.into_inner()), @@ -384,7 +381,7 @@ impl<'a, T, N: Location> Stream<'a, T, Bounded, Tick, N> { ) } - pub fn max(self) -> Optional<'a, T, Bounded, Tick, N> + pub fn max(self) -> Optional where T: Ord, { @@ -395,7 +392,7 @@ impl<'a, T, N: Location> Stream<'a, T, Bounded, Tick, N> { })) } - pub fn min(self) -> Optional<'a, T, Bounded, Tick, N> + pub fn min(self) -> Optional where T: Ord, { @@ -406,53 +403,50 @@ impl<'a, T, N: Location> Stream<'a, T, Bounded, Tick, N> { })) } - pub fn sort(self) -> Stream<'a, T, Bounded, Tick, N> + pub fn sort(self) -> Stream where T: Ord, { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Sort(Box::new(self.ir_node.into_inner())), ) } - pub fn count(self) -> Singleton<'a, usize, Bounded, Tick, N> { + pub fn count(self) -> Singleton { self.fold(q!(|| 0usize), q!(|count, _| *count += 1)) } - pub fn delta(self) -> Stream<'a, T, Bounded, Tick, N> { + pub fn delta(self) -> Stream { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Delta(Box::new(self.ir_node.into_inner())), ) } } -impl<'a, T, W, N: Location> Stream<'a, T, W, NoTick, N> { - pub fn tick_batch(self) -> Stream<'a, T, Bounded, Tick, N> { +impl<'a, T, W, N: Location<'a>> Stream { + pub fn tick_batch(self) -> Stream { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Unpersist(Box::new(self.ir_node.into_inner())), ) } - pub fn tick_prefix(self) -> Stream<'a, T, Bounded, Tick, N> + pub fn tick_prefix(self) -> Stream where T: Clone, { self.tick_batch().persist() } - pub fn inspect( - self, - f: impl IntoQuotedMut<'a, F>, - ) -> Stream<'a, T, W, NoTick, N> { + pub fn inspect(self, f: impl IntoQuotedMut<'a, F>) -> Stream { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Persist(Box::new(HfPlusNode::Inspect { f: f.splice_fn1_borrow().into(), input: Box::new(HfPlusNode::Unpersist(Box::new(self.ir_node.into_inner()))), @@ -461,25 +455,25 @@ impl<'a, T, W, N: Location> Stream<'a, T, W, NoTick, N> { } } -impl<'a, T, W, N: Location> Stream<'a, T, W, NoTick, N> { +impl<'a, T, W, N: Location<'a>> Stream { pub fn for_each(self, f: impl IntoQuotedMut<'a, F>) { - self.ir_leaves.borrow_mut().0.as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled.").push(HfPlusLeaf::ForEach { + self.flow_state.borrow_mut().leaves.as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled.").push(HfPlusLeaf::ForEach { input: Box::new(HfPlusNode::Unpersist(Box::new(self.ir_node.into_inner()))), f: f.splice_fn1().into(), }); } } -impl<'a, T, N: Location> Stream<'a, T, Unbounded, NoTick, N> { +impl<'a, T, N: Location<'a>> Stream { pub fn sample_every( self, duration: impl Quoted<'a, std::time::Duration> + Copy + 'a, - ) -> Stream<'a, T, Unbounded, NoTick, N> { + ) -> Stream { let interval = duration.splice_typed(); - let samples = Stream::<'a, tokio::time::Instant, Bounded, Tick, N>::new( + let samples = Stream::::new( self.location_kind, - self.ir_leaves.clone(), + self.flow_state.clone(), HfPlusNode::Source { source: HfPlusSource::Interval(interval.into()), location_kind: self.location_kind, @@ -493,13 +487,13 @@ impl<'a, T, N: Location> Stream<'a, T, Unbounded, NoTick, N> { self, init: impl IntoQuotedMut<'a, I>, comb: impl IntoQuotedMut<'a, F>, - ) -> Singleton<'a, A, Unbounded, NoTick, N> { + ) -> Singleton { // unbounded singletons are represented as a stream // which produces all values from all ticks every tick, // so delta will always give the lastest aggregation Singleton::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Persist(Box::new(HfPlusNode::Fold { init: init.splice_fn0().into(), acc: comb.splice_fn2_borrow_mut().into(), @@ -511,10 +505,10 @@ impl<'a, T, N: Location> Stream<'a, T, Unbounded, NoTick, N> { pub fn reduce( self, comb: impl IntoQuotedMut<'a, F>, - ) -> Optional<'a, T, Unbounded, NoTick, N> { + ) -> Optional { Optional::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Persist(Box::new(HfPlusNode::Reduce { f: comb.splice_fn2_borrow_mut().into(), input: Box::new(self.ir_node.into_inner()), @@ -522,7 +516,7 @@ impl<'a, T, N: Location> Stream<'a, T, Unbounded, NoTick, N> { ) } - pub fn max(self) -> Optional<'a, T, Unbounded, NoTick, N> + pub fn max(self) -> Optional where T: Ord, { @@ -533,7 +527,7 @@ impl<'a, T, N: Location> Stream<'a, T, Unbounded, NoTick, N> { })) } - pub fn min(self) -> Optional<'a, T, Unbounded, NoTick, N> + pub fn min(self) -> Optional where T: Ord, { @@ -545,8 +539,8 @@ impl<'a, T, N: Location> Stream<'a, T, Unbounded, NoTick, N> { } } -impl<'a, T, C, N: Location> Stream<'a, T, Bounded, C, N> { - pub fn filter_not_in(self, other: Stream<'a, T, Bounded, C, N>) -> Stream<'a, T, Bounded, C, N> +impl<'a, T, C, N: Location<'a>> Stream { + pub fn filter_not_in(self, other: Stream) -> Stream where T: Eq + Hash, { @@ -556,7 +550,7 @@ impl<'a, T, C, N: Location> Stream<'a, T, Bounded, C, N> { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Difference( Box::new(self.ir_node.into_inner()), Box::new(other.ir_node.into_inner()), @@ -565,18 +559,15 @@ impl<'a, T, C, N: Location> Stream<'a, T, Bounded, C, N> { } } -impl<'a, T: Clone, W, C, N: Location> Stream<'a, &T, W, C, N> { - pub fn cloned(self) -> Stream<'a, T, W, C, N> { +impl<'a, T: Clone, W, C, N: Location<'a>> Stream<&T, W, C, N> { + pub fn cloned(self) -> Stream { self.map(q!(|d| d.clone())) } } -impl<'a, K, V1, W, C, N: Location> Stream<'a, (K, V1), W, C, N> { +impl<'a, K, V1, W, C, N: Location<'a>> Stream<(K, V1), W, C, N> { // TODO(shadaj): figure out window semantics - pub fn join( - self, - n: Stream<'a, (K, V2), W2, C, N>, - ) -> Stream<'a, (K, (V1, V2)), W, C, N> + pub fn join(self, n: Stream<(K, V2), W2, C, N>) -> Stream<(K, (V1, V2)), W, C, N> where K: Eq + Hash, { @@ -586,7 +577,7 @@ impl<'a, K, V1, W, C, N: Location> Stream<'a, (K, V1), W, C, N> { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::Join( Box::new(self.ir_node.into_inner()), Box::new(n.ir_node.into_inner()), @@ -594,7 +585,7 @@ impl<'a, K, V1, W, C, N: Location> Stream<'a, (K, V1), W, C, N> { ) } - pub fn anti_join(self, n: Stream<'a, K, W2, C, N>) -> Stream<'a, (K, V1), W, C, N> + pub fn anti_join(self, n: Stream) -> Stream<(K, V1), W, C, N> where K: Eq + Hash, { @@ -604,7 +595,7 @@ impl<'a, K, V1, W, C, N: Location> Stream<'a, (K, V1), W, C, N> { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::AntiJoin( Box::new(self.ir_node.into_inner()), Box::new(n.ir_node.into_inner()), @@ -613,15 +604,15 @@ impl<'a, K, V1, W, C, N: Location> Stream<'a, (K, V1), W, C, N> { } } -impl<'a, K: Eq + Hash, V, N: Location> Stream<'a, (K, V), Bounded, Tick, N> { +impl<'a, K: Eq + Hash, V, N: Location<'a>> Stream<(K, V), Bounded, Tick, N> { pub fn fold_keyed A + 'a, F: Fn(&mut A, V) + 'a>( self, init: impl IntoQuotedMut<'a, I>, comb: impl IntoQuotedMut<'a, F>, - ) -> Stream<'a, (K, A), Bounded, Tick, N> { + ) -> Stream<(K, A), Bounded, Tick, N> { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::FoldKeyed { init: init.splice_fn0().into(), acc: comb.splice_fn2_borrow_mut().into(), @@ -633,10 +624,10 @@ impl<'a, K: Eq + Hash, V, N: Location> Stream<'a, (K, V), Bounded, Tick, N> { pub fn reduce_keyed( self, comb: impl IntoQuotedMut<'a, F>, - ) -> Stream<'a, (K, V), Bounded, Tick, N> { + ) -> Stream<(K, V), Bounded, Tick, N> { Stream::new( self.location_kind, - self.ir_leaves, + self.flow_state, HfPlusNode::ReduceKeyed { f: comb.splice_fn2_borrow_mut().into(), input: Box::new(self.ir_node.into_inner()), @@ -698,13 +689,13 @@ pub(super) fn deserialize_bincode(tagged: bool) -> Pipeline } } -impl<'a, T, W, N: Location> Stream<'a, T, W, NoTick, N> { - pub fn send_bincode( +impl<'a, T, W, N: Location<'a>> Stream { + pub fn send_bincode, CoreType>( self, other: &N2, - ) -> Stream<'a, N::Out, Unbounded, NoTick, N2> + ) -> Stream, Unbounded, NoTick, N2> where - N: CanSend = T>, + N: CanSend<'a, N2, In = T>, CoreType: Serialize + DeserializeOwned, { let serialize_pipeline = Some(serialize_bincode::(N::is_demux())); @@ -713,7 +704,7 @@ impl<'a, T, W, N: Location> Stream<'a, T, W, NoTick, N> { Stream::new( other.id(), - self.ir_leaves, + self.flow_state, HfPlusNode::Network { from_location: self.location_kind, from_key: None, @@ -727,23 +718,23 @@ impl<'a, T, W, N: Location> Stream<'a, T, W, NoTick, N> { ) } - pub fn send_bincode_external( + pub fn send_bincode_external( self, other: &ExternalProcess, ) -> ExternalBincodeStream> where - N: CanSend, In = T, Out = CoreType>, + N: CanSend<'a, ExternalProcess<'a, N2>, In = T, Out = CoreType>, CoreType: Serialize + DeserializeOwned, // for now, we restirct Out to be CoreType, which means no tagged cluster -> external { let serialize_pipeline = Some(serialize_bincode::(N::is_demux())); - let mut leaves_borrow = self.ir_leaves.borrow_mut(); + let mut flow_state_borrow = self.flow_state.borrow_mut(); - let external_key = leaves_borrow.1; - leaves_borrow.1 += 1; + let external_key = flow_state_borrow.next_external_out; + flow_state_borrow.next_external_out += 1; - let leaves = leaves_borrow.0.as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled()"); + let leaves = flow_state_borrow.leaves.as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled()"); let dummy_f: syn::Expr = syn::parse_quote!(()); @@ -768,16 +759,16 @@ impl<'a, T, W, N: Location> Stream<'a, T, W, NoTick, N> { } } - pub fn send_bytes( + pub fn send_bytes>( self, other: &N2, - ) -> Stream<'a, N::Out, Unbounded, NoTick, N2> + ) -> Stream, Unbounded, NoTick, N2> where - N: CanSend = T>, + N: CanSend<'a, N2, In = T>, { Stream::new( other.id(), - self.ir_leaves, + self.flow_state, HfPlusNode::Network { from_location: self.location_kind, from_key: None, @@ -795,15 +786,15 @@ impl<'a, T, W, N: Location> Stream<'a, T, W, NoTick, N> { ) } - pub fn send_bytes_external(self, other: &ExternalProcess) -> ExternalBytesPort + pub fn send_bytes_external(self, other: &ExternalProcess) -> ExternalBytesPort where - N: CanSend, In = T, Out = Bytes>, + N: CanSend<'a, ExternalProcess<'a, N2>, In = T, Out = Bytes>, { - let mut leaves_borrow = self.ir_leaves.borrow_mut(); - let external_key = leaves_borrow.1; - leaves_borrow.1 += 1; + let mut flow_state_borrow = self.flow_state.borrow_mut(); + let external_key = flow_state_borrow.next_external_out; + flow_state_borrow.next_external_out += 1; - let leaves = leaves_borrow.0.as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled()"); + let leaves = flow_state_borrow.leaves.as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled()"); let dummy_f: syn::Expr = syn::parse_quote!(()); @@ -827,33 +818,33 @@ impl<'a, T, W, N: Location> Stream<'a, T, W, NoTick, N> { } } - pub fn send_bincode_interleaved( + pub fn send_bincode_interleaved, Tag, CoreType>( self, other: &N2, - ) -> Stream<'a, CoreType, Unbounded, NoTick, N2> + ) -> Stream where - N: CanSend = T, Out = (Tag, CoreType)>, + N: CanSend<'a, N2, In = T, Out = (Tag, CoreType)>, CoreType: Serialize + DeserializeOwned, { self.send_bincode::(other).map(q!(|(_, b)| b)) } - pub fn send_bytes_interleaved( + pub fn send_bytes_interleaved, Tag>( self, other: &N2, - ) -> Stream<'a, Bytes, Unbounded, NoTick, N2> + ) -> Stream where - N: CanSend = T, Out = (Tag, Bytes)>, + N: CanSend<'a, N2, In = T, Out = (Tag, Bytes)>, { self.send_bytes::(other).map(q!(|(_, b)| b)) } pub fn broadcast_bincode( self, - other: &Cluster, - ) -> Stream<'a, N::Out, Unbounded, NoTick, Cluster> + other: &Cluster<'a, C2>, + ) -> Stream, Unbounded, NoTick, Cluster<'a, C2>> where - N: CanSend, In = (u32, T)>, + N: CanSend<'a, Cluster<'a, C2>, In = (u32, T)>, T: Clone + Serialize + DeserializeOwned, { let ids = other.members(); @@ -867,10 +858,10 @@ impl<'a, T, W, N: Location> Stream<'a, T, W, NoTick, N> { pub fn broadcast_bincode_interleaved( self, - other: &Cluster, - ) -> Stream<'a, T, Unbounded, NoTick, Cluster> + other: &Cluster<'a, C2>, + ) -> Stream> where - N: CanSend, In = (u32, T), Out = (Tag, T)> + 'a, + N: CanSend<'a, Cluster<'a, C2>, In = (u32, T), Out = (Tag, T)> + 'a, T: Clone + Serialize + DeserializeOwned, { self.broadcast_bincode(other).map(q!(|(_, b)| b)) @@ -878,10 +869,10 @@ impl<'a, T, W, N: Location> Stream<'a, T, W, NoTick, N> { pub fn broadcast_bytes( self, - other: &Cluster, - ) -> Stream<'a, N::Out, Unbounded, NoTick, Cluster> + other: &Cluster<'a, C2>, + ) -> Stream, Unbounded, NoTick, Cluster<'a, C2>> where - N: CanSend, In = (u32, T)> + 'a, + N: CanSend<'a, Cluster<'a, C2>, In = (u32, T)> + 'a, T: Clone, { let ids = other.members(); @@ -895,10 +886,10 @@ impl<'a, T, W, N: Location> Stream<'a, T, W, NoTick, N> { pub fn broadcast_bytes_interleaved( self, - other: &Cluster, - ) -> Stream<'a, Bytes, Unbounded, NoTick, Cluster> + other: &Cluster<'a, C2>, + ) -> Stream> where - N: CanSend, In = (u32, T), Out = (Tag, Bytes)> + 'a, + N: CanSend<'a, Cluster<'a, C2>, In = (u32, T), Out = (Tag, Bytes)> + 'a, T: Clone, { self.broadcast_bytes(other).map(q!(|(_, b)| b)) @@ -915,6 +906,7 @@ mod tests { use stageleft::q; use crate::deploy::TrybuildHost; + use crate::location::Location; use crate::FlowBuilder; struct P1 {} @@ -934,7 +926,7 @@ mod tests { let second_node = flow.process::(); let external = flow.external_process::(); - let numbers = flow.source_iter(&first_node, q!(0..10)); + let numbers = first_node.source_iter(q!(0..10)); let out_port = numbers .map(q!(|n| SendOverNetwork { n })) .send_bincode(&second_node) diff --git a/hydroflow_plus_test/src/cluster/compute_pi.rs b/hydroflow_plus_test/src/cluster/compute_pi.rs index 0be1e74b64f9..106c39fecfdc 100644 --- a/hydroflow_plus_test/src/cluster/compute_pi.rs +++ b/hydroflow_plus_test/src/cluster/compute_pi.rs @@ -6,12 +6,15 @@ use stageleft::*; pub struct Worker {} pub struct Leader {} -pub fn compute_pi(flow: &FlowBuilder, batch_size: usize) -> (Cluster, Process) { +pub fn compute_pi<'a>( + flow: &FlowBuilder<'a>, + batch_size: usize, +) -> (Cluster<'a, Worker>, Process<'a, Leader>) { let cluster = flow.cluster(); let process = flow.process(); - let trials = flow - .spin_batch(&cluster, q!(batch_size)) + let trials = cluster + .spin_batch(q!(batch_size)) .map(q!(|_| rand::random::<(f64, f64)>())) .map(q!(|(x, y)| x * x + y * y < 1.0)) .fold( diff --git a/hydroflow_plus_test/src/cluster/many_to_many.rs b/hydroflow_plus_test/src/cluster/many_to_many.rs index c3e35b85b5cb..e3aa96870a5e 100644 --- a/hydroflow_plus_test/src/cluster/many_to_many.rs +++ b/hydroflow_plus_test/src/cluster/many_to_many.rs @@ -1,9 +1,10 @@ use hydroflow_plus::*; use stageleft::*; -pub fn many_to_many(flow: &FlowBuilder) -> Cluster<()> { +pub fn many_to_many<'a>(flow: &FlowBuilder<'a>) -> Cluster<'a, ()> { let cluster = flow.cluster(); - flow.source_iter(&cluster, q!(0..2)) + cluster + .source_iter(q!(0..2)) .broadcast_bincode(&cluster) .for_each(q!(|n| println!("cluster received: {:?}", n))); diff --git a/hydroflow_plus_test/src/cluster/map_reduce.rs b/hydroflow_plus_test/src/cluster/map_reduce.rs index b3a87d25c733..1e76c1fc14eb 100644 --- a/hydroflow_plus_test/src/cluster/map_reduce.rs +++ b/hydroflow_plus_test/src/cluster/map_reduce.rs @@ -4,12 +4,12 @@ use stageleft::*; pub struct Leader {} pub struct Worker {} -pub fn map_reduce(flow: &FlowBuilder) -> (Process, Cluster) { +pub fn map_reduce<'a>(flow: &FlowBuilder<'a>) -> (Process<'a, Leader>, Cluster<'a, Worker>) { let process = flow.process(); let cluster = flow.cluster(); - let words = flow - .source_iter(&process, q!(vec!["abc", "abc", "xyz", "abc"])) + let words = process + .source_iter(q!(vec!["abc", "abc", "xyz", "abc"])) .map(q!(|s| s.to_string())); let all_ids_vec = cluster.members(); diff --git a/hydroflow_plus_test/src/cluster/paxos.rs b/hydroflow_plus_test/src/cluster/paxos.rs index 352f724df7ea..9625762870c3 100644 --- a/hydroflow_plus_test/src/cluster/paxos.rs +++ b/hydroflow_plus_test/src/cluster/paxos.rs @@ -2,6 +2,7 @@ use std::collections::HashMap; use std::time::Duration; use hydroflow_plus::*; +use location::Location; use serde::de::DeserializeOwned; use serde::{Deserialize, Serialize}; use stageleft::*; @@ -67,26 +68,25 @@ struct P2b

{ pub fn paxos_core<'a, P: PaxosPayload>( flow: &FlowBuilder<'a>, r_to_acceptors_checkpoint: impl FnOnce( - &Cluster, + &Cluster<'a, Acceptor>, ) -> Stream< - 'a, (u32, i32), Unbounded, NoTick, - Cluster, + Cluster<'a, Acceptor>, >, c_to_proposers: impl FnOnce( - &Cluster, - ) -> Stream<'a, P, Unbounded, NoTick, Cluster>, + &Cluster<'a, Proposer>, + ) -> Stream>, f: usize, i_am_leader_send_timeout: u64, i_am_leader_check_timeout: u64, i_am_leader_check_timeout_delay_multiplier: usize, ) -> ( - Cluster, - Cluster, - Stream<'a, Ballot, Unbounded, NoTick, Cluster>, - Stream<'a, (i32, P), Unbounded, NoTick, Cluster>, + Cluster<'a, Proposer>, + Cluster<'a, Acceptor>, + Stream>, + Stream<(i32, P), Unbounded, NoTick, Cluster<'a, Proposer>>, ) { let proposers = flow.cluster::(); let acceptors = flow.cluster::(); @@ -94,29 +94,29 @@ pub fn paxos_core<'a, P: PaxosPayload>( let c_to_proposers = c_to_proposers(&proposers); // Proposers. - flow.source_iter(&proposers, q!(["Proposers say hello"])) + proposers + .source_iter(q!(["Proposers say hello"])) .for_each(q!(|s| println!("{}", s))); let p_id = proposers.self_id(); let (p_to_proposers_i_am_leader_complete_cycle, p_to_proposers_i_am_leader) = - flow.cycle::>(&proposers); + proposers.cycle::>(); let (a_to_proposers_p1b_complete_cycle, a_to_proposers_p1b) = - flow.cycle::>(&proposers); + proposers.cycle::>(); let (a_to_proposers_p2b_complete_cycle, a_to_proposers_p2b) = - flow.cycle::>(&proposers); + proposers.cycle::>(); // a_to_proposers_p2b.clone().for_each(q!(|(_, p2b): (u32, P2b)| println!("Proposer received P2b: {:?}", p2b))); // p_to_proposers_i_am_leader.clone().for_each(q!(|ballot: Ballot| println!("Proposer received I am leader: {:?}", ballot))); // c_to_proposers.clone().for_each(q!(|payload: ClientPayload| println!("Client sent proposer payload: {:?}", payload))); let p_received_max_ballot = p_max_ballot( - flow, &proposers, a_to_proposers_p1b.clone(), a_to_proposers_p2b.clone(), p_to_proposers_i_am_leader.clone(), ); let (p_ballot_num, p_has_largest_ballot) = - p_ballot_calc(flow, &proposers, p_received_max_ballot.latest_tick()); + p_ballot_calc(&proposers, p_received_max_ballot.latest_tick()); let (p_is_leader, p_log_to_try_commit, p_max_slot, p_log_holes) = p_p1b( &proposers, @@ -131,7 +131,6 @@ pub fn paxos_core<'a, P: PaxosPayload>( p_is_leader.clone(), &proposers, p_to_proposers_i_am_leader, - flow, &acceptors, i_am_leader_send_timeout, i_am_leader_check_timeout, @@ -140,7 +139,6 @@ pub fn paxos_core<'a, P: PaxosPayload>( p_to_proposers_i_am_leader_complete_cycle.complete(p_to_proposers_i_am_leader_from_others); let (p_next_slot, p_to_acceptors_p2a) = p_p2a( - flow, &proposers, p_max_slot, c_to_proposers, @@ -158,16 +156,16 @@ pub fn paxos_core<'a, P: PaxosPayload>( .map(q!(move |(_is_leader, ballot_num)| Ballot { num: ballot_num, id: p_id})) // Only tell the clients once when leader election concludes .all_ticks(); // End tell clients that leader election has completed - let p_to_replicas = p_p2b(flow, &proposers, a_to_proposers_p2b, f); + let p_to_replicas = p_p2b(&proposers, a_to_proposers_p2b, f); // Acceptors. - flow.source_iter(&acceptors, q!(["Acceptors say hello"])) + acceptors + .source_iter(q!(["Acceptors say hello"])) .for_each(q!(|s| println!("{}", s))); let r_to_acceptors_checkpoint = r_to_acceptors_checkpoint(&acceptors); // p_to_acceptors_p2a.clone().for_each(q!(|p2a: P2a| println!("Acceptor received P2a: {:?}", p2a))); let (a_to_proposers_p1b_new, a_to_proposers_p2b_new) = acceptor( - flow, p_to_acceptors_p1a.inspect(q!(|p1a| println!("Acceptor received P1a: {:?}", p1a))), p_to_acceptors_p2a, r_to_acceptors_checkpoint, @@ -188,16 +186,15 @@ pub fn paxos_core<'a, P: PaxosPayload>( #[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] fn acceptor<'a, P: PaxosPayload>( - flow: &FlowBuilder<'a>, - p_to_acceptors_p1a: Stream<'a, P1a, Unbounded, NoTick, Cluster>, - p_to_acceptors_p2a: Stream<'a, P2a

, Unbounded, NoTick, Cluster>, - r_to_acceptors_checkpoint: Stream<'a, (u32, i32), Unbounded, NoTick, Cluster>, - proposers: &Cluster, - acceptors: &Cluster, + p_to_acceptors_p1a: Stream>, + p_to_acceptors_p2a: Stream, Unbounded, NoTick, Cluster<'a, Acceptor>>, + r_to_acceptors_checkpoint: Stream<(u32, i32), Unbounded, NoTick, Cluster<'a, Acceptor>>, + proposers: &Cluster<'a, Proposer>, + acceptors: &Cluster<'a, Acceptor>, f: usize, ) -> ( - Stream<'a, (u32, P1b

), Unbounded, NoTick, Cluster>, - Stream<'a, (u32, P2b

), Unbounded, NoTick, Cluster>, + Stream<(u32, P1b

), Unbounded, NoTick, Cluster<'a, Proposer>>, + Stream<(u32, P2b

), Unbounded, NoTick, Cluster<'a, Proposer>>, ) { // Get the latest checkpoint sequence per replica let a_checkpoint_largest_seqs = @@ -220,7 +217,7 @@ fn acceptor<'a, P: PaxosPayload>( .continue_if(a_checkpoints_quorum_reached) .map(q!(|(_sender, seq)| seq)) .min() - .unwrap_or(flow.singleton(acceptors, q!(-1)).latest_tick()) + .unwrap_or(acceptors.singleton(q!(-1)).latest_tick()) .delta() .map(q!(|min_seq| ( min_seq, @@ -237,7 +234,7 @@ fn acceptor<'a, P: PaxosPayload>( .clone() .map(q!(|p1a| p1a.ballot)) .max() - .unwrap_or(flow.singleton(acceptors, q!(Ballot { num: 0, id: 0 }))); + .unwrap_or(acceptors.singleton(q!(Ballot { num: 0, id: 0 }))); let a_p2as_to_place_in_log = p_to_acceptors_p2a .clone() .tick_batch() @@ -323,14 +320,12 @@ fn acceptor<'a, P: PaxosPayload>( } fn p_p2b<'a, P: PaxosPayload>( - flow: &FlowBuilder<'a>, - proposers: &Cluster, - a_to_proposers_p2b: Stream<'a, (u32, P2b

), Unbounded, NoTick, Cluster>, + proposers: &Cluster<'a, Proposer>, + a_to_proposers_p2b: Stream<(u32, P2b

), Unbounded, NoTick, Cluster<'a, Proposer>>, f: usize, -) -> Stream<'a, (i32, P), Unbounded, NoTick, Cluster> { - let (p_broadcasted_p2b_slots_complete_cycle, p_broadcasted_p2b_slots) = - flow.tick_cycle(proposers); - let (p_persisted_p2bs_complete_cycle, p_persisted_p2bs) = flow.tick_cycle(proposers); +) -> Stream<(i32, P), Unbounded, NoTick, Cluster<'a, Proposer>> { + let (p_broadcasted_p2b_slots_complete_cycle, p_broadcasted_p2b_slots) = proposers.tick_cycle(); + let (p_persisted_p2bs_complete_cycle, p_persisted_p2bs) = proposers.tick_cycle(); let p_p2b = a_to_proposers_p2b .clone() .tick_batch() @@ -399,24 +394,23 @@ fn p_p2b<'a, P: PaxosPayload>( reason = "internal paxos code // TODO" )] fn p_p2a<'a, P: PaxosPayload>( - flow: &FlowBuilder<'a>, - proposers: &Cluster, - p_max_slot: Optional<'a, i32, Bounded, Tick, Cluster>, - c_to_proposers: Stream<'a, P, Unbounded, NoTick, Cluster>, - p_ballot_num: Singleton<'a, u32, Bounded, Tick, Cluster>, - p_log_to_try_commit: Stream<'a, P2a

, Bounded, Tick, Cluster>, - p_log_holes: Stream<'a, P2a

, Bounded, Tick, Cluster>, - p_is_leader: Optional<'a, bool, Bounded, Tick, Cluster>, - acceptors: &Cluster, + proposers: &Cluster<'a, Proposer>, + p_max_slot: Optional>, + c_to_proposers: Stream>, + p_ballot_num: Singleton>, + p_log_to_try_commit: Stream, Bounded, Tick, Cluster<'a, Proposer>>, + p_log_holes: Stream, Bounded, Tick, Cluster<'a, Proposer>>, + p_is_leader: Optional>, + acceptors: &Cluster<'a, Acceptor>, ) -> ( - Optional<'a, i32, Bounded, Tick, Cluster>, - Stream<'a, P2a

, Unbounded, NoTick, Cluster>, + Optional>, + Stream, Unbounded, NoTick, Cluster<'a, Acceptor>>, ) { let p_id = proposers.self_id(); let (p_next_slot_complete_cycle, p_next_slot) = - flow.tick_cycle::>(proposers); + proposers.tick_cycle::>(); let p_next_slot_after_reconciling_p1bs = p_max_slot - .unwrap_or(flow.singleton(proposers, q!(-1)).latest_tick()) + .unwrap_or(proposers.singleton(q!(-1)).latest_tick()) // .inspect(q!(|max_slot| println!("{} p_max_slot: {:?}", context.current_tick(), max_slot))) .continue_unless(p_next_slot.clone()) .map(q!(|max_slot| max_slot + 1)); @@ -472,16 +466,16 @@ fn p_p2a<'a, P: PaxosPayload>( // Proposer logic for processing p1bs, determining if the proposer is now the leader, which uncommitted messages to commit, what the maximum slot is in the p1bs, and which no-ops to commit to fill log holes. #[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] fn p_p1b<'a, P: PaxosPayload>( - proposers: &Cluster, - a_to_proposers_p1b: Stream<'a, (u32, P1b

), Unbounded, NoTick, Cluster>, - p_ballot_num: Singleton<'a, u32, Bounded, Tick, Cluster>, - p_has_largest_ballot: Optional<'a, (Ballot, u32), Bounded, Tick, Cluster>, + proposers: &Cluster<'a, Proposer>, + a_to_proposers_p1b: Stream<(u32, P1b

), Unbounded, NoTick, Cluster<'a, Proposer>>, + p_ballot_num: Singleton>, + p_has_largest_ballot: Optional<(Ballot, u32), Bounded, Tick, Cluster<'a, Proposer>>, f: usize, ) -> ( - Optional<'a, bool, Bounded, Tick, Cluster>, - Stream<'a, P2a

, Bounded, Tick, Cluster>, - Optional<'a, i32, Bounded, Tick, Cluster>, - Stream<'a, P2a

, Bounded, Tick, Cluster>, + Optional>, + Stream, Bounded, Tick, Cluster<'a, Proposer>>, + Optional>, + Stream, Bounded, Tick, Cluster<'a, Proposer>>, ) { let p_id = proposers.self_id(); let p_relevant_p1bs = a_to_proposers_p1b @@ -567,12 +561,11 @@ fn p_p1b<'a, P: PaxosPayload>( // Proposer logic to calculate the largest ballot received so far. fn p_max_ballot<'a, P: PaxosPayload>( - flow: &FlowBuilder<'a>, - proposers: &Cluster, - a_to_proposers_p1b: Stream<'a, (u32, P1b

), Unbounded, NoTick, Cluster>, - a_to_proposers_p2b: Stream<'a, (u32, P2b

), Unbounded, NoTick, Cluster>, - p_to_proposers_i_am_leader: Stream<'a, Ballot, Unbounded, NoTick, Cluster>, -) -> Singleton<'a, Ballot, Unbounded, NoTick, Cluster> { + proposers: &Cluster<'a, Proposer>, + a_to_proposers_p1b: Stream<(u32, P1b

), Unbounded, NoTick, Cluster<'a, Proposer>>, + a_to_proposers_p2b: Stream<(u32, P2b

), Unbounded, NoTick, Cluster<'a, Proposer>>, + p_to_proposers_i_am_leader: Stream>, +) -> Singleton> { let p_received_p1b_ballots = a_to_proposers_p1b .clone() .map(q!(|(_, p1b)| p1b.max_ballot)); @@ -583,22 +576,21 @@ fn p_max_ballot<'a, P: PaxosPayload>( .union(p_received_p2b_ballots) .union(p_to_proposers_i_am_leader) .max() - .unwrap_or(flow.singleton(proposers, q!(Ballot { num: 0, id: 0 }))) + .unwrap_or(proposers.singleton(q!(Ballot { num: 0, id: 0 }))) } // Proposer logic to calculate the next ballot number. Expects p_received_max_ballot, the largest ballot received so far. Outputs streams: ballot_num, and has_largest_ballot, which only contains a value if we have the largest ballot. #[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] fn p_ballot_calc<'a>( - flow: &FlowBuilder<'a>, - proposers: &Cluster, - p_received_max_ballot: Singleton<'a, Ballot, Bounded, Tick, Cluster>, + proposers: &Cluster<'a, Proposer>, + p_received_max_ballot: Singleton>, ) -> ( - Singleton<'a, u32, Bounded, Tick, Cluster>, - Optional<'a, (Ballot, u32), Bounded, Tick, Cluster>, + Singleton>, + Optional<(Ballot, u32), Bounded, Tick, Cluster<'a, Proposer>>, ) { let p_id = proposers.self_id(); let (p_ballot_num_complete_cycle, p_ballot_num) = - flow.tick_cycle_with_initial(proposers, flow.singleton(proposers, q!(0)).latest_tick()); + proposers.tick_cycle_with_initial(proposers.singleton(q!(0)).latest_tick()); let p_new_ballot_num = p_received_max_ballot .clone() @@ -639,24 +631,24 @@ fn p_ballot_calc<'a>( reason = "internal paxos code // TODO" )] fn p_p1a<'a>( - p_ballot_num: Singleton<'a, u32, Bounded, Tick, Cluster>, - p_is_leader: Optional<'a, bool, Bounded, Tick, Cluster>, - proposers: &Cluster, - p_to_proposers_i_am_leader: Stream<'a, Ballot, Unbounded, NoTick, Cluster>, - flow: &FlowBuilder<'a>, - acceptors: &Cluster, + p_ballot_num: Singleton>, + p_is_leader: Optional>, + proposers: &Cluster<'a, Proposer>, + p_to_proposers_i_am_leader: Stream>, + acceptors: &Cluster<'a, Acceptor>, i_am_leader_send_timeout: u64, // How often to heartbeat i_am_leader_check_timeout: u64, // How often to check if heartbeat expired i_am_leader_check_timeout_delay_multiplier: usize, /* Initial delay, multiplied by proposer pid, to stagger proposers checking for timeouts */ ) -> ( - Stream<'a, Ballot, Unbounded, NoTick, Cluster>, - Stream<'a, P1a, Unbounded, NoTick, Cluster>, + Stream>, + Stream>, ) { let p_id = proposers.self_id(); let p_to_proposers_i_am_leader_new = p_ballot_num .clone() .continue_if( - flow.source_interval(proposers, q!(Duration::from_secs(i_am_leader_send_timeout))) + proposers + .source_interval(q!(Duration::from_secs(i_am_leader_send_timeout))) .latest_tick(), ) .continue_if(p_is_leader.clone()) @@ -675,7 +667,7 @@ fn p_p1a<'a>( }), ); // Add random delay depending on node ID so not everyone sends p1a at the same time - let p_leader_expired = flow.source_interval_delayed(proposers, q!(Duration::from_secs((p_id * i_am_leader_check_timeout_delay_multiplier as u32).into())), q!(Duration::from_secs(i_am_leader_check_timeout))) + let p_leader_expired = proposers.source_interval_delayed(q!(Duration::from_secs((p_id * i_am_leader_check_timeout_delay_multiplier as u32).into())), q!(Duration::from_secs(i_am_leader_check_timeout))) .cross_singleton(p_latest_received_i_am_leader.clone()) .latest_tick() // .inspect(q!(|v| println!("Proposer checking if leader expired"))) diff --git a/hydroflow_plus_test/src/cluster/paxos_bench.rs b/hydroflow_plus_test/src/cluster/paxos_bench.rs index 75e0627bcd1b..fdf76683b706 100644 --- a/hydroflow_plus_test/src/cluster/paxos_bench.rs +++ b/hydroflow_plus_test/src/cluster/paxos_bench.rs @@ -45,8 +45,8 @@ impl PaxosPayload for ClientPayload {} // Important: By convention, all relations that represent booleans either have a single "true" value or nothing. // This allows us to use the continue_if_exists() and continue_if_empty() operators as if they were if (true) and if (false) statements. #[expect(clippy::too_many_arguments, reason = "internal paxos code // TODO")] -pub fn paxos_bench( - flow: &FlowBuilder, +pub fn paxos_bench<'a>( + flow: &FlowBuilder<'a>, f: usize, num_clients_per_node: usize, median_latency_window_size: usize, /* How many latencies to keep in the window for calculating the median */ @@ -55,15 +55,15 @@ pub fn paxos_bench( i_am_leader_check_timeout: u64, // How often to check if heartbeat expired i_am_leader_check_timeout_delay_multiplier: usize, /* Initial delay, multiplied by proposer pid, to stagger proposers checking for timeouts */ ) -> ( - Cluster, - Cluster, - Cluster, - Cluster, + Cluster<'a, Proposer>, + Cluster<'a, Acceptor>, + Cluster<'a, Client>, + Cluster<'a, Replica>, ) { let clients = flow.cluster::(); let replicas = flow.cluster::(); - let (c_to_proposers_complete_cycle, c_to_proposers) = flow.cycle(&clients); + let (c_to_proposers_complete_cycle, c_to_proposers) = clients.cycle(); let (proposers, acceptors, p_to_clients_new_leader_elected, r_new_processed_payloads) = paxos_with_replica( @@ -81,7 +81,6 @@ pub fn paxos_bench( &clients, p_to_clients_new_leader_elected.broadcast_bincode_interleaved(&clients), r_new_processed_payloads.send_bincode(&clients), - flow, num_clients_per_node, median_latency_window_size, f, @@ -97,21 +96,21 @@ pub fn paxos_bench( )] fn paxos_with_replica<'a>( flow: &FlowBuilder<'a>, - replicas: &Cluster, - c_to_proposers: Stream<'a, (u32, ClientPayload), Unbounded, NoTick, Cluster>, + replicas: &Cluster<'a, Replica>, + c_to_proposers: Stream<(u32, ClientPayload), Unbounded, NoTick, Cluster<'a, Client>>, f: usize, i_am_leader_send_timeout: u64, i_am_leader_check_timeout: u64, i_am_leader_check_timeout_delay_multiplier: usize, checkpoint_frequency: usize, ) -> ( - Cluster, - Cluster, - Stream<'a, Ballot, Unbounded, NoTick, Cluster>, - Stream<'a, (u32, ReplicaPayload), Unbounded, NoTick, Cluster>, + Cluster<'a, Proposer>, + Cluster<'a, Acceptor>, + Stream>, + Stream<(u32, ReplicaPayload), Unbounded, NoTick, Cluster<'a, Replica>>, ) { let (r_to_acceptors_checkpoint_complete_cycle, r_to_acceptors_checkpoint) = - flow.cycle::>(replicas); + replicas.cycle::>(); let (proposers, acceptors, p_to_clients_new_leader_elected, p_to_replicas) = paxos_core( flow, @@ -124,7 +123,6 @@ fn paxos_with_replica<'a>( ); let (r_to_acceptors_checkpoint_new, r_new_processed_payloads) = replica( - flow, replicas, p_to_replicas .map(q!(|(slot, data)| ReplicaPayload { @@ -149,15 +147,14 @@ fn paxos_with_replica<'a>( // Replicas. All relations for replicas will be prefixed with r. Expects ReplicaPayload on p_to_replicas, outputs a stream of (client address, ReplicaPayload) after processing. #[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] pub fn replica<'a>( - flow: &FlowBuilder<'a>, - replicas: &Cluster, - p_to_replicas: Stream<'a, ReplicaPayload, Unbounded, NoTick, Cluster>, + replicas: &Cluster<'a, Replica>, + p_to_replicas: Stream>, checkpoint_frequency: usize, ) -> ( - Stream<'a, i32, Unbounded, NoTick, Cluster>, - Stream<'a, (u32, ReplicaPayload), Unbounded, NoTick, Cluster>, + Stream>, + Stream<(u32, ReplicaPayload), Unbounded, NoTick, Cluster<'a, Replica>>, ) { - let (r_buffered_payloads_complete_cycle, r_buffered_payloads) = flow.tick_cycle(replicas); + let (r_buffered_payloads_complete_cycle, r_buffered_payloads) = replicas.tick_cycle(); // p_to_replicas.inspect(q!(|payload: ReplicaPayload| println!("Replica received payload: {:?}", payload))); let r_sorted_payloads = p_to_replicas .clone() @@ -166,8 +163,8 @@ pub fn replica<'a>( .sort(); // Create a cycle since we'll use this seq before we define it let (r_highest_seq_complete_cycle, r_highest_seq) = - flow.tick_cycle::>(replicas); - let empty_slot = flow.singleton_first_tick(replicas, q!(-1)); + replicas.tick_cycle::>(); + let empty_slot = replicas.singleton_first_tick(q!(-1)); // Either the max sequence number executed so far or -1. Need to union otherwise r_highest_seq is empty and joins with it will fail let r_highest_seq_with_default = r_highest_seq.union(empty_slot); // Find highest the sequence number of any payload that can be processed in this tick. This is the payload right before a hole. @@ -222,11 +219,11 @@ pub fn replica<'a>( // Send checkpoints to the acceptors when we've processed enough payloads let (r_checkpointed_seqs_complete_cycle, r_checkpointed_seqs) = - flow.tick_cycle::>(replicas); + replicas.tick_cycle::>(); let r_max_checkpointed_seq = r_checkpointed_seqs .persist() .max() - .unwrap_or(flow.singleton(replicas, q!(-1)).latest_tick()); + .unwrap_or(replicas.singleton(q!(-1)).latest_tick()); let r_checkpoint_seq_new = r_max_checkpointed_seq .cross_singleton(r_new_highest_seq) .filter_map(q!( @@ -250,20 +247,18 @@ pub fn replica<'a>( // Clients. All relations for clients will be prefixed with c. All ClientPayloads will contain the virtual client number as key and the client's machine ID (to string) as value. Expects p_to_clients_leader_elected containing Ballots whenever the leader is elected, and r_to_clients_payload_applied containing ReplicaPayloads whenever a payload is committed. Outputs (leader address, ClientPayload) when a new leader is elected or when the previous payload is committed. fn bench_client<'a, B: LeaderElected + std::fmt::Debug>( - clients: &Cluster, - p_to_clients_leader_elected: Stream<'a, B, Unbounded, NoTick, Cluster>, + clients: &Cluster<'a, Client>, + p_to_clients_leader_elected: Stream>, r_to_clients_payload_applied: Stream< - 'a, (u32, ReplicaPayload), Unbounded, NoTick, - Cluster, + Cluster<'a, Client>, >, - flow: &FlowBuilder<'a>, num_clients_per_node: usize, median_latency_window_size: usize, f: usize, -) -> Stream<'a, (u32, ClientPayload), Unbounded, NoTick, Cluster> { +) -> Stream<(u32, ClientPayload), Unbounded, NoTick, Cluster<'a, Client>> { let c_id = clients.self_id(); // r_to_clients_payload_applied.clone().inspect(q!(|payload: &(u32, ReplicaPayload)| println!("Client received payload: {:?}", payload))); // Only keep the latest leader @@ -289,7 +284,7 @@ fn bench_client<'a, B: LeaderElected + std::fmt::Debug>( ))); // Whenever replicas confirm that a payload was committed, collected it and wait for a quorum let (c_pending_quorum_payloads_complete_cycle, c_pending_quorum_payloads) = - flow.tick_cycle(clients); + clients.tick_cycle(); let c_received_payloads = r_to_clients_payload_applied .tick_batch() .map(q!(|(sender, replica_payload)| ( @@ -332,7 +327,7 @@ fn bench_client<'a, B: LeaderElected + std::fmt::Debug>( // Track statistics let (c_timers_complete_cycle, c_timers) = - flow.tick_cycle::>(clients); + clients.tick_cycle::>(); let c_new_timers_when_leader_elected = c_new_leader_ballot .map(q!(|_| SystemTime::now())) .flat_map(q!( @@ -352,7 +347,7 @@ fn bench_client<'a, B: LeaderElected + std::fmt::Debug>( })); c_timers_complete_cycle.complete_next_tick(c_new_timers); - let c_stats_output_timer = flow.source_interval(clients, q!(Duration::from_secs(1))); + let c_stats_output_timer = clients.source_interval(q!(Duration::from_secs(1))); let c_latency_reset = c_stats_output_timer .clone() diff --git a/hydroflow_plus_test/src/cluster/simple_cluster.rs b/hydroflow_plus_test/src/cluster/simple_cluster.rs index 3b06d9c79402..428c020f49b1 100644 --- a/hydroflow_plus_test/src/cluster/simple_cluster.rs +++ b/hydroflow_plus_test/src/cluster/simple_cluster.rs @@ -1,14 +1,12 @@ use hydroflow_plus::*; use stageleft::*; -pub fn simple_cluster(flow: &FlowBuilder) -> (Process<()>, Cluster<()>) { +pub fn simple_cluster<'a>(flow: &FlowBuilder<'a>) -> (Process<'a, ()>, Cluster<'a, ()>) { let process = flow.process(); let cluster = flow.cluster(); - let numbers = flow.source_iter(&process, q!(0..5)); - let ids = flow - .source_iter(&process, cluster.members()) - .map(q!(|&id| id)); + let numbers = process.source_iter(q!(0..5)); + let ids = process.source_iter(cluster.members()).map(q!(|&id| id)); let cluster_self_id = cluster.self_id(); diff --git a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__compute_pi__tests__compute_pi_ir.snap b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__compute_pi__tests__compute_pi_ir.snap index bc6617bf546e..c8bbf0a947ec 100644 --- a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__compute_pi__tests__compute_pi_ir.snap +++ b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__compute_pi__tests__compute_pi_ir.snap @@ -51,9 +51,9 @@ expression: built.ir() input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < () , (f64 , f64) > ({ use crate :: __staged :: cluster :: compute_pi :: * ; | _ | rand :: random :: < (f64 , f64) > () }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: builder :: * ; | _ | () }), + f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: location :: * ; | _ | () }), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < () , std :: ops :: Range < usize > > ({ use hydroflow_plus :: __staged :: builder :: * ; let batch_size = { use crate :: __staged :: cluster :: compute_pi :: * ; let batch_size = 8192usize ; batch_size } ; move | _ | 0 .. batch_size }), + f: stageleft :: runtime_support :: fn1_type_hint :: < () , std :: ops :: Range < usize > > ({ use hydroflow_plus :: __staged :: location :: * ; let batch_size = { use crate :: __staged :: cluster :: compute_pi :: * ; let batch_size = 8192usize ; batch_size } ; move | _ | 0 .. batch_size }), input: Source { source: Spin, location_kind: Cluster( diff --git a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__compute_pi__tests__compute_pi_ir@surface_graph_0.snap b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__compute_pi__tests__compute_pi_ir@surface_graph_0.snap index 234bb8f616bb..41e4f54f9479 100644 --- a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__compute_pi__tests__compute_pi_ir@surface_graph_0.snap +++ b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__compute_pi__tests__compute_pi_ir@surface_graph_0.snap @@ -3,8 +3,8 @@ source: hydroflow_plus_test/src/cluster/compute_pi.rs expression: ir.surface_syntax_string() --- 1v1 = spin (); -2v1 = flat_map (stageleft :: runtime_support :: fn1_type_hint :: < () , std :: ops :: Range < usize > > ({ use hydroflow_plus :: __staged :: builder :: * ; let batch_size = { use crate :: __staged :: cluster :: compute_pi :: * ; let batch_size = 8192usize ; batch_size } ; move | _ | 0 .. batch_size })); -3v1 = map (stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: builder :: * ; | _ | () })); +2v1 = flat_map (stageleft :: runtime_support :: fn1_type_hint :: < () , std :: ops :: Range < usize > > ({ use hydroflow_plus :: __staged :: location :: * ; let batch_size = { use crate :: __staged :: cluster :: compute_pi :: * ; let batch_size = 8192usize ; batch_size } ; move | _ | 0 .. batch_size })); +3v1 = map (stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: location :: * ; | _ | () })); 4v1 = map (stageleft :: runtime_support :: fn1_type_hint :: < () , (f64 , f64) > ({ use crate :: __staged :: cluster :: compute_pi :: * ; | _ | rand :: random :: < (f64 , f64) > () })); 5v1 = map (stageleft :: runtime_support :: fn1_type_hint :: < (f64 , f64) , bool > ({ use crate :: __staged :: cluster :: compute_pi :: * ; | (x , y) | x * x + y * y < 1.0 })); 6v1 = fold :: < 'tick > (stageleft :: runtime_support :: fn0_type_hint :: < (u64 , u64) > ({ use crate :: __staged :: cluster :: compute_pi :: * ; | | (0u64 , 0u64) }) , stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u64 , u64) , bool , () > ({ use crate :: __staged :: cluster :: compute_pi :: * ; | (inside , total) , sample_inside | { if sample_inside { * inside += 1 ; } * total += 1 ; } })); diff --git a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__paxos_bench__tests__paxos_ir.snap b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__paxos_bench__tests__paxos_ir.snap index 46452afa88e2..d5aba2eee9ab 100644 --- a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__paxos_bench__tests__paxos_ir.snap +++ b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__paxos_bench__tests__paxos_ir.snap @@ -81,7 +81,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, ), location_kind: Cluster( 2, @@ -105,7 +105,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -130,7 +130,7 @@ expression: built.ir() CrossSingleton( Source { source: Stream( - { use hydroflow_plus :: __staged :: builder :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_2 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + { use hydroflow_plus :: __staged :: location :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_2 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, ), location_kind: Cluster( 2, @@ -219,7 +219,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -301,7 +301,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, ), location_kind: Cluster( 2, @@ -325,7 +325,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -416,7 +416,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -494,7 +494,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -576,7 +576,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, ), location_kind: Cluster( 2, @@ -600,7 +600,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -702,7 +702,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -727,7 +727,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; - 1 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; - 1 } ; [e] }, ), location_kind: Cluster( 2, @@ -1049,7 +1049,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -1131,7 +1131,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, ), location_kind: Cluster( 2, @@ -1155,7 +1155,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -1237,7 +1237,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -1319,7 +1319,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, ), location_kind: Cluster( 2, @@ -1343,7 +1343,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -1435,7 +1435,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -1460,7 +1460,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; - 1 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; - 1 } ; [e] }, ), location_kind: Cluster( 2, @@ -1782,7 +1782,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -1864,7 +1864,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, ), location_kind: Cluster( 2, @@ -1888,7 +1888,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -2221,7 +2221,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -2243,7 +2243,7 @@ expression: built.ir() CrossSingleton( Source { source: Stream( - { use hydroflow_plus :: __staged :: builder :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_2 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + { use hydroflow_plus :: __staged :: location :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_2 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, ), location_kind: Cluster( 2, @@ -2332,7 +2332,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -2414,7 +2414,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, ), location_kind: Cluster( 2, @@ -2438,7 +2438,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -2540,7 +2540,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -2562,7 +2562,7 @@ expression: built.ir() CrossSingleton( Source { source: Stream( - { use hydroflow_plus :: __staged :: builder :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_2 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + { use hydroflow_plus :: __staged :: location :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_2 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, ), location_kind: Cluster( 2, @@ -2651,7 +2651,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -2733,7 +2733,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, ), location_kind: Cluster( 2, @@ -2757,7 +2757,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -2800,7 +2800,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, ), location_kind: Cluster( 3, @@ -2913,7 +2913,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -2945,7 +2945,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -3016,7 +3016,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -3088,7 +3088,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -3122,7 +3122,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -3238,7 +3238,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -3306,7 +3306,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -3388,7 +3388,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, ), location_kind: Cluster( 2, @@ -3412,7 +3412,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -3504,7 +3504,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -3526,7 +3526,7 @@ expression: built.ir() CrossSingleton( Source { source: Stream( - { use hydroflow_plus :: __staged :: builder :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_2 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + { use hydroflow_plus :: __staged :: location :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_2 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, ), location_kind: Cluster( 2, @@ -3615,7 +3615,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -3697,7 +3697,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, ), location_kind: Cluster( 2, @@ -3721,7 +3721,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -3764,7 +3764,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, ), location_kind: Cluster( 3, @@ -3906,7 +3906,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; - 1 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; - 1 } ; [e] }, ), location_kind: Cluster( 3, @@ -4057,7 +4057,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -4089,7 +4089,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -4160,7 +4160,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -4232,7 +4232,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -4266,7 +4266,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -4382,7 +4382,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -4450,7 +4450,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -4532,7 +4532,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, ), location_kind: Cluster( 2, @@ -4556,7 +4556,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -4648,7 +4648,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -4670,7 +4670,7 @@ expression: built.ir() CrossSingleton( Source { source: Stream( - { use hydroflow_plus :: __staged :: builder :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_2 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + { use hydroflow_plus :: __staged :: location :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_2 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, ), location_kind: Cluster( 2, @@ -4759,7 +4759,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -4841,7 +4841,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, ), location_kind: Cluster( 2, @@ -4865,7 +4865,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -4908,7 +4908,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, ), location_kind: Cluster( 3, @@ -5186,7 +5186,7 @@ expression: built.ir() }, Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos_bench :: * ; - 1 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos_bench :: * ; - 1 } ; [e] }, ), location_kind: Cluster( 1, @@ -5476,7 +5476,7 @@ expression: built.ir() }, Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos_bench :: * ; - 1 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos_bench :: * ; - 1 } ; [e] }, ), location_kind: Cluster( 1, @@ -5529,7 +5529,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos_bench :: * ; - 1 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos_bench :: * ; - 1 } ; [e] }, ), location_kind: Cluster( 1, @@ -5803,7 +5803,7 @@ expression: built.ir() }, Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos_bench :: * ; - 1 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos_bench :: * ; - 1 } ; [e] }, ), location_kind: Cluster( 1, @@ -5859,7 +5859,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos_bench :: * ; - 1 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos_bench :: * ; - 1 } ; [e] }, ), location_kind: Cluster( 1, @@ -6133,7 +6133,7 @@ expression: built.ir() }, Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos_bench :: * ; - 1 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos_bench :: * ; - 1 } ; [e] }, ), location_kind: Cluster( 1, @@ -6617,7 +6617,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -6699,7 +6699,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, ), location_kind: Cluster( 2, @@ -6723,7 +6723,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -6780,7 +6780,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -7546,7 +7546,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -7628,7 +7628,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, ), location_kind: Cluster( 2, @@ -7652,7 +7652,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -7709,7 +7709,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -7999,7 +7999,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -8081,7 +8081,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, ), location_kind: Cluster( 2, @@ -8105,7 +8105,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, @@ -8162,7 +8162,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( 2, diff --git a/hydroflow_plus_test/src/distributed/first_ten.rs b/hydroflow_plus_test/src/distributed/first_ten.rs index d08264977740..18b7703f8ab4 100644 --- a/hydroflow_plus_test/src/distributed/first_ten.rs +++ b/hydroflow_plus_test/src/distributed/first_ten.rs @@ -11,23 +11,23 @@ struct SendOverNetwork { pub struct P1 {} pub struct P2 {} -pub fn first_ten_distributed( - flow: &FlowBuilder, +pub fn first_ten_distributed<'a>( + flow: &FlowBuilder<'a>, ) -> ( - ExternalProcess<()>, + ExternalProcess<'a, ()>, ExternalBincodeSink, - Process, - Process, + Process<'a, P1>, + Process<'a, P2>, ) { let external_process = flow.external_process::<()>(); let process = flow.process::(); let second_process = flow.process::(); let (numbers_external_port, numbers_external) = - flow.source_external_bincode(&external_process, &process); + external_process.source_external_bincode(&process); numbers_external.for_each(q!(|n| println!("hi: {:?}", n))); - let numbers = flow.source_iter(&process, q!(0..10)); + let numbers = process.source_iter(q!(0..10)); numbers .map(q!(|n| SendOverNetwork { n })) .send_bincode(&second_process) diff --git a/hydroflow_plus_test_local/src/local/chat_app.rs b/hydroflow_plus_test_local/src/local/chat_app.rs index 2e2e281bd7f2..6ede979ea63f 100644 --- a/hydroflow_plus_test_local/src/local/chat_app.rs +++ b/hydroflow_plus_test_local/src/local/chat_app.rs @@ -14,11 +14,8 @@ pub fn chat_app<'a>( ) -> impl Quoted<'a, Hydroflow<'a>> { let process = flow.process::<()>(); - let users = flow - .source_stream(&process, users_stream) - .tick_batch() - .persist(); - let messages = flow.source_stream(&process, messages); + let users = process.source_stream(users_stream).tick_batch().persist(); + let messages = process.source_stream(messages); let messages = if replay_messages { messages.tick_batch().persist() } else { diff --git a/hydroflow_plus_test_local/src/local/compute_pi.rs b/hydroflow_plus_test_local/src/local/compute_pi.rs index 8d07b742ff83..af4882fa2cfa 100644 --- a/hydroflow_plus_test_local/src/local/compute_pi.rs +++ b/hydroflow_plus_test_local/src/local/compute_pi.rs @@ -4,11 +4,11 @@ use hydroflow_plus::deploy::SingleProcessGraph; use hydroflow_plus::*; use stageleft::*; -pub fn compute_pi(flow: &FlowBuilder, batch_size: RuntimeData) -> Process<()> { +pub fn compute_pi<'a>(flow: &FlowBuilder<'a>, batch_size: RuntimeData) -> Process<'a, ()> { let process = flow.process(); - let trials = flow - .spin_batch(&process, q!(batch_size)) + let trials = process + .spin_batch(q!(batch_size)) .map(q!(|_| rand::random::<(f64, f64)>())) .map(q!(|(x, y)| x * x + y * y < 1.0)) .fold( diff --git a/hydroflow_plus_test_local/src/local/count_elems.rs b/hydroflow_plus_test_local/src/local/count_elems.rs index 47cf777e38c1..04818fed51ad 100644 --- a/hydroflow_plus_test_local/src/local/count_elems.rs +++ b/hydroflow_plus_test_local/src/local/count_elems.rs @@ -11,7 +11,7 @@ pub fn count_elems_generic<'a, T: 'a>( ) -> impl Quoted<'a, Hydroflow<'a>> { let process = flow.process::<()>(); - let source = flow.source_stream(&process, input_stream); + let source = process.source_stream(input_stream); let count = source .map(q!(|_| 1)) .tick_batch() diff --git a/hydroflow_plus_test_local/src/local/first_ten.rs b/hydroflow_plus_test_local/src/local/first_ten.rs index 4a684fe8d6d0..8da36aa823c6 100644 --- a/hydroflow_plus_test_local/src/local/first_ten.rs +++ b/hydroflow_plus_test_local/src/local/first_ten.rs @@ -4,7 +4,7 @@ use stageleft::*; pub fn first_ten(flow: &FlowBuilder) { let process = flow.process::<()>(); - let numbers = flow.source_iter(&process, q!(0..10)); + let numbers = process.source_iter(q!(0..10)); numbers.for_each(q!(|n| println!("{}", n))); } diff --git a/hydroflow_plus_test_local/src/local/graph_reachability.rs b/hydroflow_plus_test_local/src/local/graph_reachability.rs index 84237d44261d..c3a27f6c1d3f 100644 --- a/hydroflow_plus_test_local/src/local/graph_reachability.rs +++ b/hydroflow_plus_test_local/src/local/graph_reachability.rs @@ -13,10 +13,10 @@ pub fn graph_reachability<'a>( ) -> impl Quoted<'a, Hydroflow<'a>> { let process = flow.process::<()>(); - let roots = flow.source_stream(&process, roots); - let edges = flow.source_stream(&process, edges); + let roots = process.source_stream(roots); + let edges = process.source_stream(edges); - let (set_reached_cycle, reached_cycle) = flow.cycle(&process); + let (set_reached_cycle, reached_cycle) = process.cycle(); let reached = roots.union(reached_cycle); let reachable = reached diff --git a/hydroflow_plus_test_local/src/local/negation.rs b/hydroflow_plus_test_local/src/local/negation.rs index c5517c47cfb7..9fb75b678198 100644 --- a/hydroflow_plus_test_local/src/local/negation.rs +++ b/hydroflow_plus_test_local/src/local/negation.rs @@ -12,12 +12,12 @@ pub fn test_difference<'a>( ) -> impl Quoted<'a, Hydroflow<'a>> { let process = flow.process::<()>(); - let mut source = flow.source_iter(&process, q!(0..5)).tick_batch(); + let mut source = process.source_iter(q!(0..5)).tick_batch(); if persist1 { source = source.persist(); } - let mut source2 = flow.source_iter(&process, q!(3..6)).tick_batch(); + let mut source2 = process.source_iter(q!(3..6)).tick_batch(); if persist2 { source2 = source2.persist(); } @@ -39,15 +39,15 @@ pub fn test_anti_join<'a>( ) -> impl Quoted<'a, Hydroflow<'a>> { let process = flow.process::<()>(); - let mut source = flow - .source_iter(&process, q!(0..5)) + let mut source = process + .source_iter(q!(0..5)) .map(q!(|v| (v, v))) .tick_batch(); if persist1 { source = source.persist(); } - let mut source2 = flow.source_iter(&process, q!(3..6)).tick_batch(); + let mut source2 = process.source_iter(q!(3..6)).tick_batch(); if persist2 { source2 = source2.persist(); } diff --git a/hydroflow_plus_test_local/src/local/teed_join.rs b/hydroflow_plus_test_local/src/local/teed_join.rs index 42ffb698a165..951ad9608874 100644 --- a/hydroflow_plus_test_local/src/local/teed_join.rs +++ b/hydroflow_plus_test_local/src/local/teed_join.rs @@ -19,7 +19,7 @@ pub fn teed_join<'a, S: Stream + Unpin + 'a>( let node_zero = flow.process::(); let node_one = flow.process::(); - let source = flow.source_stream(&node_zero, input_stream).tick_batch(); + let source = node_zero.source_stream(input_stream).tick_batch(); let map1 = source.clone().map(q!(|v| (v + 1, ()))); let map2 = source.map(q!(|v| (v - 1, ()))); @@ -35,7 +35,7 @@ pub fn teed_join<'a, S: Stream + Unpin + 'a>( })); } - let source_node_id_1 = flow.source_iter(&node_one, q!(0..5)); + let source_node_id_1 = node_one.source_iter(q!(0..5)); source_node_id_1.for_each(q!(|v| { output.send(v).unwrap(); })); diff --git a/template/hydroflow_plus/src/first_ten_distributed.rs b/template/hydroflow_plus/src/first_ten_distributed.rs index 7b0e966bf7da..5fb1c4b48fff 100644 --- a/template/hydroflow_plus/src/first_ten_distributed.rs +++ b/template/hydroflow_plus/src/first_ten_distributed.rs @@ -4,11 +4,11 @@ use stageleft::*; pub struct P1 {} pub struct P2 {} -pub fn first_ten_distributed(flow: &FlowBuilder) -> (Process, Process) { +pub fn first_ten_distributed<'a>(flow: &FlowBuilder<'a>) -> (Process<'a, P1>, Process<'a, P2>) { let process = flow.process::(); let second_process = flow.process::(); - let numbers = flow.source_iter(&process, q!(0..10)); + let numbers = process.source_iter(q!(0..10)); numbers .send_bincode(&second_process) .for_each(q!(|n| println!("{}", n)));