hydro_lang/live_collections/stream/mod.rs
1//! Definitions for the [`Stream`] live collection.
2
3use std::cell::RefCell;
4use std::future::Future;
5use std::hash::Hash;
6use std::marker::PhantomData;
7use std::ops::Deref;
8use std::rc::Rc;
9
10use stageleft::{IntoQuotedMut, QuotedWithContext, QuotedWithContextWithProps, q, quote_type};
11use tokio::time::Instant;
12
13use super::boundedness::{Bounded, Boundedness, IsBounded, Unbounded};
14use super::keyed_singleton::KeyedSingleton;
15use super::keyed_stream::{Generate, KeyedStream};
16use super::optional::Optional;
17use super::singleton::Singleton;
18use crate::compile::builder::{CycleId, FlowState};
19use crate::compile::ir::{
20 CollectionKind, HydroIrOpMetadata, HydroNode, HydroRoot, SharedNode, StreamOrder, StreamRetry,
21};
22#[cfg(stageleft_runtime)]
23use crate::forward_handle::{CycleCollection, CycleCollectionWithInitial, ReceiverComplete};
24use crate::forward_handle::{ForwardRef, TickCycle};
25use crate::live_collections::batch_atomic::BatchAtomic;
26use crate::live_collections::singleton::SingletonBound;
27#[cfg(stageleft_runtime)]
28use crate::location::dynamic::{DynLocation, LocationId};
29use crate::location::tick::{Atomic, DeferTick};
30use crate::location::{Location, Tick, TopLevel, check_matching_location};
31use crate::manual_expr::ManualExpr;
32use crate::nondet::{NonDet, nondet};
33use crate::prelude::manual_proof;
34use crate::properties::{
35 AggFuncAlgebra, ApplyMonotoneStream, ValidCommutativityFor, ValidIdempotenceFor,
36};
37
38pub mod networking;
39
40/// A trait implemented by valid ordering markers ([`TotalOrder`] and [`NoOrder`]).
41#[sealed::sealed]
42pub trait Ordering:
43 MinOrder<Self, Min = Self> + MinOrder<TotalOrder, Min = Self> + MinOrder<NoOrder, Min = NoOrder>
44{
45 /// The [`StreamOrder`] corresponding to this type.
46 const ORDERING_KIND: StreamOrder;
47}
48
49/// Marks the stream as being totally ordered, which means that there are
50/// no sources of non-determinism (other than intentional ones) that will
51/// affect the order of elements.
52pub enum TotalOrder {}
53
54#[sealed::sealed]
55impl Ordering for TotalOrder {
56 const ORDERING_KIND: StreamOrder = StreamOrder::TotalOrder;
57}
58
59/// Marks the stream as having no order, which means that the order of
60/// elements may be affected by non-determinism.
61///
62/// This restricts certain operators, such as `fold` and `reduce`, to only
63/// be used with commutative aggregation functions.
64pub enum NoOrder {}
65
66#[sealed::sealed]
67impl Ordering for NoOrder {
68 const ORDERING_KIND: StreamOrder = StreamOrder::NoOrder;
69}
70
71/// Marker trait for an [`Ordering`] that is available when `Self` is a weaker guarantee than
72/// `Other`, which means that a stream with `Other` guarantees can be safely converted to
73/// have `Self` guarantees instead.
74#[sealed::sealed]
75pub trait WeakerOrderingThan<Other: ?Sized>: Ordering {}
76#[sealed::sealed]
77impl<O: Ordering, O2: Ordering> WeakerOrderingThan<O2> for O where O: MinOrder<O2, Min = O> {}
78
79/// Helper trait for determining the weakest of two orderings.
80#[sealed::sealed]
81pub trait MinOrder<Other: ?Sized> {
82 /// The weaker of the two orderings.
83 type Min: Ordering;
84}
85
86#[sealed::sealed]
87impl<O: Ordering> MinOrder<O> for TotalOrder {
88 type Min = O;
89}
90
91#[sealed::sealed]
92impl<O: Ordering> MinOrder<O> for NoOrder {
93 type Min = NoOrder;
94}
95
96/// A trait implemented by valid retries markers ([`ExactlyOnce`] and [`AtLeastOnce`]).
97#[sealed::sealed]
98pub trait Retries:
99 MinRetries<Self, Min = Self>
100 + MinRetries<ExactlyOnce, Min = Self>
101 + MinRetries<AtLeastOnce, Min = AtLeastOnce>
102{
103 /// The [`StreamRetry`] corresponding to this type.
104 const RETRIES_KIND: StreamRetry;
105}
106
107/// Marks the stream as having deterministic message cardinality, with no
108/// possibility of duplicates.
109pub enum ExactlyOnce {}
110
111#[sealed::sealed]
112impl Retries for ExactlyOnce {
113 const RETRIES_KIND: StreamRetry = StreamRetry::ExactlyOnce;
114}
115
116/// Marks the stream as having non-deterministic message cardinality, which
117/// means that duplicates may occur, but messages will not be dropped.
118pub enum AtLeastOnce {}
119
120#[sealed::sealed]
121impl Retries for AtLeastOnce {
122 const RETRIES_KIND: StreamRetry = StreamRetry::AtLeastOnce;
123}
124
125/// Marker trait for a [`Retries`] that is available when `Self` is a weaker guarantee than
126/// `Other`, which means that a stream with `Other` guarantees can be safely converted to
127/// have `Self` guarantees instead.
128#[sealed::sealed]
129pub trait WeakerRetryThan<Other: ?Sized>: Retries {}
130#[sealed::sealed]
131impl<R: Retries, R2: Retries> WeakerRetryThan<R2> for R where R: MinRetries<R2, Min = R> {}
132
133/// Helper trait for determining the weakest of two retry guarantees.
134#[sealed::sealed]
135pub trait MinRetries<Other: ?Sized> {
136 /// The weaker of the two retry guarantees.
137 type Min: Retries + WeakerRetryThan<Self> + WeakerRetryThan<Other>;
138}
139
140#[sealed::sealed]
141impl<R: Retries> MinRetries<R> for ExactlyOnce {
142 type Min = R;
143}
144
145#[sealed::sealed]
146impl<R: Retries> MinRetries<R> for AtLeastOnce {
147 type Min = AtLeastOnce;
148}
149
150#[sealed::sealed]
151#[diagnostic::on_unimplemented(
152 message = "The input stream must be totally-ordered (`TotalOrder`), but has order `{Self}`. Strengthen the order upstream or consider a different API.",
153 label = "required here",
154 note = "To intentionally process the stream by observing a non-deterministic (shuffled) order of elements, use `.assume_ordering`. This introduces non-determinism so avoid unless necessary."
155)]
156/// Marker trait that is implemented for the [`TotalOrder`] ordering guarantee.
157pub trait IsOrdered: Ordering {}
158
159#[sealed::sealed]
160#[diagnostic::do_not_recommend]
161impl IsOrdered for TotalOrder {}
162
163#[sealed::sealed]
164#[diagnostic::on_unimplemented(
165 message = "The input stream must be exactly-once (`ExactlyOnce`), but has retries `{Self}`. Strengthen the retries guarantee upstream or consider a different API.",
166 label = "required here",
167 note = "To intentionally process the stream by observing non-deterministic (randomly duplicated) retries, use `.assume_retries`. This introduces non-determinism so avoid unless necessary."
168)]
169/// Marker trait that is implemented for the [`ExactlyOnce`] retries guarantee.
170pub trait IsExactlyOnce: Retries {}
171
172#[sealed::sealed]
173#[diagnostic::do_not_recommend]
174impl IsExactlyOnce for ExactlyOnce {}
175
176/// Streaming sequence of elements with type `Type`.
177///
178/// This live collection represents a growing sequence of elements, with new elements being
179/// asynchronously appended to the end of the sequence. This can be used to model the arrival
180/// of network input, such as API requests, or streaming ingestion.
181///
182/// By default, all streams have deterministic ordering and each element is materialized exactly
183/// once. But streams can also capture non-determinism via the `Order` and `Retries` type
184/// parameters. When the ordering / retries guarantee is relaxed, fewer APIs will be available
185/// on the stream. For example, if the stream is unordered, you cannot invoke [`Stream::first`].
186///
187/// Type Parameters:
188/// - `Type`: the type of elements in the stream
189/// - `Loc`: the location where the stream is being materialized
190/// - `Bound`: the boundedness of the stream, which is either [`Bounded`] or [`Unbounded`]
191/// - `Order`: the ordering of the stream, which is either [`TotalOrder`] or [`NoOrder`]
192/// (default is [`TotalOrder`])
193/// - `Retries`: the retry guarantee of the stream, which is either [`ExactlyOnce`] or
194/// [`AtLeastOnce`] (default is [`ExactlyOnce`])
195pub struct Stream<
196 Type,
197 Loc,
198 Bound: Boundedness = Unbounded,
199 Order: Ordering = TotalOrder,
200 Retry: Retries = ExactlyOnce,
201> {
202 pub(crate) location: Loc,
203 pub(crate) ir_node: RefCell<HydroNode>,
204 pub(crate) flow_state: FlowState,
205
206 _phantom: PhantomData<(Type, Loc, Bound, Order, Retry)>,
207}
208
209impl<T, L, B: Boundedness, O: Ordering, R: Retries> Drop for Stream<T, L, B, O, R> {
210 fn drop(&mut self) {
211 let ir_node = self.ir_node.replace(HydroNode::Placeholder);
212 if !matches!(ir_node, HydroNode::Placeholder) && !ir_node.is_shared_with_others() {
213 self.flow_state.borrow_mut().try_push_root(HydroRoot::Null {
214 input: Box::new(ir_node),
215 op_metadata: HydroIrOpMetadata::new(),
216 });
217 }
218 }
219}
220
221impl<'a, T, L, O: Ordering, R: Retries> From<Stream<T, L, Bounded, O, R>>
222 for Stream<T, L, Unbounded, O, R>
223where
224 L: Location<'a>,
225{
226 fn from(stream: Stream<T, L, Bounded, O, R>) -> Stream<T, L, Unbounded, O, R> {
227 let new_meta = stream
228 .location
229 .new_node_metadata(Stream::<T, L, Unbounded, O, R>::collection_kind());
230
231 Stream {
232 location: stream.location.clone(),
233 flow_state: stream.flow_state.clone(),
234 ir_node: RefCell::new(HydroNode::Cast {
235 inner: Box::new(stream.ir_node.replace(HydroNode::Placeholder)),
236 metadata: new_meta,
237 }),
238 _phantom: PhantomData,
239 }
240 }
241}
242
243impl<'a, T, L, B: Boundedness, R: Retries> From<Stream<T, L, B, TotalOrder, R>>
244 for Stream<T, L, B, NoOrder, R>
245where
246 L: Location<'a>,
247{
248 fn from(stream: Stream<T, L, B, TotalOrder, R>) -> Stream<T, L, B, NoOrder, R> {
249 stream.weaken_ordering()
250 }
251}
252
253impl<'a, T, L, B: Boundedness, O: Ordering> From<Stream<T, L, B, O, ExactlyOnce>>
254 for Stream<T, L, B, O, AtLeastOnce>
255where
256 L: Location<'a>,
257{
258 fn from(stream: Stream<T, L, B, O, ExactlyOnce>) -> Stream<T, L, B, O, AtLeastOnce> {
259 stream.weaken_retries()
260 }
261}
262
263impl<'a, T, L, O: Ordering, R: Retries> DeferTick for Stream<T, Tick<L>, Bounded, O, R>
264where
265 L: Location<'a>,
266{
267 fn defer_tick(self) -> Self {
268 Stream::defer_tick(self)
269 }
270}
271
272impl<'a, T, L, O: Ordering, R: Retries> CycleCollection<'a, TickCycle>
273 for Stream<T, Tick<L>, Bounded, O, R>
274where
275 L: Location<'a>,
276{
277 type Location = Tick<L>;
278
279 fn create_source(cycle_id: CycleId, location: Tick<L>) -> Self {
280 Stream::new(
281 location.clone(),
282 HydroNode::CycleSource {
283 cycle_id,
284 metadata: location.new_node_metadata(Self::collection_kind()),
285 },
286 )
287 }
288}
289
290impl<'a, T, L, O: Ordering, R: Retries> CycleCollectionWithInitial<'a, TickCycle>
291 for Stream<T, Tick<L>, Bounded, O, R>
292where
293 L: Location<'a>,
294{
295 type Location = Tick<L>;
296
297 fn location(&self) -> &Self::Location {
298 self.location()
299 }
300
301 fn create_source_with_initial(cycle_id: CycleId, initial: Self, location: Tick<L>) -> Self {
302 let from_previous_tick: Stream<T, Tick<L>, Bounded, O, R> = Stream::new(
303 location.clone(),
304 HydroNode::DeferTick {
305 input: Box::new(HydroNode::CycleSource {
306 cycle_id,
307 metadata: location.new_node_metadata(Self::collection_kind()),
308 }),
309 metadata: location.new_node_metadata(Self::collection_kind()),
310 },
311 );
312
313 from_previous_tick.chain(initial.filter_if(location.optional_first_tick(q!(())).is_some()))
314 }
315}
316
317impl<'a, T, L, O: Ordering, R: Retries> ReceiverComplete<'a, TickCycle>
318 for Stream<T, Tick<L>, Bounded, O, R>
319where
320 L: Location<'a>,
321{
322 fn complete(self, cycle_id: CycleId, expected_location: LocationId) {
323 assert_eq!(
324 Location::id(&self.location),
325 expected_location,
326 "locations do not match"
327 );
328 self.location
329 .flow_state()
330 .borrow_mut()
331 .push_root(HydroRoot::CycleSink {
332 cycle_id,
333 input: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
334 op_metadata: HydroIrOpMetadata::new(),
335 });
336 }
337}
338
339impl<'a, T, L, B: Boundedness, O: Ordering, R: Retries> CycleCollection<'a, ForwardRef>
340 for Stream<T, L, B, O, R>
341where
342 L: Location<'a>,
343{
344 type Location = L;
345
346 fn create_source(cycle_id: CycleId, location: L) -> Self {
347 Stream::new(
348 location.clone(),
349 HydroNode::CycleSource {
350 cycle_id,
351 metadata: location.new_node_metadata(Self::collection_kind()),
352 },
353 )
354 }
355}
356
357impl<'a, T, L, B: Boundedness, O: Ordering, R: Retries> ReceiverComplete<'a, ForwardRef>
358 for Stream<T, L, B, O, R>
359where
360 L: Location<'a>,
361{
362 fn complete(self, cycle_id: CycleId, expected_location: LocationId) {
363 assert_eq!(
364 Location::id(&self.location),
365 expected_location,
366 "locations do not match"
367 );
368 self.location
369 .flow_state()
370 .borrow_mut()
371 .push_root(HydroRoot::CycleSink {
372 cycle_id,
373 input: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
374 op_metadata: HydroIrOpMetadata::new(),
375 });
376 }
377}
378
379impl<'a, T, L, B: Boundedness, O: Ordering, R: Retries> Clone for Stream<T, L, B, O, R>
380where
381 T: Clone,
382 L: Location<'a>,
383{
384 fn clone(&self) -> Self {
385 if !matches!(self.ir_node.borrow().deref(), HydroNode::Tee { .. }) {
386 let orig_ir_node = self.ir_node.replace(HydroNode::Placeholder);
387 *self.ir_node.borrow_mut() = HydroNode::Tee {
388 inner: SharedNode(Rc::new(RefCell::new(orig_ir_node))),
389 metadata: self.location.new_node_metadata(Self::collection_kind()),
390 };
391 }
392
393 let HydroNode::Tee { inner, metadata } = &*self.ir_node.borrow() else {
394 unreachable!()
395 };
396 Stream {
397 location: self.location.clone(),
398 flow_state: self.flow_state.clone(),
399 ir_node: HydroNode::Tee {
400 inner: SharedNode(inner.0.clone()),
401 metadata: metadata.clone(),
402 }
403 .into(),
404 _phantom: PhantomData,
405 }
406 }
407}
408
409impl<'a, T, L, B: Boundedness, O: Ordering, R: Retries> Stream<T, L, B, O, R>
410where
411 L: Location<'a>,
412{
413 pub(crate) fn new(location: L, ir_node: HydroNode) -> Self {
414 debug_assert_eq!(ir_node.metadata().location_id, Location::id(&location));
415 debug_assert_eq!(ir_node.metadata().collection_kind, Self::collection_kind());
416
417 let flow_state = location.flow_state().clone();
418 Stream {
419 location,
420 flow_state,
421 ir_node: RefCell::new(ir_node),
422 _phantom: PhantomData,
423 }
424 }
425
426 /// Returns the [`Location`] where this stream is being materialized.
427 pub fn location(&self) -> &L {
428 &self.location
429 }
430
431 /// Weakens the consistency of this live collection to not guarantee any consistency across
432 /// cluster members (if this collection is on a cluster).
433 pub fn weaken_consistency(self) -> Stream<T, L::DropConsistency, B, O, R>
434 where
435 L: Location<'a>,
436 {
437 if L::consistency()
438 .is_none_or(|c| c == crate::location::dynamic::ClusterConsistency::NoConsistency)
439 {
440 // already no consistency
441 Stream::new(
442 self.location.drop_consistency(),
443 self.ir_node.replace(HydroNode::Placeholder),
444 )
445 } else {
446 Stream::new(
447 self.location.drop_consistency(),
448 HydroNode::Cast {
449 inner: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
450 metadata: self.location.drop_consistency().new_node_metadata(Stream::<
451 T,
452 L::DropConsistency,
453 B,
454 O,
455 R,
456 >::collection_kind(
457 )),
458 },
459 )
460 }
461 }
462
463 /// Casts this live collection to have the consistency guarantees specified in the given
464 /// location type parameter. The developer must ensure that the strengthened consistency
465 /// is actually guaranteed, via the proof field (see [`crate::prelude::manual_proof`]).
466 pub fn assert_has_consistency_of<L2: Location<'a, DropConsistency = L::DropConsistency>>(
467 self,
468 _proof: impl crate::properties::ConsistencyProof,
469 ) -> Stream<T, L2, B, O, R>
470 where
471 L: Location<'a>,
472 {
473 if L::consistency() == L2::consistency() {
474 Stream::new(
475 self.location.with_consistency_of(),
476 self.ir_node.replace(HydroNode::Placeholder),
477 )
478 } else {
479 Stream::new(
480 self.location.with_consistency_of(),
481 HydroNode::AssertIsConsistent {
482 inner: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
483 trusted: false,
484 metadata: self
485 .location
486 .clone()
487 .with_consistency_of::<L2>()
488 .new_node_metadata(Stream::<T, L2, B, O, R>::collection_kind()),
489 },
490 )
491 }
492 }
493
494 pub(crate) fn assert_has_consistency_of_trusted<
495 L2: Location<'a, DropConsistency = L::DropConsistency>,
496 >(
497 self,
498 _proof: impl crate::properties::ConsistencyProof,
499 ) -> Stream<T, L2, B, O, R>
500 where
501 L: Location<'a>,
502 {
503 if L::consistency() == L2::consistency() {
504 Stream::new(
505 self.location.with_consistency_of(),
506 self.ir_node.replace(HydroNode::Placeholder),
507 )
508 } else {
509 Stream::new(
510 self.location.with_consistency_of(),
511 HydroNode::AssertIsConsistent {
512 inner: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
513 trusted: true,
514 metadata: self
515 .location
516 .clone()
517 .with_consistency_of::<L2>()
518 .new_node_metadata(Stream::<T, L2, B, O, R>::collection_kind()),
519 },
520 )
521 }
522 }
523
524 pub(crate) fn collection_kind() -> CollectionKind {
525 CollectionKind::Stream {
526 bound: B::BOUND_KIND,
527 order: O::ORDERING_KIND,
528 retry: R::RETRIES_KIND,
529 element_type: quote_type::<T>().into(),
530 }
531 }
532
533 /// Produces a stream based on invoking `f` on each element.
534 /// If you do not want to modify the stream and instead only want to view
535 /// each item use [`Stream::inspect`] instead.
536 ///
537 /// # Example
538 /// ```rust
539 /// # #[cfg(feature = "deploy")] {
540 /// # use hydro_lang::prelude::*;
541 /// # use futures::StreamExt;
542 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
543 /// let words = process.source_iter(q!(vec!["hello", "world"]));
544 /// words.map(q!(|x| x.to_uppercase()))
545 /// # }, |mut stream| async move {
546 /// # for w in vec!["HELLO", "WORLD"] {
547 /// # assert_eq!(stream.next().await.unwrap(), w);
548 /// # }
549 /// # }));
550 /// # }
551 /// ```
552 pub fn map<U, F>(self, f: impl IntoQuotedMut<'a, F, L>) -> Stream<U, L, B, O, R>
553 where
554 F: Fn(T) -> U + 'a,
555 {
556 let f = crate::singleton_ref::with_singleton_capture(|| {
557 f.splice_fn1_ctx(&self.location).into()
558 });
559 Stream::new(
560 self.location.clone(),
561 HydroNode::Map {
562 f,
563 input: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
564 metadata: self
565 .location
566 .new_node_metadata(Stream::<U, L, B, O, R>::collection_kind()),
567 },
568 )
569 }
570
571 /// For each item `i` in the input stream, transform `i` using `f` and then treat the
572 /// result as an [`Iterator`] to produce items one by one. The implementation for [`Iterator`]
573 /// for the output type `U` must produce items in a **deterministic** order.
574 ///
575 /// For example, `U` could be a `Vec`, but not a `HashSet`. If the order of the items in `U` is
576 /// not deterministic, use [`Stream::flat_map_unordered`] instead.
577 ///
578 /// # Example
579 /// ```rust
580 /// # #[cfg(feature = "deploy")] {
581 /// # use hydro_lang::prelude::*;
582 /// # use futures::StreamExt;
583 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
584 /// process
585 /// .source_iter(q!(vec![vec![1, 2], vec![3, 4]]))
586 /// .flat_map_ordered(q!(|x| x))
587 /// # }, |mut stream| async move {
588 /// // 1, 2, 3, 4
589 /// # for w in (1..5) {
590 /// # assert_eq!(stream.next().await.unwrap(), w);
591 /// # }
592 /// # }));
593 /// # }
594 /// ```
595 pub fn flat_map_ordered<U, I, F>(self, f: impl IntoQuotedMut<'a, F, L>) -> Stream<U, L, B, O, R>
596 where
597 I: IntoIterator<Item = U>,
598 F: Fn(T) -> I + 'a,
599 {
600 let f = crate::singleton_ref::with_singleton_capture(|| {
601 f.splice_fn1_ctx(&self.location).into()
602 });
603 Stream::new(
604 self.location.clone(),
605 HydroNode::FlatMap {
606 f,
607 input: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
608 metadata: self
609 .location
610 .new_node_metadata(Stream::<U, L, B, O, R>::collection_kind()),
611 },
612 )
613 }
614
615 /// Like [`Stream::flat_map_ordered`], but allows the implementation of [`Iterator`]
616 /// for the output type `U` to produce items in any order.
617 ///
618 /// # Example
619 /// ```rust
620 /// # #[cfg(feature = "deploy")] {
621 /// # use hydro_lang::{prelude::*, live_collections::stream::{NoOrder, ExactlyOnce}};
622 /// # use futures::StreamExt;
623 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test::<_, _, _, NoOrder, ExactlyOnce>(|process| {
624 /// process
625 /// .source_iter(q!(vec![
626 /// std::collections::HashSet::<i32>::from_iter(vec![1, 2]),
627 /// std::collections::HashSet::from_iter(vec![3, 4]),
628 /// ]))
629 /// .flat_map_unordered(q!(|x| x))
630 /// # }, |mut stream| async move {
631 /// // 1, 2, 3, 4, but in no particular order
632 /// # let mut results = Vec::new();
633 /// # for w in (1..5) {
634 /// # results.push(stream.next().await.unwrap());
635 /// # }
636 /// # results.sort();
637 /// # assert_eq!(results, vec![1, 2, 3, 4]);
638 /// # }));
639 /// # }
640 /// ```
641 pub fn flat_map_unordered<U, I, F>(
642 self,
643 f: impl IntoQuotedMut<'a, F, L>,
644 ) -> Stream<U, L, B, NoOrder, R>
645 where
646 I: IntoIterator<Item = U>,
647 F: Fn(T) -> I + 'a,
648 {
649 let f = crate::singleton_ref::with_singleton_capture(|| {
650 f.splice_fn1_ctx(&self.location).into()
651 });
652 Stream::new(
653 self.location.clone(),
654 HydroNode::FlatMap {
655 f,
656 input: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
657 metadata: self
658 .location
659 .new_node_metadata(Stream::<U, L, B, NoOrder, R>::collection_kind()),
660 },
661 )
662 }
663
664 /// For each item `i` in the input stream, treat `i` as an [`Iterator`] and produce its items one by one.
665 /// The implementation for [`Iterator`] for the element type `T` must produce items in a **deterministic** order.
666 ///
667 /// For example, `T` could be a `Vec`, but not a `HashSet`. If the order of the items in `T` is
668 /// not deterministic, use [`Stream::flatten_unordered`] instead.
669 ///
670 /// ```rust
671 /// # #[cfg(feature = "deploy")] {
672 /// # use hydro_lang::prelude::*;
673 /// # use futures::StreamExt;
674 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
675 /// process
676 /// .source_iter(q!(vec![vec![1, 2], vec![3, 4]]))
677 /// .flatten_ordered()
678 /// # }, |mut stream| async move {
679 /// // 1, 2, 3, 4
680 /// # for w in (1..5) {
681 /// # assert_eq!(stream.next().await.unwrap(), w);
682 /// # }
683 /// # }));
684 /// # }
685 /// ```
686 pub fn flatten_ordered<U>(self) -> Stream<U, L, B, O, R>
687 where
688 T: IntoIterator<Item = U>,
689 {
690 self.flat_map_ordered(q!(|d| d))
691 }
692
693 /// Like [`Stream::flatten_ordered`], but allows the implementation of [`Iterator`]
694 /// for the element type `T` to produce items in any order.
695 ///
696 /// # Example
697 /// ```rust
698 /// # #[cfg(feature = "deploy")] {
699 /// # use hydro_lang::{prelude::*, live_collections::stream::{NoOrder, ExactlyOnce}};
700 /// # use futures::StreamExt;
701 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test::<_, _, _, NoOrder, ExactlyOnce>(|process| {
702 /// process
703 /// .source_iter(q!(vec![
704 /// std::collections::HashSet::<i32>::from_iter(vec![1, 2]),
705 /// std::collections::HashSet::from_iter(vec![3, 4]),
706 /// ]))
707 /// .flatten_unordered()
708 /// # }, |mut stream| async move {
709 /// // 1, 2, 3, 4, but in no particular order
710 /// # let mut results = Vec::new();
711 /// # for w in (1..5) {
712 /// # results.push(stream.next().await.unwrap());
713 /// # }
714 /// # results.sort();
715 /// # assert_eq!(results, vec![1, 2, 3, 4]);
716 /// # }));
717 /// # }
718 /// ```
719 pub fn flatten_unordered<U>(self) -> Stream<U, L, B, NoOrder, R>
720 where
721 T: IntoIterator<Item = U>,
722 {
723 self.flat_map_unordered(q!(|d| d))
724 }
725
726 /// For each item in the input stream, apply `f` to produce a [`futures::stream::Stream`],
727 /// then emit the elements of that stream one by one. When the inner stream yields
728 /// `Pending`, this operator yields as well.
729 pub fn flat_map_stream_blocking<U, S, F>(
730 self,
731 f: impl IntoQuotedMut<'a, F, L>,
732 ) -> Stream<U, L, B, O, R>
733 where
734 S: futures::Stream<Item = U>,
735 F: Fn(T) -> S + 'a,
736 {
737 let f = f.splice_fn1_ctx(&self.location).into();
738 Stream::new(
739 self.location.clone(),
740 HydroNode::FlatMapStreamBlocking {
741 f,
742 input: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
743 metadata: self
744 .location
745 .new_node_metadata(Stream::<U, L, B, O, R>::collection_kind()),
746 },
747 )
748 }
749
750 /// For each item in the input stream, treat it as a [`futures::stream::Stream`] and
751 /// emit its elements one by one. When the inner stream yields `Pending`, this operator
752 /// yields as well.
753 pub fn flatten_stream_blocking<U>(self) -> Stream<U, L, B, O, R>
754 where
755 T: futures::Stream<Item = U>,
756 {
757 self.flat_map_stream_blocking(q!(|d| d))
758 }
759
760 /// Creates a stream containing only the elements of the input stream that satisfy a predicate
761 /// `f`, preserving the order of the elements.
762 ///
763 /// The closure `f` receives a reference `&T` rather than an owned value `T` because filtering does
764 /// not modify or take ownership of the values. If you need to modify the values while filtering
765 /// use [`Stream::filter_map`] instead.
766 ///
767 /// # Example
768 /// ```rust
769 /// # #[cfg(feature = "deploy")] {
770 /// # use hydro_lang::prelude::*;
771 /// # use futures::StreamExt;
772 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
773 /// process
774 /// .source_iter(q!(vec![1, 2, 3, 4]))
775 /// .filter(q!(|&x| x > 2))
776 /// # }, |mut stream| async move {
777 /// // 3, 4
778 /// # for w in (3..5) {
779 /// # assert_eq!(stream.next().await.unwrap(), w);
780 /// # }
781 /// # }));
782 /// # }
783 /// ```
784 pub fn filter<F>(self, f: impl IntoQuotedMut<'a, F, L>) -> Self
785 where
786 F: Fn(&T) -> bool + 'a,
787 {
788 let f = crate::singleton_ref::with_singleton_capture(|| {
789 f.splice_fn1_borrow_ctx(&self.location).into()
790 });
791 Stream::new(
792 self.location.clone(),
793 HydroNode::Filter {
794 f,
795 input: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
796 metadata: self.location.new_node_metadata(Self::collection_kind()),
797 },
798 )
799 }
800
801 /// Splits the stream into two streams based on a predicate, without cloning elements.
802 ///
803 /// Elements for which `f` returns `true` are sent to the first output stream,
804 /// and elements for which `f` returns `false` are sent to the second output stream.
805 ///
806 /// Unlike using `filter` twice, this only evaluates the predicate once per element
807 /// and does not require `T: Clone`.
808 ///
809 /// The closure `f` receives a reference `&T` rather than an owned value `T` because
810 /// the predicate is only used for routing; the element itself is moved to the
811 /// appropriate output stream.
812 ///
813 /// # Example
814 /// ```rust
815 /// # #[cfg(feature = "deploy")] {
816 /// # use hydro_lang::prelude::*;
817 /// # use hydro_lang::live_collections::stream::{NoOrder, ExactlyOnce};
818 /// # use futures::StreamExt;
819 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test::<_, _, _, NoOrder, ExactlyOnce>(|process| {
820 /// let numbers: Stream<_, _, Unbounded> = process.source_iter(q!(vec![1, 2, 3, 4, 5, 6])).into();
821 /// let (evens, odds) = numbers.partition(q!(|&x| x % 2 == 0));
822 /// // evens: 2, 4, 6 tagged with true; odds: 1, 3, 5 tagged with false
823 /// evens.map(q!(|x| (x, true)))
824 /// .merge_unordered(odds.map(q!(|x| (x, false))))
825 /// # }, |mut stream| async move {
826 /// # let mut results = Vec::new();
827 /// # for _ in 0..6 {
828 /// # results.push(stream.next().await.unwrap());
829 /// # }
830 /// # results.sort();
831 /// # assert_eq!(results, vec![(1, false), (2, true), (3, false), (4, true), (5, false), (6, true)]);
832 /// # }));
833 /// # }
834 /// ```
835 #[expect(
836 clippy::type_complexity,
837 reason = "return type mirrors the input stream type"
838 )]
839 pub fn partition<F>(
840 self,
841 f: impl IntoQuotedMut<'a, F, L>,
842 ) -> (Stream<T, L, B, O, R>, Stream<T, L, B, O, R>)
843 where
844 F: Fn(&T) -> bool + 'a,
845 {
846 let f = crate::singleton_ref::with_singleton_capture(|| {
847 f.splice_fn1_borrow_ctx(&self.location).into()
848 });
849 let shared = SharedNode(Rc::new(RefCell::new(
850 self.ir_node.replace(HydroNode::Placeholder),
851 )));
852
853 let true_stream = Stream::new(
854 self.location.clone(),
855 HydroNode::Partition {
856 inner: SharedNode(shared.0.clone()),
857 f: f.clone(),
858 is_true: true,
859 metadata: self.location.new_node_metadata(Self::collection_kind()),
860 },
861 );
862
863 let false_stream = Stream::new(
864 self.location.clone(),
865 HydroNode::Partition {
866 inner: SharedNode(shared.0),
867 f,
868 is_true: false,
869 metadata: self.location.new_node_metadata(Self::collection_kind()),
870 },
871 );
872
873 (true_stream, false_stream)
874 }
875
876 /// An operator that both filters and maps. It yields only the items for which the supplied closure `f` returns `Some(value)`.
877 ///
878 /// # Example
879 /// ```rust
880 /// # #[cfg(feature = "deploy")] {
881 /// # use hydro_lang::prelude::*;
882 /// # use futures::StreamExt;
883 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
884 /// process
885 /// .source_iter(q!(vec!["1", "hello", "world", "2"]))
886 /// .filter_map(q!(|s| s.parse::<usize>().ok()))
887 /// # }, |mut stream| async move {
888 /// // 1, 2
889 /// # for w in (1..3) {
890 /// # assert_eq!(stream.next().await.unwrap(), w);
891 /// # }
892 /// # }));
893 /// # }
894 /// ```
895 pub fn filter_map<U, F>(self, f: impl IntoQuotedMut<'a, F, L>) -> Stream<U, L, B, O, R>
896 where
897 F: Fn(T) -> Option<U> + 'a,
898 {
899 let f = f.splice_fn1_ctx(&self.location).into();
900 Stream::new(
901 self.location.clone(),
902 HydroNode::FilterMap {
903 f,
904 input: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
905 metadata: self
906 .location
907 .new_node_metadata(Stream::<U, L, B, O, R>::collection_kind()),
908 },
909 )
910 }
911
912 /// Generates a stream that maps each input element `i` to a tuple `(i, x)`,
913 /// where `x` is the final value of `other`, a bounded [`Singleton`] or [`Optional`].
914 /// If `other` is an empty [`Optional`], no values will be produced.
915 ///
916 /// # Example
917 /// ```rust
918 /// # #[cfg(feature = "deploy")] {
919 /// # use hydro_lang::prelude::*;
920 /// # use futures::StreamExt;
921 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
922 /// let tick = process.tick();
923 /// let batch = process
924 /// .source_iter(q!(vec![1, 2, 3, 4]))
925 /// .batch(&tick, nondet!(/** test */));
926 /// let count = batch.clone().count(); // `count()` returns a singleton
927 /// batch.cross_singleton(count).all_ticks()
928 /// # }, |mut stream| async move {
929 /// // (1, 4), (2, 4), (3, 4), (4, 4)
930 /// # for w in vec![(1, 4), (2, 4), (3, 4), (4, 4)] {
931 /// # assert_eq!(stream.next().await.unwrap(), w);
932 /// # }
933 /// # }));
934 /// # }
935 /// ```
936 pub fn cross_singleton<O2>(
937 self,
938 other: impl Into<Optional<O2, L, Bounded>>,
939 ) -> Stream<(T, O2), L, B, O, R>
940 where
941 O2: Clone,
942 {
943 let other: Optional<O2, L, Bounded> = other.into();
944 check_matching_location(&self.location, &other.location);
945
946 Stream::new(
947 self.location.clone(),
948 HydroNode::CrossSingleton {
949 left: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
950 right: Box::new(other.ir_node.replace(HydroNode::Placeholder)),
951 metadata: self
952 .location
953 .new_node_metadata(Stream::<(T, O2), L, B, O, R>::collection_kind()),
954 },
955 )
956 }
957
958 /// Passes this stream through if the boolean signal is `true`, otherwise the output is empty.
959 ///
960 /// # Example
961 /// ```rust
962 /// # #[cfg(feature = "deploy")] {
963 /// # use hydro_lang::prelude::*;
964 /// # use futures::StreamExt;
965 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
966 /// let tick = process.tick();
967 /// // ticks are lazy by default, forces the second tick to run
968 /// tick.spin_batch(q!(1)).all_ticks().for_each(q!(|_| {}));
969 ///
970 /// let signal = tick.optional_first_tick(q!(())).is_some(); // true on tick 1, false on tick 2
971 /// let batch_first_tick = process
972 /// .source_iter(q!(vec![1, 2, 3, 4]))
973 /// .batch(&tick, nondet!(/** test */));
974 /// let batch_second_tick = process
975 /// .source_iter(q!(vec![5, 6, 7, 8]))
976 /// .batch(&tick, nondet!(/** test */))
977 /// .defer_tick();
978 /// batch_first_tick.chain(batch_second_tick)
979 /// .filter_if(signal)
980 /// .all_ticks()
981 /// # }, |mut stream| async move {
982 /// // [1, 2, 3, 4]
983 /// # for w in vec![1, 2, 3, 4] {
984 /// # assert_eq!(stream.next().await.unwrap(), w);
985 /// # }
986 /// # }));
987 /// # }
988 /// ```
989 pub fn filter_if(self, signal: Singleton<bool, L, Bounded>) -> Stream<T, L, B, O, R> {
990 self.cross_singleton(signal.filter(q!(|b| *b)))
991 .map(q!(|(d, _)| d))
992 }
993
994 /// Passes this stream through if the argument (a [`Bounded`] [`Optional`]`) is non-null, otherwise the output is empty.
995 ///
996 /// Useful for gating the release of elements based on a condition, such as only processing requests if you are the
997 /// leader of a cluster.
998 ///
999 /// # Example
1000 /// ```rust
1001 /// # #[cfg(feature = "deploy")] {
1002 /// # use hydro_lang::prelude::*;
1003 /// # use futures::StreamExt;
1004 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
1005 /// let tick = process.tick();
1006 /// // ticks are lazy by default, forces the second tick to run
1007 /// tick.spin_batch(q!(1)).all_ticks().for_each(q!(|_| {}));
1008 ///
1009 /// let batch_first_tick = process
1010 /// .source_iter(q!(vec![1, 2, 3, 4]))
1011 /// .batch(&tick, nondet!(/** test */));
1012 /// let batch_second_tick = process
1013 /// .source_iter(q!(vec![5, 6, 7, 8]))
1014 /// .batch(&tick, nondet!(/** test */))
1015 /// .defer_tick(); // appears on the second tick
1016 /// let some_on_first_tick = tick.optional_first_tick(q!(()));
1017 /// batch_first_tick.chain(batch_second_tick)
1018 /// .filter_if_some(some_on_first_tick)
1019 /// .all_ticks()
1020 /// # }, |mut stream| async move {
1021 /// // [1, 2, 3, 4]
1022 /// # for w in vec![1, 2, 3, 4] {
1023 /// # assert_eq!(stream.next().await.unwrap(), w);
1024 /// # }
1025 /// # }));
1026 /// # }
1027 /// ```
1028 #[deprecated(note = "use `filter_if` with `Optional::is_some()` instead")]
1029 pub fn filter_if_some<U>(self, signal: Optional<U, L, Bounded>) -> Stream<T, L, B, O, R> {
1030 self.filter_if(signal.is_some())
1031 }
1032
1033 /// Passes this stream through if the argument (a [`Bounded`] [`Optional`]`) is null, otherwise the output is empty.
1034 ///
1035 /// Useful for gating the release of elements based on a condition, such as triggering a protocol if you are missing
1036 /// some local state.
1037 ///
1038 /// # Example
1039 /// ```rust
1040 /// # #[cfg(feature = "deploy")] {
1041 /// # use hydro_lang::prelude::*;
1042 /// # use futures::StreamExt;
1043 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
1044 /// let tick = process.tick();
1045 /// // ticks are lazy by default, forces the second tick to run
1046 /// tick.spin_batch(q!(1)).all_ticks().for_each(q!(|_| {}));
1047 ///
1048 /// let batch_first_tick = process
1049 /// .source_iter(q!(vec![1, 2, 3, 4]))
1050 /// .batch(&tick, nondet!(/** test */));
1051 /// let batch_second_tick = process
1052 /// .source_iter(q!(vec![5, 6, 7, 8]))
1053 /// .batch(&tick, nondet!(/** test */))
1054 /// .defer_tick(); // appears on the second tick
1055 /// let some_on_first_tick = tick.optional_first_tick(q!(()));
1056 /// batch_first_tick.chain(batch_second_tick)
1057 /// .filter_if_none(some_on_first_tick)
1058 /// .all_ticks()
1059 /// # }, |mut stream| async move {
1060 /// // [5, 6, 7, 8]
1061 /// # for w in vec![5, 6, 7, 8] {
1062 /// # assert_eq!(stream.next().await.unwrap(), w);
1063 /// # }
1064 /// # }));
1065 /// # }
1066 /// ```
1067 #[deprecated(note = "use `filter_if` with `!Optional::is_some()` instead")]
1068 pub fn filter_if_none<U>(self, other: Optional<U, L, Bounded>) -> Stream<T, L, B, O, R> {
1069 self.filter_if(other.is_none())
1070 }
1071
1072 /// Forms the cross-product (Cartesian product, cross-join) of the items in the 2 input streams,
1073 /// returning all tupled pairs.
1074 ///
1075 /// When the right side is [`Bounded`], it is accumulated first and the left side streams
1076 /// through, preserving the left side's ordering. When both sides are [`Unbounded`], a
1077 /// symmetric hash join is used and ordering is [`NoOrder`].
1078 ///
1079 /// # Example
1080 /// ```rust
1081 /// # #[cfg(feature = "deploy")] {
1082 /// # use hydro_lang::prelude::*;
1083 /// # use std::collections::HashSet;
1084 /// # use futures::StreamExt;
1085 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
1086 /// let tick = process.tick();
1087 /// let stream1 = process.source_iter(q!(vec![1, 2]));
1088 /// let stream2 = process.source_iter(q!(vec!['a', 'b']));
1089 /// stream1.cross_product(stream2)
1090 /// # }, |mut stream| async move {
1091 /// // (1, 'a'), (1, 'b'), (2, 'a'), (2, 'b') in any order
1092 /// # let expected = HashSet::from([(1, 'a'), (1, 'b'), (2, 'a'), (2, 'b')]);
1093 /// # stream.map(|i| assert!(expected.contains(&i)));
1094 /// # }));
1095 /// # }
1096 #[expect(
1097 clippy::type_complexity,
1098 reason = "MinRetries projection in return type"
1099 )]
1100 pub fn cross_product<T2, B2: Boundedness, O2: Ordering, R2: Retries>(
1101 self,
1102 other: Stream<T2, L, B2, O2, R2>,
1103 ) -> Stream<(T, T2), L, B, B2::PreserveOrderIfBounded<O>, <R as MinRetries<R2>>::Min>
1104 where
1105 T: Clone,
1106 T2: Clone,
1107 R: MinRetries<R2>,
1108 {
1109 self.map(q!(|v| ((), v)))
1110 .join(other.map(q!(|v| ((), v))))
1111 .map(q!(|((), (v1, v2))| (v1, v2)))
1112 }
1113
1114 /// Takes one stream as input and filters out any duplicate occurrences. The output
1115 /// contains all unique values from the input.
1116 ///
1117 /// # Example
1118 /// ```rust
1119 /// # #[cfg(feature = "deploy")] {
1120 /// # use hydro_lang::prelude::*;
1121 /// # use futures::StreamExt;
1122 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
1123 /// let tick = process.tick();
1124 /// process.source_iter(q!(vec![1, 2, 3, 2, 1, 4])).unique()
1125 /// # }, |mut stream| async move {
1126 /// # for w in vec![1, 2, 3, 4] {
1127 /// # assert_eq!(stream.next().await.unwrap(), w);
1128 /// # }
1129 /// # }));
1130 /// # }
1131 /// ```
1132 pub fn unique(self) -> Stream<T, L, B, O, ExactlyOnce>
1133 where
1134 T: Eq + Hash,
1135 {
1136 Stream::new(
1137 self.location.clone(),
1138 HydroNode::Unique {
1139 input: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
1140 metadata: self
1141 .location
1142 .new_node_metadata(Stream::<T, L, B, O, ExactlyOnce>::collection_kind()),
1143 },
1144 )
1145 }
1146
1147 /// Outputs everything in this stream that is *not* contained in the `other` stream.
1148 ///
1149 /// The `other` stream must be [`Bounded`], since this function will wait until
1150 /// all its elements are available before producing any output.
1151 /// # Example
1152 /// ```rust
1153 /// # #[cfg(feature = "deploy")] {
1154 /// # use hydro_lang::prelude::*;
1155 /// # use futures::StreamExt;
1156 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
1157 /// let tick = process.tick();
1158 /// let stream = process
1159 /// .source_iter(q!(vec![ 1, 2, 3, 4 ]))
1160 /// .batch(&tick, nondet!(/** test */));
1161 /// let batch = process
1162 /// .source_iter(q!(vec![1, 2]))
1163 /// .batch(&tick, nondet!(/** test */));
1164 /// stream.filter_not_in(batch).all_ticks()
1165 /// # }, |mut stream| async move {
1166 /// # for w in vec![3, 4] {
1167 /// # assert_eq!(stream.next().await.unwrap(), w);
1168 /// # }
1169 /// # }));
1170 /// # }
1171 /// ```
1172 pub fn filter_not_in<O2: Ordering, B2>(self, other: Stream<T, L, B2, O2, R>) -> Self
1173 where
1174 T: Eq + Hash,
1175 B2: IsBounded,
1176 {
1177 check_matching_location(&self.location, &other.location);
1178
1179 Stream::new(
1180 self.location.clone(),
1181 HydroNode::Difference {
1182 pos: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
1183 neg: Box::new(other.ir_node.replace(HydroNode::Placeholder)),
1184 metadata: self
1185 .location
1186 .new_node_metadata(Stream::<T, L, Bounded, O, R>::collection_kind()),
1187 },
1188 )
1189 }
1190
1191 /// An operator which allows you to "inspect" each element of a stream without
1192 /// modifying it. The closure `f` is called on a reference to each item. This is
1193 /// mainly useful for debugging, and should not be used to generate side-effects.
1194 ///
1195 /// # Example
1196 /// ```rust
1197 /// # #[cfg(feature = "deploy")] {
1198 /// # use hydro_lang::prelude::*;
1199 /// # use futures::StreamExt;
1200 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
1201 /// let nums = process.source_iter(q!(vec![1, 2]));
1202 /// // prints "1 * 10 = 10" and "2 * 10 = 20"
1203 /// nums.inspect(q!(|x| println!("{} * 10 = {}", x, x * 10)))
1204 /// # }, |mut stream| async move {
1205 /// # for w in vec![1, 2] {
1206 /// # assert_eq!(stream.next().await.unwrap(), w);
1207 /// # }
1208 /// # }));
1209 /// # }
1210 /// ```
1211 pub fn inspect<F>(self, f: impl IntoQuotedMut<'a, F, L::DropConsistency>) -> Self
1212 where
1213 F: Fn(&T) + 'a,
1214 {
1215 let f = crate::singleton_ref::with_singleton_capture(|| {
1216 f.splice_fn1_borrow_ctx(&self.location.drop_consistency())
1217 .into()
1218 });
1219
1220 Stream::new(
1221 self.location.clone(),
1222 HydroNode::Inspect {
1223 f,
1224 input: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
1225 metadata: self.location.new_node_metadata(Self::collection_kind()),
1226 },
1227 )
1228 }
1229
1230 /// Executes the provided closure for every element in this stream.
1231 ///
1232 /// Because the closure may have side effects, the stream must have deterministic order
1233 /// ([`TotalOrder`]) and no retries ([`ExactlyOnce`]). If the side effects can tolerate
1234 /// out-of-order or duplicate execution, use [`Stream::assume_ordering`] and
1235 /// [`Stream::assume_retries`] with an explanation for why this is the case.
1236 pub fn for_each<F: Fn(T) + 'a>(self, f: impl IntoQuotedMut<'a, F, L>)
1237 where
1238 O: IsOrdered,
1239 R: IsExactlyOnce,
1240 {
1241 let f = f.splice_fn1_ctx(&self.location).into();
1242 self.location
1243 .flow_state()
1244 .borrow_mut()
1245 .push_root(HydroRoot::ForEach {
1246 input: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
1247 f,
1248 op_metadata: HydroIrOpMetadata::new(),
1249 });
1250 }
1251
1252 /// Sends all elements of this stream to a provided [`futures::Sink`], such as an external
1253 /// TCP socket to some other server. You should _not_ use this API for interacting with
1254 /// external clients, instead see [`Location::bidi_external_many_bytes`] and
1255 /// [`Location::bidi_external_many_bincode`]. This should be used for custom, low-level
1256 /// interaction with asynchronous sinks.
1257 pub fn dest_sink<S>(self, sink: impl QuotedWithContext<'a, S, L>)
1258 where
1259 O: IsOrdered,
1260 R: IsExactlyOnce,
1261 S: 'a + futures::Sink<T> + Unpin,
1262 {
1263 self.location
1264 .flow_state()
1265 .borrow_mut()
1266 .push_root(HydroRoot::DestSink {
1267 sink: sink.splice_typed_ctx(&self.location).into(),
1268 input: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
1269 op_metadata: HydroIrOpMetadata::new(),
1270 });
1271 }
1272
1273 /// Maps each element `x` of the stream to `(i, x)`, where `i` is the index of the element.
1274 ///
1275 /// # Example
1276 /// ```rust
1277 /// # #[cfg(feature = "deploy")] {
1278 /// # use hydro_lang::{prelude::*, live_collections::stream::{TotalOrder, ExactlyOnce}};
1279 /// # use futures::StreamExt;
1280 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test::<_, _, _, TotalOrder, ExactlyOnce>(|process| {
1281 /// let tick = process.tick();
1282 /// let numbers = process.source_iter(q!(vec![1, 2, 3, 4]));
1283 /// numbers.enumerate()
1284 /// # }, |mut stream| async move {
1285 /// // (0, 1), (1, 2), (2, 3), (3, 4)
1286 /// # for w in vec![(0, 1), (1, 2), (2, 3), (3, 4)] {
1287 /// # assert_eq!(stream.next().await.unwrap(), w);
1288 /// # }
1289 /// # }));
1290 /// # }
1291 /// ```
1292 pub fn enumerate(self) -> Stream<(usize, T), L, B, O, R>
1293 where
1294 O: IsOrdered,
1295 R: IsExactlyOnce,
1296 {
1297 Stream::new(
1298 self.location.clone(),
1299 HydroNode::Enumerate {
1300 input: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
1301 metadata: self.location.new_node_metadata(Stream::<
1302 (usize, T),
1303 L,
1304 B,
1305 TotalOrder,
1306 ExactlyOnce,
1307 >::collection_kind()),
1308 },
1309 )
1310 }
1311
1312 /// Combines elements of the stream into a [`Singleton`], by starting with an intitial value,
1313 /// generated by the `init` closure, and then applying the `comb` closure to each element in the stream.
1314 /// Unlike iterators, `comb` takes the accumulator by `&mut` reference, so that it can be modified in place.
1315 ///
1316 /// Depending on the input stream guarantees, the closure may need to be commutative
1317 /// (for unordered streams) or idempotent (for streams with non-deterministic duplicates).
1318 ///
1319 /// # Example
1320 /// ```rust
1321 /// # #[cfg(feature = "deploy")] {
1322 /// # use hydro_lang::prelude::*;
1323 /// # use futures::StreamExt;
1324 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
1325 /// let words = process.source_iter(q!(vec!["HELLO", "WORLD"]));
1326 /// words
1327 /// .fold(q!(|| String::new()), q!(|acc, x| acc.push_str(x)))
1328 /// .into_stream()
1329 /// # }, |mut stream| async move {
1330 /// // "HELLOWORLD"
1331 /// # assert_eq!(stream.next().await.unwrap(), "HELLOWORLD");
1332 /// # }));
1333 /// # }
1334 /// ```
1335 pub fn fold<A, I, F, C, Idemp, M, B2: SingletonBound>(
1336 self,
1337 init: impl IntoQuotedMut<'a, I, L>,
1338 comb: impl IntoQuotedMut<'a, F, L, AggFuncAlgebra<C, Idemp, M>>,
1339 ) -> Singleton<A, L, B2>
1340 where
1341 I: Fn() -> A + 'a,
1342 F: 'a + Fn(&mut A, T),
1343 C: ValidCommutativityFor<O>,
1344 Idemp: ValidIdempotenceFor<R>,
1345 B: ApplyMonotoneStream<M, B2>,
1346 {
1347 let init = init.splice_fn0_ctx(&self.location).into();
1348 let (comb, proof) = comb.splice_fn2_borrow_mut_ctx_props(&self.location);
1349 proof.register_proof(&comb);
1350
1351 // Only assume_retries (for idempotence), not assume_ordering.
1352 // The fold hook in the simulator handles ordering non-determinism directly.
1353 let nondet = nondet!(/** the combinator function is commutative and idempotent */);
1354 let retried: Stream<T, L::DropConsistency, B, O, ExactlyOnce> = self.assume_retries(nondet);
1355
1356 let core = HydroNode::Fold {
1357 init,
1358 acc: comb.into(),
1359 input: Box::new(retried.ir_node.replace(HydroNode::Placeholder)),
1360 metadata: retried
1361 .location
1362 .new_node_metadata(Singleton::<A, L::DropConsistency, B2>::collection_kind()),
1363 // we do not guarantee consistency at this point because if the algebraic properties
1364 // do not hold in practice, replica consistency may fail to be maintained, so we
1365 // would like the simulator to assert consistency; in the future, this will be dynamic
1366 // based on the proof mechanism
1367 };
1368
1369 Singleton::new(retried.location.clone(), core)
1370 .assert_has_consistency_of(manual_proof!(/** algebraic properties */))
1371 }
1372
1373 /// Combines elements of the stream into an [`Optional`], by starting with the first element in the stream,
1374 /// and then applying the `comb` closure to each element in the stream. The [`Optional`] will be empty
1375 /// until the first element in the input arrives. Unlike iterators, `comb` takes the accumulator by `&mut`
1376 /// reference, so that it can be modified in place.
1377 ///
1378 /// Depending on the input stream guarantees, the closure may need to be commutative
1379 /// (for unordered streams) or idempotent (for streams with non-deterministic duplicates).
1380 ///
1381 /// # Example
1382 /// ```rust
1383 /// # #[cfg(feature = "deploy")] {
1384 /// # use hydro_lang::prelude::*;
1385 /// # use futures::StreamExt;
1386 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
1387 /// let bools = process.source_iter(q!(vec![false, true, false]));
1388 /// bools.reduce(q!(|acc, x| *acc |= x)).into_stream()
1389 /// # }, |mut stream| async move {
1390 /// // true
1391 /// # assert_eq!(stream.next().await.unwrap(), true);
1392 /// # }));
1393 /// # }
1394 /// ```
1395 pub fn reduce<F, C, Idemp>(
1396 self,
1397 comb: impl IntoQuotedMut<'a, F, L, AggFuncAlgebra<C, Idemp>>,
1398 ) -> Optional<T, L, B>
1399 where
1400 F: Fn(&mut T, T) + 'a,
1401 C: ValidCommutativityFor<O>,
1402 Idemp: ValidIdempotenceFor<R>,
1403 {
1404 let (f, proof) = comb.splice_fn2_borrow_mut_ctx_props(&self.location);
1405 proof.register_proof(&f);
1406
1407 let nondet = nondet!(/** the combinator function is commutative and idempotent */);
1408 let ordered_etc: Stream<T, L::DropConsistency, B> =
1409 self.assume_retries(nondet).assume_ordering(nondet);
1410
1411 let core = HydroNode::Reduce {
1412 f: f.into(),
1413 input: Box::new(ordered_etc.ir_node.replace(HydroNode::Placeholder)),
1414 metadata: ordered_etc
1415 .location
1416 .new_node_metadata(Optional::<T, L::DropConsistency, B>::collection_kind()),
1417 };
1418
1419 Optional::new(ordered_etc.location.clone(), core)
1420 .assert_has_consistency_of(manual_proof!(/** algebraic properties */))
1421 }
1422
1423 /// Computes the maximum element in the stream as an [`Optional`], which
1424 /// will be empty until the first element in the input arrives.
1425 ///
1426 /// # Example
1427 /// ```rust
1428 /// # #[cfg(feature = "deploy")] {
1429 /// # use hydro_lang::prelude::*;
1430 /// # use futures::StreamExt;
1431 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
1432 /// let tick = process.tick();
1433 /// let numbers = process.source_iter(q!(vec![1, 2, 3, 4]));
1434 /// let batch = numbers.batch(&tick, nondet!(/** test */));
1435 /// batch.max().all_ticks()
1436 /// # }, |mut stream| async move {
1437 /// // 4
1438 /// # assert_eq!(stream.next().await.unwrap(), 4);
1439 /// # }));
1440 /// # }
1441 /// ```
1442 pub fn max(self) -> Optional<T, L, B>
1443 where
1444 T: Ord,
1445 {
1446 self.assume_retries_trusted::<ExactlyOnce>(nondet!(/** max is idempotent */))
1447 .assume_ordering_trusted_bounded::<TotalOrder>(
1448 nondet!(/** max is commutative, but order affects intermediates */),
1449 )
1450 .reduce(q!(|curr, new| {
1451 if new > *curr {
1452 *curr = new;
1453 }
1454 }))
1455 }
1456
1457 /// Computes the minimum element in the stream as an [`Optional`], which
1458 /// will be empty until the first element in the input arrives.
1459 ///
1460 /// # Example
1461 /// ```rust
1462 /// # #[cfg(feature = "deploy")] {
1463 /// # use hydro_lang::prelude::*;
1464 /// # use futures::StreamExt;
1465 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
1466 /// let tick = process.tick();
1467 /// let numbers = process.source_iter(q!(vec![1, 2, 3, 4]));
1468 /// let batch = numbers.batch(&tick, nondet!(/** test */));
1469 /// batch.min().all_ticks()
1470 /// # }, |mut stream| async move {
1471 /// // 1
1472 /// # assert_eq!(stream.next().await.unwrap(), 1);
1473 /// # }));
1474 /// # }
1475 /// ```
1476 pub fn min(self) -> Optional<T, L, B>
1477 where
1478 T: Ord,
1479 {
1480 self.assume_retries_trusted::<ExactlyOnce>(nondet!(/** min is idempotent */))
1481 .assume_ordering_trusted_bounded::<TotalOrder>(
1482 nondet!(/** max is commutative, but order affects intermediates */),
1483 )
1484 .reduce(q!(|curr, new| {
1485 if new < *curr {
1486 *curr = new;
1487 }
1488 }))
1489 }
1490
1491 /// Computes the first element in the stream as an [`Optional`], which
1492 /// will be empty until the first element in the input arrives.
1493 ///
1494 /// This requires the stream to have a [`TotalOrder`] guarantee, otherwise
1495 /// re-ordering of elements may cause the first element to change.
1496 ///
1497 /// # Example
1498 /// ```rust
1499 /// # #[cfg(feature = "deploy")] {
1500 /// # use hydro_lang::prelude::*;
1501 /// # use futures::StreamExt;
1502 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
1503 /// let tick = process.tick();
1504 /// let numbers = process.source_iter(q!(vec![1, 2, 3, 4]));
1505 /// let batch = numbers.batch(&tick, nondet!(/** test */));
1506 /// batch.first().all_ticks()
1507 /// # }, |mut stream| async move {
1508 /// // 1
1509 /// # assert_eq!(stream.next().await.unwrap(), 1);
1510 /// # }));
1511 /// # }
1512 /// ```
1513 pub fn first(self) -> Optional<T, L, B>
1514 where
1515 O: IsOrdered,
1516 {
1517 self.make_totally_ordered()
1518 .assume_retries_trusted::<ExactlyOnce>(nondet!(/** first is idempotent */))
1519 .generator(q!(|| ()), q!(|_, item| Generate::Return(item)))
1520 .reduce(q!(|_, _| {}))
1521 }
1522
1523 /// Computes the last element in the stream as an [`Optional`], which
1524 /// will be empty until an element in the input arrives.
1525 ///
1526 /// This requires the stream to have a [`TotalOrder`] guarantee, otherwise
1527 /// re-ordering of elements may cause the last element to change.
1528 ///
1529 /// # Example
1530 /// ```rust
1531 /// # #[cfg(feature = "deploy")] {
1532 /// # use hydro_lang::prelude::*;
1533 /// # use futures::StreamExt;
1534 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
1535 /// let tick = process.tick();
1536 /// let numbers = process.source_iter(q!(vec![1, 2, 3, 4]));
1537 /// let batch = numbers.batch(&tick, nondet!(/** test */));
1538 /// batch.last().all_ticks()
1539 /// # }, |mut stream| async move {
1540 /// // 4
1541 /// # assert_eq!(stream.next().await.unwrap(), 4);
1542 /// # }));
1543 /// # }
1544 /// ```
1545 pub fn last(self) -> Optional<T, L, B>
1546 where
1547 O: IsOrdered,
1548 {
1549 self.make_totally_ordered()
1550 .assume_retries_trusted::<ExactlyOnce>(nondet!(/** last is idempotent */))
1551 .reduce(q!(|curr, new| *curr = new))
1552 }
1553
1554 /// Returns a stream containing at most the first `n` elements of the input stream,
1555 /// preserving the original order. Similar to `LIMIT` in SQL.
1556 ///
1557 /// This requires the stream to have a [`TotalOrder`] guarantee and [`ExactlyOnce`]
1558 /// retries, since the result depends on the order and cardinality of elements.
1559 ///
1560 /// # Example
1561 /// ```rust
1562 /// # #[cfg(feature = "deploy")] {
1563 /// # use hydro_lang::prelude::*;
1564 /// # use futures::StreamExt;
1565 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
1566 /// let numbers = process.source_iter(q!(vec![10, 20, 30, 40, 50]));
1567 /// numbers.limit(q!(3))
1568 /// # }, |mut stream| async move {
1569 /// // 10, 20, 30
1570 /// # for w in vec![10, 20, 30] {
1571 /// # assert_eq!(stream.next().await.unwrap(), w);
1572 /// # }
1573 /// # }));
1574 /// # }
1575 /// ```
1576 pub fn limit(
1577 self,
1578 n: impl QuotedWithContext<'a, usize, L> + Copy + 'a,
1579 ) -> Stream<T, L, B, TotalOrder, ExactlyOnce>
1580 where
1581 O: IsOrdered,
1582 R: IsExactlyOnce,
1583 {
1584 self.generator(
1585 q!(|| 0usize),
1586 q!(move |count, item| {
1587 if *count == n {
1588 Generate::Break
1589 } else {
1590 *count += 1;
1591 if *count == n {
1592 Generate::Return(item)
1593 } else {
1594 Generate::Yield(item)
1595 }
1596 }
1597 }),
1598 )
1599 }
1600
1601 /// Collects all the elements of this stream into a single [`Vec`] element.
1602 ///
1603 /// If the input stream is [`Unbounded`], the output [`Singleton`] will be [`Unbounded`] as
1604 /// well, which means that the value of the [`Vec`] will asynchronously grow as new elements
1605 /// are added. On such a value, you can use [`Singleton::snapshot`] to grab an instance of
1606 /// the vector at an arbitrary point in time.
1607 ///
1608 /// # Example
1609 /// ```rust
1610 /// # #[cfg(feature = "deploy")] {
1611 /// # use hydro_lang::prelude::*;
1612 /// # use futures::StreamExt;
1613 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
1614 /// let tick = process.tick();
1615 /// let numbers = process.source_iter(q!(vec![1, 2, 3, 4]));
1616 /// let batch = numbers.batch(&tick, nondet!(/** test */));
1617 /// batch.collect_vec().all_ticks() // emit each tick's Vec into an unbounded stream
1618 /// # }, |mut stream| async move {
1619 /// // [ vec![1, 2, 3, 4] ]
1620 /// # for w in vec![vec![1, 2, 3, 4]] {
1621 /// # assert_eq!(stream.next().await.unwrap(), w);
1622 /// # }
1623 /// # }));
1624 /// # }
1625 /// ```
1626 pub fn collect_vec(self) -> Singleton<Vec<T>, L, B>
1627 where
1628 O: IsOrdered,
1629 R: IsExactlyOnce,
1630 {
1631 self.make_totally_ordered().make_exactly_once().fold(
1632 q!(|| vec![]),
1633 q!(|acc, v| {
1634 acc.push(v);
1635 }),
1636 )
1637 }
1638
1639 /// Applies a function to each element of the stream, maintaining an internal state (accumulator)
1640 /// and emitting each intermediate result.
1641 ///
1642 /// Unlike `fold` which only returns the final accumulated value, `scan` produces a new stream
1643 /// containing all intermediate accumulated values. The scan operation can also terminate early
1644 /// by returning `None`.
1645 ///
1646 /// The function takes a mutable reference to the accumulator and the current element, and returns
1647 /// an `Option<U>`. If the function returns `Some(value)`, `value` is emitted to the output stream.
1648 /// If the function returns `None`, the stream is terminated and no more elements are processed.
1649 ///
1650 /// # Examples
1651 ///
1652 /// Basic usage - running sum:
1653 /// ```rust
1654 /// # #[cfg(feature = "deploy")] {
1655 /// # use hydro_lang::prelude::*;
1656 /// # use futures::StreamExt;
1657 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
1658 /// process.source_iter(q!(vec![1, 2, 3, 4])).scan(
1659 /// q!(|| 0),
1660 /// q!(|acc, x| {
1661 /// *acc += x;
1662 /// Some(*acc)
1663 /// }),
1664 /// )
1665 /// # }, |mut stream| async move {
1666 /// // Output: 1, 3, 6, 10
1667 /// # for w in vec![1, 3, 6, 10] {
1668 /// # assert_eq!(stream.next().await.unwrap(), w);
1669 /// # }
1670 /// # }));
1671 /// # }
1672 /// ```
1673 ///
1674 /// Early termination example:
1675 /// ```rust
1676 /// # #[cfg(feature = "deploy")] {
1677 /// # use hydro_lang::prelude::*;
1678 /// # use futures::StreamExt;
1679 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
1680 /// process.source_iter(q!(vec![1, 2, 3, 4])).scan(
1681 /// q!(|| 1),
1682 /// q!(|state, x| {
1683 /// *state = *state * x;
1684 /// if *state > 6 {
1685 /// None // Terminate the stream
1686 /// } else {
1687 /// Some(-*state)
1688 /// }
1689 /// }),
1690 /// )
1691 /// # }, |mut stream| async move {
1692 /// // Output: -1, -2, -6
1693 /// # for w in vec![-1, -2, -6] {
1694 /// # assert_eq!(stream.next().await.unwrap(), w);
1695 /// # }
1696 /// # }));
1697 /// # }
1698 /// ```
1699 pub fn scan<A, U, I, F>(
1700 self,
1701 init: impl IntoQuotedMut<'a, I, L>,
1702 f: impl IntoQuotedMut<'a, F, L>,
1703 ) -> Stream<U, L, B, TotalOrder, ExactlyOnce>
1704 where
1705 O: IsOrdered,
1706 R: IsExactlyOnce,
1707 I: Fn() -> A + 'a,
1708 F: Fn(&mut A, T) -> Option<U> + 'a,
1709 {
1710 let init = init.splice_fn0_ctx(&self.location).into();
1711 let f = f.splice_fn2_borrow_mut_ctx(&self.location).into();
1712
1713 Stream::new(
1714 self.location.clone(),
1715 HydroNode::Scan {
1716 init,
1717 acc: f,
1718 input: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
1719 metadata: self.location.new_node_metadata(
1720 Stream::<U, L, B, TotalOrder, ExactlyOnce>::collection_kind(),
1721 ),
1722 },
1723 )
1724 }
1725
1726 /// Async version of [`Stream::scan`]. Applies an async function to each element of the
1727 /// stream, maintaining an internal state (accumulator) and emitting the values returned
1728 /// by the function.
1729 ///
1730 /// The closure runs synchronously (so it can mutate the accumulator), then returns a
1731 /// future. The future is polled to completion. If it resolves to `Some`, the value is
1732 /// emitted. If it resolves to `None`, the item is filtered out.
1733 ///
1734 /// # Examples
1735 ///
1736 /// ```rust
1737 /// # #[cfg(feature = "deploy")] {
1738 /// # use hydro_lang::prelude::*;
1739 /// # use futures::StreamExt;
1740 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
1741 /// process
1742 /// .source_iter(q!(vec![1, 2, 3, 4]))
1743 /// .scan_async_blocking(
1744 /// q!(|| 0),
1745 /// q!(|acc, x| {
1746 /// *acc += x;
1747 /// let val = *acc;
1748 /// async move { Some(val) }
1749 /// }),
1750 /// )
1751 /// # }, |mut stream| async move {
1752 /// // Output: 1, 3, 6, 10
1753 /// # for w in vec![1, 3, 6, 10] {
1754 /// # assert_eq!(stream.next().await.unwrap(), w);
1755 /// # }
1756 /// # }));
1757 /// # }
1758 /// ```
1759 pub fn scan_async_blocking<A, U, I, F, Fut>(
1760 self,
1761 init: impl IntoQuotedMut<'a, I, L>,
1762 f: impl IntoQuotedMut<'a, F, L>,
1763 ) -> Stream<U, L, B, TotalOrder, ExactlyOnce>
1764 where
1765 O: IsOrdered,
1766 R: IsExactlyOnce,
1767 I: Fn() -> A + 'a,
1768 F: Fn(&mut A, T) -> Fut + 'a,
1769 Fut: Future<Output = Option<U>> + 'a,
1770 {
1771 let init = init.splice_fn0_ctx(&self.location).into();
1772 let f = f.splice_fn2_borrow_mut_ctx(&self.location).into();
1773
1774 Stream::new(
1775 self.location.clone(),
1776 HydroNode::ScanAsyncBlocking {
1777 init,
1778 acc: f,
1779 input: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
1780 metadata: self.location.new_node_metadata(
1781 Stream::<U, L, B, TotalOrder, ExactlyOnce>::collection_kind(),
1782 ),
1783 },
1784 )
1785 }
1786
1787 /// Iteratively processes the elements of the stream using a state machine that can yield
1788 /// elements as it processes its inputs. This is designed to mirror the unstable generator
1789 /// syntax in Rust, without requiring special syntax.
1790 ///
1791 /// Like [`Stream::scan`], this function takes in an initializer that emits the initial
1792 /// state. The second argument defines the processing logic, taking in a mutable reference
1793 /// to the state and the value to be processed. It emits a [`Generate`] value, whose
1794 /// variants define what is emitted and whether further inputs should be processed.
1795 ///
1796 /// # Example
1797 /// ```rust
1798 /// # #[cfg(feature = "deploy")] {
1799 /// # use hydro_lang::prelude::*;
1800 /// # use futures::StreamExt;
1801 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
1802 /// process.source_iter(q!(vec![1, 3, 100, 10])).generator(
1803 /// q!(|| 0),
1804 /// q!(|acc, x| {
1805 /// *acc += x;
1806 /// if *acc > 100 {
1807 /// hydro_lang::live_collections::keyed_stream::Generate::Return("done!".to_owned())
1808 /// } else if *acc % 2 == 0 {
1809 /// hydro_lang::live_collections::keyed_stream::Generate::Yield("even".to_owned())
1810 /// } else {
1811 /// hydro_lang::live_collections::keyed_stream::Generate::Continue
1812 /// }
1813 /// }),
1814 /// )
1815 /// # }, |mut stream| async move {
1816 /// // Output: "even", "done!"
1817 /// # let mut results = Vec::new();
1818 /// # for _ in 0..2 {
1819 /// # results.push(stream.next().await.unwrap());
1820 /// # }
1821 /// # results.sort();
1822 /// # assert_eq!(results, vec!["done!".to_owned(), "even".to_owned()]);
1823 /// # }));
1824 /// # }
1825 /// ```
1826 pub fn generator<A, U, I, F>(
1827 self,
1828 init: impl IntoQuotedMut<'a, I, L> + Copy,
1829 f: impl IntoQuotedMut<'a, F, L> + Copy,
1830 ) -> Stream<U, L, B, TotalOrder, ExactlyOnce>
1831 where
1832 O: IsOrdered,
1833 R: IsExactlyOnce,
1834 I: Fn() -> A + 'a,
1835 F: Fn(&mut A, T) -> Generate<U> + 'a,
1836 {
1837 let init: ManualExpr<I, _> = ManualExpr::new(move |ctx: &L| init.splice_fn0_ctx(ctx));
1838 let f: ManualExpr<F, _> = ManualExpr::new(move |ctx: &L| f.splice_fn2_borrow_mut_ctx(ctx));
1839
1840 let this = self.make_totally_ordered().make_exactly_once();
1841
1842 // State is Option<Option<A>>:
1843 // None = not yet initialized
1844 // Some(Some(a)) = active with state a
1845 // Some(None) = terminated
1846 let scan_init = q!(|| None)
1847 .splice_fn0_ctx::<Option<Option<A>>>(&this.location)
1848 .into();
1849 let scan_f = q!(move |state: &mut Option<Option<_>>, v| {
1850 if state.is_none() {
1851 *state = Some(Some(init()));
1852 }
1853 match state {
1854 Some(Some(state_value)) => match f(state_value, v) {
1855 Generate::Yield(out) => Some(Some(out)),
1856 Generate::Return(out) => {
1857 *state = Some(None);
1858 Some(Some(out))
1859 }
1860 // Unlike KeyedStream, we can terminate the scan directly on
1861 // Break/Return because there is only one state (no other keys
1862 // that still need processing).
1863 Generate::Break => None,
1864 Generate::Continue => Some(None),
1865 },
1866 // State is Some(None) after Return; terminate the scan.
1867 _ => None,
1868 }
1869 })
1870 .splice_fn2_borrow_mut_ctx::<Option<Option<A>>, T, _>(&this.location)
1871 .into();
1872
1873 let scan_node = HydroNode::Scan {
1874 init: scan_init,
1875 acc: scan_f,
1876 input: Box::new(this.ir_node.replace(HydroNode::Placeholder)),
1877 metadata: this.location.new_node_metadata(Stream::<
1878 Option<U>,
1879 L,
1880 B,
1881 TotalOrder,
1882 ExactlyOnce,
1883 >::collection_kind()),
1884 };
1885
1886 let flatten_f = q!(|d| d)
1887 .splice_fn1_ctx::<Option<U>, _>(&this.location)
1888 .into();
1889 let flatten_node = HydroNode::FlatMap {
1890 f: flatten_f,
1891 input: Box::new(scan_node),
1892 metadata: this
1893 .location
1894 .new_node_metadata(Stream::<U, L, B, TotalOrder, ExactlyOnce>::collection_kind()),
1895 };
1896
1897 Stream::new(this.location.clone(), flatten_node)
1898 }
1899
1900 /// Given a time interval, returns a stream corresponding to samples taken from the
1901 /// stream roughly at that interval. The output will have elements in the same order
1902 /// as the input, but with arbitrary elements skipped between samples. There is also
1903 /// no guarantee on the exact timing of the samples.
1904 ///
1905 /// # Non-Determinism
1906 /// The output stream is non-deterministic in which elements are sampled, since this
1907 /// is controlled by a clock.
1908 pub fn sample_every(
1909 self,
1910 interval: impl QuotedWithContext<'a, std::time::Duration, L> + Copy + 'a,
1911 nondet: NonDet,
1912 ) -> Stream<T, L::DropConsistency, Unbounded, O, AtLeastOnce>
1913 where
1914 L: TopLevel<'a>,
1915 {
1916 let samples = self.location.source_interval(interval);
1917
1918 let tick = self.location.tick();
1919 self.batch(&tick, nondet)
1920 .filter_if(samples.batch(&tick, nondet).first().is_some())
1921 .all_ticks()
1922 .weaken_retries()
1923 }
1924
1925 /// Given a timeout duration, returns an [`Optional`] which will have a value if the
1926 /// stream has not emitted a value since that duration.
1927 ///
1928 /// # Non-Determinism
1929 /// Timeout relies on non-deterministic sampling of the stream, so depending on when
1930 /// samples take place, timeouts may be non-deterministically generated or missed,
1931 /// and the notification of the timeout may be delayed as well. There is also no
1932 /// guarantee on how long the [`Optional`] will have a value after the timeout is
1933 /// detected based on when the next sample is taken.
1934 pub fn timeout(
1935 self,
1936 duration: impl QuotedWithContext<'a, std::time::Duration, Tick<L::DropConsistency>> + Copy + 'a,
1937 nondet: NonDet,
1938 ) -> Optional<(), L::DropConsistency, Unbounded>
1939 where
1940 L: TopLevel<'a>,
1941 {
1942 let tick = self.location.tick();
1943
1944 let latest_received = self.assume_retries::<ExactlyOnce>(nondet).fold(
1945 q!(|| None),
1946 q!(
1947 |latest, _| {
1948 *latest = Some(Instant::now());
1949 },
1950 commutative = manual_proof!(/** TODO */)
1951 ),
1952 );
1953
1954 latest_received
1955 .snapshot(&tick, nondet)
1956 .filter_map(q!(move |latest_received| {
1957 if let Some(latest_received) = latest_received {
1958 if Instant::now().duration_since(latest_received) > duration {
1959 Some(())
1960 } else {
1961 None
1962 }
1963 } else {
1964 Some(())
1965 }
1966 }))
1967 .latest()
1968 }
1969
1970 /// Shifts this stream into an atomic context, which guarantees that any downstream logic
1971 /// will all be executed synchronously before any outputs are yielded (in [`Stream::end_atomic`]).
1972 ///
1973 /// This is useful to enforce local consistency constraints, such as ensuring that a write is
1974 /// processed before an acknowledgement is emitted.
1975 pub fn atomic(self) -> Stream<T, Atomic<L>, B, O, R> {
1976 let id = self.location.flow_state().borrow_mut().next_clock_id();
1977 let out_location = Atomic {
1978 tick: Tick {
1979 id,
1980 l: self.location.clone(),
1981 },
1982 };
1983 Stream::new(
1984 out_location.clone(),
1985 HydroNode::BeginAtomic {
1986 inner: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
1987 metadata: out_location
1988 .new_node_metadata(Stream::<T, Atomic<L>, B, O, R>::collection_kind()),
1989 },
1990 )
1991 }
1992
1993 /// Given a tick, returns a stream corresponding to a batch of elements segmented by
1994 /// that tick. These batches are guaranteed to be contiguous across ticks and preserve
1995 /// the order of the input. The output stream will execute in the [`Tick`] that was
1996 /// used to create the atomic section.
1997 ///
1998 /// # Non-Determinism
1999 /// The batch boundaries are non-deterministic and may change across executions.
2000 pub fn batch<L2: Location<'a, DropConsistency = L::DropConsistency>>(
2001 self,
2002 tick: &Tick<L2>,
2003 _nondet: NonDet,
2004 ) -> Stream<T, Tick<L::DropConsistency>, Bounded, O, R> {
2005 assert_eq!(Location::id(tick.outer()), Location::id(&self.location));
2006 Stream::new(
2007 tick.drop_consistency(),
2008 HydroNode::Batch {
2009 inner: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
2010 metadata: tick
2011 .new_node_metadata(Stream::<T, Tick<L>, Bounded, O, R>::collection_kind()),
2012 },
2013 )
2014 }
2015
2016 /// An operator which allows you to "name" a `HydroNode`.
2017 /// This is only used for testing, to correlate certain `HydroNode`s with IDs.
2018 pub fn ir_node_named(self, name: &str) -> Stream<T, L, B, O, R> {
2019 {
2020 let mut node = self.ir_node.borrow_mut();
2021 let metadata = node.metadata_mut();
2022 metadata.tag = Some(name.to_owned());
2023 }
2024 self
2025 }
2026
2027 /// Turns this [`Stream`] into a [`Optional`], under the invariant assumption that there is at
2028 /// most one element. If this invariant is broken, the program may exhibit undefined behavior,
2029 /// so uses must be carefully vetted.
2030 pub(crate) fn cast_at_most_one_element(self) -> Optional<T, L, B>
2031 where
2032 B: IsBounded,
2033 {
2034 Optional::new(
2035 self.location.clone(),
2036 HydroNode::Cast {
2037 inner: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
2038 metadata: self
2039 .location
2040 .new_node_metadata(Optional::<T, L, B>::collection_kind()),
2041 },
2042 )
2043 }
2044
2045 pub(crate) fn use_ordering_type<O2: Ordering>(self) -> Stream<T, L, B, O2, R> {
2046 if O::ORDERING_KIND == O2::ORDERING_KIND {
2047 Stream::new(
2048 self.location.clone(),
2049 self.ir_node.replace(HydroNode::Placeholder),
2050 )
2051 } else {
2052 panic!(
2053 "Runtime ordering {:?} did not match requested cast {:?}.",
2054 O::ORDERING_KIND,
2055 O2::ORDERING_KIND
2056 )
2057 }
2058 }
2059
2060 /// Explicitly "casts" the stream to a type with a different ordering
2061 /// guarantee. Useful in unsafe code where the ordering cannot be proven
2062 /// by the type-system.
2063 ///
2064 /// # Non-Determinism
2065 /// This function is used as an escape hatch, and any mistakes in the
2066 /// provided ordering guarantee will propagate into the guarantees
2067 /// for the rest of the program.
2068 pub fn assume_ordering<O2: Ordering>(
2069 self,
2070 _nondet: NonDet,
2071 ) -> Stream<T, L::DropConsistency, B, O2, R> {
2072 if O::ORDERING_KIND == O2::ORDERING_KIND {
2073 self.use_ordering_type().weaken_consistency()
2074 } else if O2::ORDERING_KIND == StreamOrder::NoOrder {
2075 // We can always weaken the ordering guarantee
2076 let target_location = self.location().drop_consistency();
2077 Stream::new(
2078 target_location.clone(),
2079 HydroNode::Cast {
2080 inner: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
2081 metadata: target_location
2082 .new_node_metadata(Stream::<T, L, B, O2, R>::collection_kind()),
2083 },
2084 )
2085 } else {
2086 let target_location = self.location().drop_consistency();
2087 Stream::new(
2088 target_location.clone(),
2089 HydroNode::ObserveNonDet {
2090 inner: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
2091 trusted: false,
2092 metadata: target_location
2093 .new_node_metadata(Stream::<T, L, B, O2, R>::collection_kind()),
2094 },
2095 )
2096 }
2097 }
2098
2099 // like `assume_ordering_trusted`, but only if the input stream is bounded and therefore
2100 // intermediate states will not be revealed
2101 fn assume_ordering_trusted_bounded<O2: Ordering>(
2102 self,
2103 nondet: NonDet,
2104 ) -> Stream<T, L, B, O2, R> {
2105 if B::BOUNDED {
2106 self.assume_ordering_trusted(nondet)
2107 } else {
2108 let self_location = self.location.clone();
2109 let inner: Stream<T, L::DropConsistency, B, O2, R> = self.assume_ordering(nondet);
2110 Stream::new(self_location, inner.ir_node.replace(HydroNode::Placeholder))
2111 }
2112 }
2113
2114 // only for internal APIs that have been carefully vetted to ensure that the non-determinism
2115 // is not observable
2116 pub(crate) fn assume_ordering_trusted<O2: Ordering>(
2117 self,
2118 _nondet: NonDet,
2119 ) -> Stream<T, L, B, O2, R> {
2120 if O::ORDERING_KIND == O2::ORDERING_KIND {
2121 self.use_ordering_type()
2122 } else if O2::ORDERING_KIND == StreamOrder::NoOrder {
2123 // We can always weaken the ordering guarantee
2124 Stream::new(
2125 self.location.clone(),
2126 HydroNode::Cast {
2127 inner: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
2128 metadata: self
2129 .location
2130 .new_node_metadata(Stream::<T, L, B, O2, R>::collection_kind()),
2131 },
2132 )
2133 } else {
2134 Stream::new(
2135 self.location.clone(),
2136 HydroNode::ObserveNonDet {
2137 inner: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
2138 trusted: true,
2139 metadata: self
2140 .location
2141 .new_node_metadata(Stream::<T, L, B, O2, R>::collection_kind()),
2142 },
2143 )
2144 }
2145 }
2146
2147 #[deprecated = "use `weaken_ordering::<NoOrder>()` instead"]
2148 /// Weakens the ordering guarantee provided by the stream to [`NoOrder`],
2149 /// which is always safe because that is the weakest possible guarantee.
2150 pub fn weakest_ordering(self) -> Stream<T, L, B, NoOrder, R> {
2151 self.weaken_ordering::<NoOrder>()
2152 }
2153
2154 /// Weakens the ordering guarantee provided by the stream to `O2`, with the type-system
2155 /// enforcing that `O2` is weaker than the input ordering guarantee.
2156 pub fn weaken_ordering<O2: WeakerOrderingThan<O>>(self) -> Stream<T, L, B, O2, R> {
2157 let nondet = nondet!(/** this is a weaker ordering guarantee, so it is safe to assume */);
2158 self.assume_ordering_trusted::<O2>(nondet)
2159 }
2160
2161 /// Strengthens the ordering guarantee to `TotalOrder`, given that `O: IsOrdered`, which
2162 /// implies that `O == TotalOrder`.
2163 pub fn make_totally_ordered(self) -> Stream<T, L, B, TotalOrder, R>
2164 where
2165 O: IsOrdered,
2166 {
2167 self.assume_ordering_trusted(nondet!(/** no-op */))
2168 }
2169
2170 /// Explicitly "casts" the stream to a type with a different retries
2171 /// guarantee. Useful in unsafe code where the lack of retries cannot
2172 /// be proven by the type-system.
2173 ///
2174 /// # Non-Determinism
2175 /// This function is used as an escape hatch, and any mistakes in the
2176 /// provided retries guarantee will propagate into the guarantees
2177 /// for the rest of the program.
2178 pub fn assume_retries<R2: Retries>(
2179 self,
2180 _nondet: NonDet,
2181 ) -> Stream<T, L::DropConsistency, B, O, R2> {
2182 if R::RETRIES_KIND == R2::RETRIES_KIND {
2183 Stream::new(
2184 self.location.drop_consistency(),
2185 self.ir_node.replace(HydroNode::Placeholder),
2186 )
2187 } else if R2::RETRIES_KIND == StreamRetry::AtLeastOnce {
2188 // We can always weaken the retries guarantee
2189 let target_location = self.location.drop_consistency();
2190 Stream::new(
2191 target_location.clone(),
2192 HydroNode::Cast {
2193 inner: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
2194 metadata: target_location
2195 .new_node_metadata(Stream::<T, L, B, O, R2>::collection_kind()),
2196 },
2197 )
2198 } else {
2199 let target_location = self.location.drop_consistency();
2200 Stream::new(
2201 target_location.clone(),
2202 HydroNode::ObserveNonDet {
2203 inner: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
2204 trusted: false,
2205 metadata: target_location
2206 .new_node_metadata(Stream::<T, L, B, O, R2>::collection_kind()),
2207 },
2208 )
2209 }
2210 }
2211
2212 // only for internal APIs that have been carefully vetted to ensure that the non-determinism
2213 // is not observable
2214 fn assume_retries_trusted<R2: Retries>(self, _nondet: NonDet) -> Stream<T, L, B, O, R2> {
2215 if R::RETRIES_KIND == R2::RETRIES_KIND {
2216 Stream::new(
2217 self.location.clone(),
2218 self.ir_node.replace(HydroNode::Placeholder),
2219 )
2220 } else if R2::RETRIES_KIND == StreamRetry::AtLeastOnce {
2221 // We can always weaken the retries guarantee
2222 Stream::new(
2223 self.location.clone(),
2224 HydroNode::Cast {
2225 inner: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
2226 metadata: self
2227 .location
2228 .new_node_metadata(Stream::<T, L, B, O, R2>::collection_kind()),
2229 },
2230 )
2231 } else {
2232 Stream::new(
2233 self.location.clone(),
2234 HydroNode::ObserveNonDet {
2235 inner: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
2236 trusted: true,
2237 metadata: self
2238 .location
2239 .new_node_metadata(Stream::<T, L, B, O, R2>::collection_kind()),
2240 },
2241 )
2242 }
2243 }
2244
2245 #[deprecated = "use `weaken_retries::<AtLeastOnce>()` instead"]
2246 /// Weakens the retries guarantee provided by the stream to [`AtLeastOnce`],
2247 /// which is always safe because that is the weakest possible guarantee.
2248 pub fn weakest_retries(self) -> Stream<T, L, B, O, AtLeastOnce> {
2249 self.weaken_retries::<AtLeastOnce>()
2250 }
2251
2252 /// Weakens the retries guarantee provided by the stream to `R2`, with the type-system
2253 /// enforcing that `R2` is weaker than the input retries guarantee.
2254 pub fn weaken_retries<R2: WeakerRetryThan<R>>(self) -> Stream<T, L, B, O, R2> {
2255 let nondet = nondet!(/** this is a weaker retry guarantee, so it is safe to assume */);
2256 self.assume_retries_trusted::<R2>(nondet)
2257 }
2258
2259 /// Strengthens the retry guarantee to `ExactlyOnce`, given that `R: IsExactlyOnce`, which
2260 /// implies that `R == ExactlyOnce`.
2261 pub fn make_exactly_once(self) -> Stream<T, L, B, O, ExactlyOnce>
2262 where
2263 R: IsExactlyOnce,
2264 {
2265 self.assume_retries_trusted(nondet!(/** no-op */))
2266 }
2267
2268 /// Strengthens the boundedness guarantee to `Bounded`, given that `B: IsBounded`, which
2269 /// implies that `B == Bounded`.
2270 pub fn make_bounded(self) -> Stream<T, L, Bounded, O, R>
2271 where
2272 B: IsBounded,
2273 {
2274 self.weaken_boundedness()
2275 }
2276
2277 /// Weakens the boundedness guarantee to an arbitrary boundedness `B2`, given that `B: IsBounded`,
2278 /// which implies that `B == Bounded`.
2279 pub fn weaken_boundedness<B2: Boundedness>(self) -> Stream<T, L, B2, O, R> {
2280 if B::BOUNDED == B2::BOUNDED {
2281 Stream::new(
2282 self.location.clone(),
2283 self.ir_node.replace(HydroNode::Placeholder),
2284 )
2285 } else {
2286 // We can always weaken the boundedness
2287 Stream::new(
2288 self.location.clone(),
2289 HydroNode::Cast {
2290 inner: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
2291 metadata: self
2292 .location
2293 .new_node_metadata(Stream::<T, L, B2, O, R>::collection_kind()),
2294 },
2295 )
2296 }
2297 }
2298}
2299
2300impl<'a, T, L, B: Boundedness, O: Ordering, R: Retries> Stream<&T, L, B, O, R>
2301where
2302 L: Location<'a>,
2303{
2304 /// Clone each element of the stream; akin to `map(q!(|d| d.clone()))`.
2305 ///
2306 /// # Example
2307 /// ```rust
2308 /// # #[cfg(feature = "deploy")] {
2309 /// # use hydro_lang::prelude::*;
2310 /// # use futures::StreamExt;
2311 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
2312 /// process.source_iter(q!(&[1, 2, 3])).cloned()
2313 /// # }, |mut stream| async move {
2314 /// // 1, 2, 3
2315 /// # for w in vec![1, 2, 3] {
2316 /// # assert_eq!(stream.next().await.unwrap(), w);
2317 /// # }
2318 /// # }));
2319 /// # }
2320 /// ```
2321 pub fn cloned(self) -> Stream<T, L, B, O, R>
2322 where
2323 T: Clone,
2324 {
2325 self.map(q!(|d| d.clone()))
2326 }
2327}
2328
2329impl<'a, T, L, B: Boundedness, O: Ordering> Stream<T, L, B, O, ExactlyOnce>
2330where
2331 L: Location<'a>,
2332{
2333 /// Computes the number of elements in the stream as a [`Singleton`].
2334 ///
2335 /// # Example
2336 /// ```rust
2337 /// # #[cfg(feature = "deploy")] {
2338 /// # use hydro_lang::prelude::*;
2339 /// # use futures::StreamExt;
2340 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
2341 /// let tick = process.tick();
2342 /// let numbers = process.source_iter(q!(vec![1, 2, 3, 4]));
2343 /// let batch = numbers.batch(&tick, nondet!(/** test */));
2344 /// batch.count().all_ticks()
2345 /// # }, |mut stream| async move {
2346 /// // 4
2347 /// # assert_eq!(stream.next().await.unwrap(), 4);
2348 /// # }));
2349 /// # }
2350 /// ```
2351 pub fn count(self) -> Singleton<usize, L, B::StreamToMonotone> {
2352 self.assume_ordering_trusted::<TotalOrder>(nondet!(
2353 /// Order does not affect eventual count, and also does not affect intermediate states.
2354 ))
2355 .fold(
2356 q!(|| 0usize),
2357 q!(
2358 |count, _| *count += 1,
2359 monotone = manual_proof!(/** += 1 is monotone */)
2360 ),
2361 )
2362 }
2363}
2364
2365impl<'a, T, L: Location<'a>, O: Ordering, R: Retries> Stream<T, L, Unbounded, O, R> {
2366 /// Produces a new stream that merges the elements of the two input streams.
2367 /// The result has [`NoOrder`] because the order of merging is not guaranteed.
2368 ///
2369 /// Currently, both input streams must be [`Unbounded`]. When the streams are
2370 /// [`Bounded`], you can use [`Stream::chain`] instead.
2371 ///
2372 /// # Example
2373 /// ```rust
2374 /// # #[cfg(feature = "deploy")] {
2375 /// # use hydro_lang::prelude::*;
2376 /// # use futures::StreamExt;
2377 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
2378 /// let numbers: Stream<i32, _, Unbounded> = // 1, 2, 3, 4
2379 /// # process.source_iter(q!(vec![1, 2, 3, 4])).into();
2380 /// numbers.clone().map(q!(|x| x + 1)).merge_unordered(numbers)
2381 /// # }, |mut stream| async move {
2382 /// // 2, 3, 4, 5, and 1, 2, 3, 4 merged in unknown order
2383 /// # for w in vec![2, 3, 4, 5, 1, 2, 3, 4] {
2384 /// # assert_eq!(stream.next().await.unwrap(), w);
2385 /// # }
2386 /// # }));
2387 /// # }
2388 /// ```
2389 pub fn merge_unordered<O2: Ordering, R2: Retries>(
2390 self,
2391 other: Stream<T, L, Unbounded, O2, R2>,
2392 ) -> Stream<T, L, Unbounded, NoOrder, <R as MinRetries<R2>>::Min>
2393 where
2394 R: MinRetries<R2>,
2395 {
2396 Stream::new(
2397 self.location.clone(),
2398 HydroNode::Chain {
2399 first: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
2400 second: Box::new(other.ir_node.replace(HydroNode::Placeholder)),
2401 metadata: self.location.new_node_metadata(Stream::<
2402 T,
2403 L,
2404 Unbounded,
2405 NoOrder,
2406 <R as MinRetries<R2>>::Min,
2407 >::collection_kind()),
2408 },
2409 )
2410 }
2411
2412 /// Deprecated: use [`Stream::merge_unordered`] instead.
2413 #[deprecated(note = "use `merge_unordered` instead")]
2414 pub fn interleave<O2: Ordering, R2: Retries>(
2415 self,
2416 other: Stream<T, L, Unbounded, O2, R2>,
2417 ) -> Stream<T, L, Unbounded, NoOrder, <R as MinRetries<R2>>::Min>
2418 where
2419 R: MinRetries<R2>,
2420 {
2421 self.merge_unordered(other)
2422 }
2423}
2424
2425impl<'a, T, L: Location<'a>, B: Boundedness, R: Retries> Stream<T, L, B, TotalOrder, R> {
2426 /// Produces a new stream that combines the elements of the two input streams,
2427 /// preserving the relative order of elements within each input.
2428 ///
2429 /// # Non-Determinism
2430 /// The order in which elements *across* the two streams will be interleaved is
2431 /// non-deterministic, so the order of elements will vary across runs. If the output
2432 /// order is irrelevant, use [`Stream::merge_unordered`] instead, which is deterministic
2433 /// but emits an unordered stream. For deterministic first-then-second ordering on
2434 /// bounded streams, use [`Stream::chain`].
2435 ///
2436 /// # Example
2437 /// ```rust
2438 /// # #[cfg(feature = "deploy")] {
2439 /// # use hydro_lang::prelude::*;
2440 /// # use futures::StreamExt;
2441 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
2442 /// let numbers: Stream<i32, _, Unbounded> = // 1, 3
2443 /// # process.source_iter(q!(vec![1, 3])).into();
2444 /// numbers.clone().merge_ordered(numbers.map(q!(|x| x + 1)), nondet!(/** example */))
2445 /// # }, |mut stream| async move {
2446 /// // 1, 3 and 2, 4 in some order, preserving the original local order
2447 /// # for w in vec![1, 3, 2, 4] {
2448 /// # assert_eq!(stream.next().await.unwrap(), w);
2449 /// # }
2450 /// # }));
2451 /// # }
2452 /// ```
2453 pub fn merge_ordered<R2: Retries>(
2454 self,
2455 other: Stream<T, L, B, TotalOrder, R2>,
2456 _nondet: NonDet,
2457 ) -> Stream<T, L::DropConsistency, B, TotalOrder, <R as MinRetries<R2>>::Min>
2458 where
2459 R: MinRetries<R2>,
2460 {
2461 let target_location = self.location().drop_consistency();
2462 Stream::new(
2463 target_location.clone(),
2464 HydroNode::MergeOrdered {
2465 first: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
2466 second: Box::new(other.ir_node.replace(HydroNode::Placeholder)),
2467 metadata: target_location.new_node_metadata(Stream::<
2468 T,
2469 L::DropConsistency,
2470 B,
2471 TotalOrder,
2472 <R as MinRetries<R2>>::Min,
2473 >::collection_kind()),
2474 },
2475 )
2476 }
2477}
2478
2479impl<'a, T, L, B: Boundedness, O: Ordering, R: Retries> Stream<T, L, B, O, R>
2480where
2481 L: Location<'a>,
2482{
2483 /// Produces a new stream that emits the input elements in sorted order.
2484 ///
2485 /// The input stream can have any ordering guarantee, but the output stream
2486 /// will have a [`TotalOrder`] guarantee. This operator will block until all
2487 /// elements in the input stream are available, so it requires the input stream
2488 /// to be [`Bounded`].
2489 ///
2490 /// # Example
2491 /// ```rust
2492 /// # #[cfg(feature = "deploy")] {
2493 /// # use hydro_lang::prelude::*;
2494 /// # use futures::StreamExt;
2495 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
2496 /// let tick = process.tick();
2497 /// let numbers = process.source_iter(q!(vec![4, 2, 3, 1]));
2498 /// let batch = numbers.batch(&tick, nondet!(/** test */));
2499 /// batch.sort().all_ticks()
2500 /// # }, |mut stream| async move {
2501 /// // 1, 2, 3, 4
2502 /// # for w in (1..5) {
2503 /// # assert_eq!(stream.next().await.unwrap(), w);
2504 /// # }
2505 /// # }));
2506 /// # }
2507 /// ```
2508 pub fn sort(self) -> Stream<T, L, Bounded, TotalOrder, R>
2509 where
2510 B: IsBounded,
2511 T: Ord,
2512 {
2513 let this = self.make_bounded();
2514 Stream::new(
2515 this.location.clone(),
2516 HydroNode::Sort {
2517 input: Box::new(this.ir_node.replace(HydroNode::Placeholder)),
2518 metadata: this
2519 .location
2520 .new_node_metadata(Stream::<T, L, Bounded, TotalOrder, R>::collection_kind()),
2521 },
2522 )
2523 }
2524
2525 /// Produces a new stream that first emits the elements of the `self` stream,
2526 /// and then emits the elements of the `other` stream. The output stream has
2527 /// a [`TotalOrder`] guarantee if and only if both input streams have a
2528 /// [`TotalOrder`] guarantee.
2529 ///
2530 /// Currently, both input streams must be [`Bounded`]. This operator will block
2531 /// on the first stream until all its elements are available. In a future version,
2532 /// we will relax the requirement on the `other` stream.
2533 ///
2534 /// # Example
2535 /// ```rust
2536 /// # #[cfg(feature = "deploy")] {
2537 /// # use hydro_lang::prelude::*;
2538 /// # use futures::StreamExt;
2539 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
2540 /// let tick = process.tick();
2541 /// let numbers = process.source_iter(q!(vec![1, 2, 3, 4]));
2542 /// let batch = numbers.batch(&tick, nondet!(/** test */));
2543 /// batch.clone().map(q!(|x| x + 1)).chain(batch).all_ticks()
2544 /// # }, |mut stream| async move {
2545 /// // 2, 3, 4, 5, 1, 2, 3, 4
2546 /// # for w in vec![2, 3, 4, 5, 1, 2, 3, 4] {
2547 /// # assert_eq!(stream.next().await.unwrap(), w);
2548 /// # }
2549 /// # }));
2550 /// # }
2551 /// ```
2552 pub fn chain<O2: Ordering, R2: Retries, B2: Boundedness>(
2553 self,
2554 other: Stream<T, L, B2, O2, R2>,
2555 ) -> Stream<T, L, B2, <O as MinOrder<O2>>::Min, <R as MinRetries<R2>>::Min>
2556 where
2557 B: IsBounded,
2558 O: MinOrder<O2>,
2559 R: MinRetries<R2>,
2560 {
2561 check_matching_location(&self.location, &other.location);
2562
2563 Stream::new(
2564 self.location.clone(),
2565 HydroNode::Chain {
2566 first: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
2567 second: Box::new(other.ir_node.replace(HydroNode::Placeholder)),
2568 metadata: self.location.new_node_metadata(Stream::<
2569 T,
2570 L,
2571 B2,
2572 <O as MinOrder<O2>>::Min,
2573 <R as MinRetries<R2>>::Min,
2574 >::collection_kind()),
2575 },
2576 )
2577 }
2578
2579 /// Forms the cross-product (Cartesian product, cross-join) of the items in the 2 input streams.
2580 /// Unlike [`Stream::cross_product`], the output order is totally ordered when the inputs are
2581 /// because this is compiled into a nested loop.
2582 #[expect(
2583 clippy::type_complexity,
2584 reason = "MinRetries projection in return type"
2585 )]
2586 pub fn cross_product_nested_loop<T2, O2: Ordering + MinOrder<O>, R2: Retries>(
2587 self,
2588 other: Stream<T2, L, Bounded, O2, R2>,
2589 ) -> Stream<(T, T2), L, Bounded, <O2 as MinOrder<O>>::Min, <R as MinRetries<R2>>::Min>
2590 where
2591 B: IsBounded,
2592 T: Clone,
2593 T2: Clone,
2594 R: MinRetries<R2>,
2595 {
2596 let this = self.make_bounded();
2597 check_matching_location(&this.location, &other.location);
2598
2599 Stream::new(
2600 this.location.clone(),
2601 HydroNode::CrossProduct {
2602 left: Box::new(this.ir_node.replace(HydroNode::Placeholder)),
2603 right: Box::new(other.ir_node.replace(HydroNode::Placeholder)),
2604 metadata: this.location.new_node_metadata(Stream::<
2605 (T, T2),
2606 L,
2607 Bounded,
2608 <O2 as MinOrder<O>>::Min,
2609 <R as MinRetries<R2>>::Min,
2610 >::collection_kind()),
2611 },
2612 )
2613 }
2614
2615 /// Creates a [`KeyedStream`] with the same set of keys as `keys`, but with the elements in
2616 /// `self` used as the values for *each* key.
2617 ///
2618 /// This is helpful when "broadcasting" a set of values so that all the keys have the same
2619 /// values. For example, it can be used to send the same set of elements to several cluster
2620 /// members, if the membership information is available as a [`KeyedSingleton`].
2621 ///
2622 /// # Example
2623 /// ```rust
2624 /// # #[cfg(feature = "deploy")] {
2625 /// # use hydro_lang::prelude::*;
2626 /// # use futures::StreamExt;
2627 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
2628 /// # let tick = process.tick();
2629 /// let keyed_singleton = // { 1: (), 2: () }
2630 /// # process
2631 /// # .source_iter(q!(vec![(1, ()), (2, ())]))
2632 /// # .into_keyed()
2633 /// # .batch(&tick, nondet!(/** test */))
2634 /// # .first();
2635 /// let stream = // [ "a", "b" ]
2636 /// # process
2637 /// # .source_iter(q!(vec!["a".to_owned(), "b".to_owned()]))
2638 /// # .batch(&tick, nondet!(/** test */));
2639 /// stream.repeat_with_keys(keyed_singleton)
2640 /// # .entries().all_ticks()
2641 /// # }, |mut stream| async move {
2642 /// // { 1: ["a", "b" ], 2: ["a", "b"] }
2643 /// # let mut results = Vec::new();
2644 /// # for _ in 0..4 {
2645 /// # results.push(stream.next().await.unwrap());
2646 /// # }
2647 /// # results.sort();
2648 /// # assert_eq!(results, vec![(1, "a".to_owned()), (1, "b".to_owned()), (2, "a".to_owned()), (2, "b".to_owned())]);
2649 /// # }));
2650 /// # }
2651 /// ```
2652 pub fn repeat_with_keys<K, V2>(
2653 self,
2654 keys: KeyedSingleton<K, V2, L, Bounded>,
2655 ) -> KeyedStream<K, T, L, Bounded, O, R>
2656 where
2657 B: IsBounded,
2658 K: Clone,
2659 T: Clone,
2660 {
2661 keys.keys()
2662 .assume_ordering_trusted::<TotalOrder>(
2663 nondet!(/** keyed stream does not depend on ordering of keys */),
2664 )
2665 .cross_product_nested_loop(self.make_bounded())
2666 .into_keyed()
2667 }
2668
2669 /// Consumes a stream of `Future<T>`, resolving each future while blocking subgraph
2670 /// execution until all results are available. The output order is based on when futures
2671 /// complete, and may be different than the input order.
2672 ///
2673 /// Unlike [`Stream::resolve_futures`], which allows the subgraph to continue executing
2674 /// while futures are pending, this variant blocks until the futures resolve.
2675 ///
2676 /// # Example
2677 /// ```rust
2678 /// # #[cfg(feature = "deploy")] {
2679 /// # use std::collections::HashSet;
2680 /// # use futures::StreamExt;
2681 /// # use hydro_lang::prelude::*;
2682 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
2683 /// process
2684 /// .source_iter(q!([2, 3, 1, 9, 6, 5, 4, 7, 8]))
2685 /// .map(q!(|x| async move {
2686 /// tokio::time::sleep(tokio::time::Duration::from_millis(10)).await;
2687 /// x
2688 /// }))
2689 /// .resolve_futures_blocking()
2690 /// # },
2691 /// # |mut stream| async move {
2692 /// // 1, 2, 3, 4, 5, 6, 7, 8, 9 (in any order)
2693 /// # let mut output = HashSet::new();
2694 /// # for _ in 1..10 {
2695 /// # output.insert(stream.next().await.unwrap());
2696 /// # }
2697 /// # assert_eq!(
2698 /// # output,
2699 /// # HashSet::<i32>::from_iter(1..10)
2700 /// # );
2701 /// # },
2702 /// # ));
2703 /// # }
2704 /// ```
2705 pub fn resolve_futures_blocking(self) -> Stream<T::Output, L, B, NoOrder, R>
2706 where
2707 T: Future,
2708 {
2709 Stream::new(
2710 self.location.clone(),
2711 HydroNode::ResolveFuturesBlocking {
2712 input: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
2713 metadata: self
2714 .location
2715 .new_node_metadata(Stream::<T::Output, L, B, NoOrder, R>::collection_kind()),
2716 },
2717 )
2718 }
2719
2720 /// Returns a [`Singleton`] containing `true` if the stream has no elements, or `false` otherwise.
2721 ///
2722 /// # Example
2723 /// ```rust
2724 /// # #[cfg(feature = "deploy")] {
2725 /// # use hydro_lang::prelude::*;
2726 /// # use futures::StreamExt;
2727 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
2728 /// let tick = process.tick();
2729 /// let empty: Stream<i32, _, Bounded> = process
2730 /// .source_iter(q!(Vec::<i32>::new()))
2731 /// .batch(&tick, nondet!(/** test */));
2732 /// empty.is_empty().all_ticks()
2733 /// # }, |mut stream| async move {
2734 /// // true
2735 /// # assert_eq!(stream.next().await.unwrap(), true);
2736 /// # }));
2737 /// # }
2738 /// ```
2739 #[expect(clippy::wrong_self_convention, reason = "stream function naming")]
2740 pub fn is_empty(self) -> Singleton<bool, L, Bounded>
2741 where
2742 B: IsBounded,
2743 {
2744 self.make_bounded()
2745 .assume_ordering_trusted::<TotalOrder>(
2746 nondet!(/** is_empty intermediates unaffected by order */),
2747 )
2748 .first()
2749 .is_none()
2750 }
2751}
2752
2753impl<'a, K, V1, L, B: Boundedness, O: Ordering, R: Retries> Stream<(K, V1), L, B, O, R>
2754where
2755 L: Location<'a>,
2756{
2757 #[expect(clippy::type_complexity, reason = "ordering / retries propagation")]
2758 /// Given two streams of pairs `(K, V1)` and `(K, V2)`, produces a new stream of nested pairs `(K, (V1, V2))`
2759 /// by equi-joining the two streams on the key attribute `K`.
2760 ///
2761 /// When the right-hand side is [`Bounded`], the join accumulates the right side first
2762 /// and streams the left side through, preserving the left side's ordering. When both
2763 /// sides are [`Unbounded`], a symmetric hash join is used and ordering is [`NoOrder`].
2764 ///
2765 /// # Example
2766 /// ```rust
2767 /// # #[cfg(feature = "deploy")] {
2768 /// # use hydro_lang::prelude::*;
2769 /// # use std::collections::HashSet;
2770 /// # use futures::StreamExt;
2771 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
2772 /// let tick = process.tick();
2773 /// let stream1 = process.source_iter(q!(vec![(1, 'a'), (2, 'b')]));
2774 /// let stream2 = process.source_iter(q!(vec![(1, 'x'), (2, 'y')]));
2775 /// stream1.join(stream2)
2776 /// # }, |mut stream| async move {
2777 /// // (1, ('a', 'x')), (2, ('b', 'y'))
2778 /// # let expected = HashSet::from([(1, ('a', 'x')), (2, ('b', 'y'))]);
2779 /// # stream.map(|i| assert!(expected.contains(&i)));
2780 /// # }));
2781 /// # }
2782 pub fn join<V2, B2: Boundedness, O2: Ordering, R2: Retries>(
2783 self,
2784 n: Stream<(K, V2), L, B2, O2, R2>,
2785 ) -> Stream<(K, (V1, V2)), L, B, B2::PreserveOrderIfBounded<O>, <R as MinRetries<R2>>::Min>
2786 where
2787 K: Eq + Hash + Clone,
2788 R: MinRetries<R2>,
2789 V1: Clone,
2790 V2: Clone,
2791 {
2792 check_matching_location(&self.location, &n.location);
2793
2794 let ir_node = if B2::BOUNDED {
2795 HydroNode::JoinHalf {
2796 left: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
2797 right: Box::new(n.ir_node.replace(HydroNode::Placeholder)),
2798 metadata: self.location.new_node_metadata(Stream::<
2799 (K, (V1, V2)),
2800 L,
2801 B,
2802 B2::PreserveOrderIfBounded<O>,
2803 <R as MinRetries<R2>>::Min,
2804 >::collection_kind()),
2805 }
2806 } else {
2807 HydroNode::Join {
2808 left: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
2809 right: Box::new(n.ir_node.replace(HydroNode::Placeholder)),
2810 metadata: self.location.new_node_metadata(Stream::<
2811 (K, (V1, V2)),
2812 L,
2813 B,
2814 B2::PreserveOrderIfBounded<O>,
2815 <R as MinRetries<R2>>::Min,
2816 >::collection_kind()),
2817 }
2818 };
2819
2820 Stream::new(self.location.clone(), ir_node)
2821 }
2822
2823 /// Given a stream of pairs `(K, V1)` and a bounded stream of keys `K`,
2824 /// computes the anti-join of the items in the input -- i.e. returns
2825 /// unique items in the first input that do not have a matching key
2826 /// in the second input.
2827 ///
2828 /// # Example
2829 /// ```rust
2830 /// # #[cfg(feature = "deploy")] {
2831 /// # use hydro_lang::prelude::*;
2832 /// # use futures::StreamExt;
2833 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
2834 /// let tick = process.tick();
2835 /// let stream = process
2836 /// .source_iter(q!(vec![ (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd') ]))
2837 /// .batch(&tick, nondet!(/** test */));
2838 /// let batch = process
2839 /// .source_iter(q!(vec![1, 2]))
2840 /// .batch(&tick, nondet!(/** test */));
2841 /// stream.anti_join(batch).all_ticks()
2842 /// # }, |mut stream| async move {
2843 /// # for w in vec![(3, 'c'), (4, 'd')] {
2844 /// # assert_eq!(stream.next().await.unwrap(), w);
2845 /// # }
2846 /// # }));
2847 /// # }
2848 pub fn anti_join<O2: Ordering, R2: Retries>(
2849 self,
2850 n: Stream<K, L, Bounded, O2, R2>,
2851 ) -> Stream<(K, V1), L, B, O, R>
2852 where
2853 K: Eq + Hash,
2854 {
2855 check_matching_location(&self.location, &n.location);
2856
2857 Stream::new(
2858 self.location.clone(),
2859 HydroNode::AntiJoin {
2860 pos: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
2861 neg: Box::new(n.ir_node.replace(HydroNode::Placeholder)),
2862 metadata: self
2863 .location
2864 .new_node_metadata(Stream::<(K, V1), L, B, O, R>::collection_kind()),
2865 },
2866 )
2867 }
2868}
2869
2870impl<'a, K, V, L: Location<'a>, B: Boundedness, O: Ordering, R: Retries>
2871 Stream<(K, V), L, B, O, R>
2872{
2873 /// Transforms this stream into a [`KeyedStream`], where the first element of each tuple
2874 /// is used as the key and the second element is added to the entries associated with that key.
2875 ///
2876 /// Because [`KeyedStream`] lazily groups values into buckets, this operator has zero computational
2877 /// cost and _does not_ require that the key type is hashable. Keyed streams are useful for
2878 /// performing grouped aggregations, but also for more precise ordering guarantees such as
2879 /// total ordering _within_ each group but no ordering _across_ groups.
2880 ///
2881 /// # Example
2882 /// ```rust
2883 /// # #[cfg(feature = "deploy")] {
2884 /// # use hydro_lang::prelude::*;
2885 /// # use futures::StreamExt;
2886 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
2887 /// process
2888 /// .source_iter(q!(vec![(1, 2), (1, 3), (2, 4)]))
2889 /// .into_keyed()
2890 /// # .entries()
2891 /// # }, |mut stream| async move {
2892 /// // { 1: [2, 3], 2: [4] }
2893 /// # for w in vec![(1, 2), (1, 3), (2, 4)] {
2894 /// # assert_eq!(stream.next().await.unwrap(), w);
2895 /// # }
2896 /// # }));
2897 /// # }
2898 /// ```
2899 pub fn into_keyed(self) -> KeyedStream<K, V, L, B, O, R> {
2900 KeyedStream::new(
2901 self.location.clone(),
2902 HydroNode::Cast {
2903 inner: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
2904 metadata: self
2905 .location
2906 .new_node_metadata(KeyedStream::<K, V, L, B, O, R>::collection_kind()),
2907 },
2908 )
2909 }
2910}
2911
2912impl<'a, K, V, L, O: Ordering, R: Retries> Stream<(K, V), Tick<L>, Bounded, O, R>
2913where
2914 K: Eq + Hash,
2915 L: Location<'a>,
2916{
2917 /// Given a stream of pairs `(K, V)`, produces a new stream of unique keys `K`.
2918 /// # Example
2919 /// ```rust
2920 /// # #[cfg(feature = "deploy")] {
2921 /// # use hydro_lang::prelude::*;
2922 /// # use futures::StreamExt;
2923 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
2924 /// let tick = process.tick();
2925 /// let numbers = process.source_iter(q!(vec![(1, 2), (2, 3), (1, 3), (2, 4)]));
2926 /// let batch = numbers.batch(&tick, nondet!(/** test */));
2927 /// batch.keys().all_ticks()
2928 /// # }, |mut stream| async move {
2929 /// // 1, 2
2930 /// # assert_eq!(stream.next().await.unwrap(), 1);
2931 /// # assert_eq!(stream.next().await.unwrap(), 2);
2932 /// # }));
2933 /// # }
2934 /// ```
2935 pub fn keys(self) -> Stream<K, Tick<L>, Bounded, NoOrder, ExactlyOnce> {
2936 self.into_keyed()
2937 .fold(
2938 q!(|| ()),
2939 q!(
2940 |_, _| {},
2941 commutative = manual_proof!(/** values are ignored */),
2942 idempotent = manual_proof!(/** values are ignored */)
2943 ),
2944 )
2945 .keys()
2946 }
2947}
2948
2949impl<'a, T, L, B: Boundedness, O: Ordering, R: Retries> Stream<T, Atomic<L>, B, O, R>
2950where
2951 L: Location<'a>,
2952{
2953 /// Returns a stream corresponding to the latest batch of elements being atomically
2954 /// processed. These batches are guaranteed to be contiguous across ticks and preserve
2955 /// the order of the input.
2956 ///
2957 /// # Non-Determinism
2958 /// The batch boundaries are non-deterministic and may change across executions.
2959 pub fn batch_atomic<L2: Location<'a, DropConsistency = L::DropConsistency>>(
2960 self,
2961 tick: &Tick<L2>,
2962 _nondet: NonDet,
2963 ) -> Stream<T, Tick<L::DropConsistency>, Bounded, O, R> {
2964 Stream::new(
2965 tick.drop_consistency(),
2966 HydroNode::Batch {
2967 inner: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
2968 metadata: tick
2969 .new_node_metadata(Stream::<T, Tick<L>, Bounded, O, R>::collection_kind()),
2970 },
2971 )
2972 }
2973
2974 /// Yields the elements of this stream back into a top-level, asynchronous execution context.
2975 /// See [`Stream::atomic`] for more details.
2976 pub fn end_atomic(self) -> Stream<T, L, B, O, R> {
2977 Stream::new(
2978 self.location.tick.l.clone(),
2979 HydroNode::EndAtomic {
2980 inner: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
2981 metadata: self
2982 .location
2983 .tick
2984 .l
2985 .new_node_metadata(Stream::<T, L, B, O, R>::collection_kind()),
2986 },
2987 )
2988 }
2989}
2990
2991impl<'a, F, T, L, B: Boundedness, O: Ordering, R: Retries> Stream<F, L, B, O, R>
2992where
2993 L: TopLevel<'a>,
2994 F: Future<Output = T>,
2995{
2996 /// Consumes a stream of `Future<T>`, produces a new stream of the resulting `T` outputs.
2997 /// Future outputs are produced as available, regardless of input arrival order.
2998 ///
2999 /// # Example
3000 /// ```rust
3001 /// # #[cfg(feature = "deploy")] {
3002 /// # use std::collections::HashSet;
3003 /// # use futures::StreamExt;
3004 /// # use hydro_lang::prelude::*;
3005 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
3006 /// process.source_iter(q!([2, 3, 1, 9, 6, 5, 4, 7, 8]))
3007 /// .map(q!(|x| async move {
3008 /// tokio::time::sleep(tokio::time::Duration::from_millis(10)).await;
3009 /// x
3010 /// }))
3011 /// .resolve_futures()
3012 /// # },
3013 /// # |mut stream| async move {
3014 /// // 1, 2, 3, 4, 5, 6, 7, 8, 9 (in any order)
3015 /// # let mut output = HashSet::new();
3016 /// # for _ in 1..10 {
3017 /// # output.insert(stream.next().await.unwrap());
3018 /// # }
3019 /// # assert_eq!(
3020 /// # output,
3021 /// # HashSet::<i32>::from_iter(1..10)
3022 /// # );
3023 /// # },
3024 /// # ));
3025 /// # }
3026 pub fn resolve_futures(self) -> Stream<T, L, Unbounded, NoOrder, R> {
3027 Stream::new(
3028 self.location.clone(),
3029 HydroNode::ResolveFutures {
3030 input: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
3031 metadata: self
3032 .location
3033 .new_node_metadata(Stream::<T, L, Unbounded, NoOrder, R>::collection_kind()),
3034 },
3035 )
3036 }
3037
3038 /// Consumes a stream of `Future<T>`, produces a new stream of the resulting `T` outputs.
3039 /// Future outputs are produced in the same order as the input stream.
3040 ///
3041 /// # Example
3042 /// ```rust
3043 /// # #[cfg(feature = "deploy")] {
3044 /// # use std::collections::HashSet;
3045 /// # use futures::StreamExt;
3046 /// # use hydro_lang::prelude::*;
3047 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
3048 /// process.source_iter(q!([2, 3, 1, 9, 6, 5, 4, 7, 8]))
3049 /// .map(q!(|x| async move {
3050 /// tokio::time::sleep(tokio::time::Duration::from_millis(10)).await;
3051 /// x
3052 /// }))
3053 /// .resolve_futures_ordered()
3054 /// # },
3055 /// # |mut stream| async move {
3056 /// // 2, 3, 1, 9, 6, 5, 4, 7, 8
3057 /// # let mut output = Vec::new();
3058 /// # for _ in 1..10 {
3059 /// # output.push(stream.next().await.unwrap());
3060 /// # }
3061 /// # assert_eq!(
3062 /// # output,
3063 /// # vec![2, 3, 1, 9, 6, 5, 4, 7, 8]
3064 /// # );
3065 /// # },
3066 /// # ));
3067 /// # }
3068 pub fn resolve_futures_ordered(self) -> Stream<T, L, Unbounded, O, R> {
3069 Stream::new(
3070 self.location.clone(),
3071 HydroNode::ResolveFuturesOrdered {
3072 input: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
3073 metadata: self
3074 .location
3075 .new_node_metadata(Stream::<T, L, Unbounded, O, R>::collection_kind()),
3076 },
3077 )
3078 }
3079}
3080
3081impl<'a, T, L, O: Ordering, R: Retries> Stream<T, Tick<L>, Bounded, O, R>
3082where
3083 L: Location<'a>,
3084{
3085 /// Asynchronously yields this batch of elements outside the tick as an unbounded stream,
3086 /// which will stream all the elements across _all_ tick iterations by concatenating the batches.
3087 pub fn all_ticks(self) -> Stream<T, L, Unbounded, O, R> {
3088 Stream::new(
3089 self.location.outer().clone(),
3090 HydroNode::YieldConcat {
3091 inner: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
3092 metadata: self
3093 .location
3094 .outer()
3095 .new_node_metadata(Stream::<T, L, Unbounded, O, R>::collection_kind()),
3096 },
3097 )
3098 }
3099
3100 /// Synchronously yields this batch of elements outside the tick as an unbounded stream,
3101 /// which will stream all the elements across _all_ tick iterations by concatenating the batches.
3102 ///
3103 /// Unlike [`Stream::all_ticks`], this preserves synchronous execution, as the output stream
3104 /// is emitted in an [`Atomic`] context that will process elements synchronously with the input
3105 /// stream's [`Tick`] context.
3106 pub fn all_ticks_atomic(self) -> Stream<T, Atomic<L>, Unbounded, O, R> {
3107 let out_location = Atomic {
3108 tick: self.location.clone(),
3109 };
3110
3111 Stream::new(
3112 out_location.clone(),
3113 HydroNode::YieldConcat {
3114 inner: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
3115 metadata: out_location
3116 .new_node_metadata(Stream::<T, Atomic<L>, Unbounded, O, R>::collection_kind()),
3117 },
3118 )
3119 }
3120
3121 /// Transforms the stream using the given closure in "stateful" mode, where stateful operators
3122 /// such as `fold` retrain their memory across ticks rather than resetting across batches of
3123 /// input.
3124 ///
3125 /// This API is particularly useful for stateful computation on batches of data, such as
3126 /// maintaining an accumulated state that is up to date with the current batch.
3127 ///
3128 /// # Example
3129 /// ```rust
3130 /// # #[cfg(feature = "deploy")] {
3131 /// # use hydro_lang::prelude::*;
3132 /// # use futures::StreamExt;
3133 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
3134 /// let tick = process.tick();
3135 /// # // ticks are lazy by default, forces the second tick to run
3136 /// # tick.spin_batch(q!(1)).all_ticks().for_each(q!(|_| {}));
3137 /// # let batch_first_tick = process
3138 /// # .source_iter(q!(vec![1, 2, 3, 4]))
3139 /// # .batch(&tick, nondet!(/** test */));
3140 /// # let batch_second_tick = process
3141 /// # .source_iter(q!(vec![5, 6, 7]))
3142 /// # .batch(&tick, nondet!(/** test */))
3143 /// # .defer_tick(); // appears on the second tick
3144 /// let input = // [1, 2, 3, 4 (first batch), 5, 6, 7 (second batch)]
3145 /// # batch_first_tick.chain(batch_second_tick).all_ticks();
3146 ///
3147 /// input.batch(&tick, nondet!(/** test */))
3148 /// .across_ticks(|s| s.count()).all_ticks()
3149 /// # }, |mut stream| async move {
3150 /// // [4, 7]
3151 /// assert_eq!(stream.next().await.unwrap(), 4);
3152 /// assert_eq!(stream.next().await.unwrap(), 7);
3153 /// # }));
3154 /// # }
3155 /// ```
3156 pub fn across_ticks<Out: BatchAtomic<'a>>(
3157 self,
3158 thunk: impl FnOnce(Stream<T, Atomic<L>, Unbounded, O, R>) -> Out,
3159 ) -> Out::Batched {
3160 thunk(self.all_ticks_atomic()).batched_atomic()
3161 }
3162
3163 /// Shifts the elements in `self` to the **next tick**, so that the returned stream at tick `T`
3164 /// always has the elements of `self` at tick `T - 1`.
3165 ///
3166 /// At tick `0`, the output stream is empty, since there is no previous tick.
3167 ///
3168 /// This operator enables stateful iterative processing with ticks, by sending data from one
3169 /// tick to the next. For example, you can use it to compare inputs across consecutive batches.
3170 ///
3171 /// # Example
3172 /// ```rust
3173 /// # #[cfg(feature = "deploy")] {
3174 /// # use hydro_lang::prelude::*;
3175 /// # use futures::StreamExt;
3176 /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
3177 /// let tick = process.tick();
3178 /// // ticks are lazy by default, forces the second tick to run
3179 /// tick.spin_batch(q!(1)).all_ticks().for_each(q!(|_| {}));
3180 ///
3181 /// let batch_first_tick = process
3182 /// .source_iter(q!(vec![1, 2, 3, 4]))
3183 /// .batch(&tick, nondet!(/** test */));
3184 /// let batch_second_tick = process
3185 /// .source_iter(q!(vec![0, 3, 4, 5, 6]))
3186 /// .batch(&tick, nondet!(/** test */))
3187 /// .defer_tick(); // appears on the second tick
3188 /// let changes_across_ticks = batch_first_tick.chain(batch_second_tick);
3189 ///
3190 /// changes_across_ticks.clone().filter_not_in(
3191 /// changes_across_ticks.defer_tick() // the elements from the previous tick
3192 /// ).all_ticks()
3193 /// # }, |mut stream| async move {
3194 /// // [1, 2, 3, 4 /* first tick */, 0, 5, 6 /* second tick */]
3195 /// # for w in vec![1, 2, 3, 4, 0, 5, 6] {
3196 /// # assert_eq!(stream.next().await.unwrap(), w);
3197 /// # }
3198 /// # }));
3199 /// # }
3200 /// ```
3201 pub fn defer_tick(self) -> Stream<T, Tick<L>, Bounded, O, R> {
3202 Stream::new(
3203 self.location.clone(),
3204 HydroNode::DeferTick {
3205 input: Box::new(self.ir_node.replace(HydroNode::Placeholder)),
3206 metadata: self
3207 .location
3208 .new_node_metadata(Stream::<T, Tick<L>, Bounded, O, R>::collection_kind()),
3209 },
3210 )
3211 }
3212}
3213
3214#[cfg(test)]
3215mod tests {
3216 #[cfg(feature = "deploy")]
3217 use futures::{SinkExt, StreamExt};
3218 #[cfg(feature = "deploy")]
3219 use hydro_deploy::Deployment;
3220 #[cfg(feature = "deploy")]
3221 use serde::{Deserialize, Serialize};
3222 #[cfg(any(feature = "deploy", feature = "sim"))]
3223 use stageleft::q;
3224
3225 #[cfg(any(feature = "deploy", feature = "sim"))]
3226 use crate::compile::builder::FlowBuilder;
3227 #[cfg(feature = "deploy")]
3228 use crate::live_collections::sliced::sliced;
3229 #[cfg(feature = "deploy")]
3230 use crate::live_collections::stream::ExactlyOnce;
3231 #[cfg(feature = "sim")]
3232 use crate::live_collections::stream::NoOrder;
3233 #[cfg(any(feature = "deploy", feature = "sim"))]
3234 use crate::live_collections::stream::TotalOrder;
3235 #[cfg(any(feature = "deploy", feature = "sim"))]
3236 use crate::location::Location;
3237 #[cfg(feature = "sim")]
3238 use crate::networking::TCP;
3239 #[cfg(any(feature = "deploy", feature = "sim"))]
3240 use crate::nondet::nondet;
3241
3242 mod backtrace_chained_ops;
3243
3244 #[cfg(feature = "deploy")]
3245 struct P1 {}
3246 #[cfg(feature = "deploy")]
3247 struct P2 {}
3248
3249 #[cfg(feature = "deploy")]
3250 #[derive(Serialize, Deserialize, Debug)]
3251 struct SendOverNetwork {
3252 n: u32,
3253 }
3254
3255 #[cfg(feature = "deploy")]
3256 #[tokio::test]
3257 async fn first_ten_distributed() {
3258 use crate::networking::TCP;
3259
3260 let mut deployment = Deployment::new();
3261
3262 let mut flow = FlowBuilder::new();
3263 let first_node = flow.process::<P1>();
3264 let second_node = flow.process::<P2>();
3265 let external = flow.external::<P2>();
3266
3267 let numbers = first_node.source_iter(q!(0..10));
3268 let out_port = numbers
3269 .map(q!(|n| SendOverNetwork { n }))
3270 .send(&second_node, TCP.fail_stop().bincode())
3271 .send_bincode_external(&external);
3272
3273 let nodes = flow
3274 .with_process(&first_node, deployment.Localhost())
3275 .with_process(&second_node, deployment.Localhost())
3276 .with_external(&external, deployment.Localhost())
3277 .deploy(&mut deployment);
3278
3279 deployment.deploy().await.unwrap();
3280
3281 let mut external_out = nodes.connect(out_port).await;
3282
3283 deployment.start().await.unwrap();
3284
3285 for i in 0..10 {
3286 assert_eq!(external_out.next().await.unwrap().n, i);
3287 }
3288 }
3289
3290 #[cfg(feature = "deploy")]
3291 #[tokio::test]
3292 async fn first_cardinality() {
3293 let mut deployment = Deployment::new();
3294
3295 let mut flow = FlowBuilder::new();
3296 let node = flow.process::<()>();
3297 let external = flow.external::<()>();
3298
3299 let node_tick = node.tick();
3300 let count = node_tick
3301 .singleton(q!([1, 2, 3]))
3302 .into_stream()
3303 .flatten_ordered()
3304 .first()
3305 .into_stream()
3306 .count()
3307 .all_ticks()
3308 .send_bincode_external(&external);
3309
3310 let nodes = flow
3311 .with_process(&node, deployment.Localhost())
3312 .with_external(&external, deployment.Localhost())
3313 .deploy(&mut deployment);
3314
3315 deployment.deploy().await.unwrap();
3316
3317 let mut external_out = nodes.connect(count).await;
3318
3319 deployment.start().await.unwrap();
3320
3321 assert_eq!(external_out.next().await.unwrap(), 1);
3322 }
3323
3324 #[cfg(feature = "deploy")]
3325 #[tokio::test]
3326 async fn unbounded_reduce_remembers_state() {
3327 let mut deployment = Deployment::new();
3328
3329 let mut flow = FlowBuilder::new();
3330 let node = flow.process::<()>();
3331 let external = flow.external::<()>();
3332
3333 let (input_port, input) = node.source_external_bincode(&external);
3334 let out = input
3335 .reduce(q!(|acc, v| *acc += v))
3336 .sample_eager(nondet!(/** test */))
3337 .send_bincode_external(&external);
3338
3339 let nodes = flow
3340 .with_process(&node, deployment.Localhost())
3341 .with_external(&external, deployment.Localhost())
3342 .deploy(&mut deployment);
3343
3344 deployment.deploy().await.unwrap();
3345
3346 let mut external_in = nodes.connect(input_port).await;
3347 let mut external_out = nodes.connect(out).await;
3348
3349 deployment.start().await.unwrap();
3350
3351 external_in.send(1).await.unwrap();
3352 assert_eq!(external_out.next().await.unwrap(), 1);
3353
3354 external_in.send(2).await.unwrap();
3355 assert_eq!(external_out.next().await.unwrap(), 3);
3356 }
3357
3358 #[cfg(feature = "deploy")]
3359 #[tokio::test]
3360 async fn top_level_bounded_cross_singleton() {
3361 let mut deployment = Deployment::new();
3362
3363 let mut flow = FlowBuilder::new();
3364 let node = flow.process::<()>();
3365 let external = flow.external::<()>();
3366
3367 let (input_port, input) =
3368 node.source_external_bincode::<_, _, TotalOrder, ExactlyOnce>(&external);
3369
3370 let out = input
3371 .cross_singleton(
3372 node.source_iter(q!(vec![1, 2, 3]))
3373 .fold(q!(|| 0), q!(|acc, v| *acc += v)),
3374 )
3375 .send_bincode_external(&external);
3376
3377 let nodes = flow
3378 .with_process(&node, deployment.Localhost())
3379 .with_external(&external, deployment.Localhost())
3380 .deploy(&mut deployment);
3381
3382 deployment.deploy().await.unwrap();
3383
3384 let mut external_in = nodes.connect(input_port).await;
3385 let mut external_out = nodes.connect(out).await;
3386
3387 deployment.start().await.unwrap();
3388
3389 external_in.send(1).await.unwrap();
3390 assert_eq!(external_out.next().await.unwrap(), (1, 6));
3391
3392 external_in.send(2).await.unwrap();
3393 assert_eq!(external_out.next().await.unwrap(), (2, 6));
3394 }
3395
3396 #[cfg(feature = "deploy")]
3397 #[tokio::test]
3398 async fn top_level_bounded_reduce_cardinality() {
3399 let mut deployment = Deployment::new();
3400
3401 let mut flow = FlowBuilder::new();
3402 let node = flow.process::<()>();
3403 let external = flow.external::<()>();
3404
3405 let (input_port, input) =
3406 node.source_external_bincode::<_, _, TotalOrder, ExactlyOnce>(&external);
3407
3408 let out = sliced! {
3409 let input = use(input, nondet!(/** test */));
3410 let v = use(node.source_iter(q!(vec![1, 2, 3])).reduce(q!(|acc, v| *acc += v)), nondet!(/** test */));
3411 input.cross_singleton(v.into_stream().count())
3412 }
3413 .send_bincode_external(&external);
3414
3415 let nodes = flow
3416 .with_process(&node, deployment.Localhost())
3417 .with_external(&external, deployment.Localhost())
3418 .deploy(&mut deployment);
3419
3420 deployment.deploy().await.unwrap();
3421
3422 let mut external_in = nodes.connect(input_port).await;
3423 let mut external_out = nodes.connect(out).await;
3424
3425 deployment.start().await.unwrap();
3426
3427 external_in.send(1).await.unwrap();
3428 assert_eq!(external_out.next().await.unwrap(), (1, 1));
3429
3430 external_in.send(2).await.unwrap();
3431 assert_eq!(external_out.next().await.unwrap(), (2, 1));
3432 }
3433
3434 #[cfg(feature = "deploy")]
3435 #[tokio::test]
3436 async fn top_level_bounded_into_singleton_cardinality() {
3437 let mut deployment = Deployment::new();
3438
3439 let mut flow = FlowBuilder::new();
3440 let node = flow.process::<()>();
3441 let external = flow.external::<()>();
3442
3443 let (input_port, input) =
3444 node.source_external_bincode::<_, _, TotalOrder, ExactlyOnce>(&external);
3445
3446 let out = sliced! {
3447 let input = use(input, nondet!(/** test */));
3448 let v = use(node.source_iter(q!(vec![1, 2, 3])).reduce(q!(|acc, v| *acc += v)).into_singleton(), nondet!(/** test */));
3449 input.cross_singleton(v.into_stream().count())
3450 }
3451 .send_bincode_external(&external);
3452
3453 let nodes = flow
3454 .with_process(&node, deployment.Localhost())
3455 .with_external(&external, deployment.Localhost())
3456 .deploy(&mut deployment);
3457
3458 deployment.deploy().await.unwrap();
3459
3460 let mut external_in = nodes.connect(input_port).await;
3461 let mut external_out = nodes.connect(out).await;
3462
3463 deployment.start().await.unwrap();
3464
3465 external_in.send(1).await.unwrap();
3466 assert_eq!(external_out.next().await.unwrap(), (1, 1));
3467
3468 external_in.send(2).await.unwrap();
3469 assert_eq!(external_out.next().await.unwrap(), (2, 1));
3470 }
3471
3472 #[cfg(feature = "deploy")]
3473 #[tokio::test]
3474 async fn atomic_fold_replays_each_tick() {
3475 let mut deployment = Deployment::new();
3476
3477 let mut flow = FlowBuilder::new();
3478 let node = flow.process::<()>();
3479 let external = flow.external::<()>();
3480
3481 let (input_port, input) =
3482 node.source_external_bincode::<_, _, TotalOrder, ExactlyOnce>(&external);
3483 let tick = node.tick();
3484
3485 let out = input
3486 .batch(&tick, nondet!(/** test */))
3487 .cross_singleton(
3488 node.source_iter(q!(vec![1, 2, 3]))
3489 .atomic()
3490 .fold(q!(|| 0), q!(|acc, v| *acc += v))
3491 .snapshot_atomic(&tick, nondet!(/** test */)),
3492 )
3493 .all_ticks()
3494 .send_bincode_external(&external);
3495
3496 let nodes = flow
3497 .with_process(&node, deployment.Localhost())
3498 .with_external(&external, deployment.Localhost())
3499 .deploy(&mut deployment);
3500
3501 deployment.deploy().await.unwrap();
3502
3503 let mut external_in = nodes.connect(input_port).await;
3504 let mut external_out = nodes.connect(out).await;
3505
3506 deployment.start().await.unwrap();
3507
3508 external_in.send(1).await.unwrap();
3509 assert_eq!(external_out.next().await.unwrap(), (1, 6));
3510
3511 external_in.send(2).await.unwrap();
3512 assert_eq!(external_out.next().await.unwrap(), (2, 6));
3513 }
3514
3515 #[cfg(feature = "deploy")]
3516 #[tokio::test]
3517 async fn unbounded_scan_remembers_state() {
3518 let mut deployment = Deployment::new();
3519
3520 let mut flow = FlowBuilder::new();
3521 let node = flow.process::<()>();
3522 let external = flow.external::<()>();
3523
3524 let (input_port, input) = node.source_external_bincode(&external);
3525 let out = input
3526 .scan(
3527 q!(|| 0),
3528 q!(|acc, v| {
3529 *acc += v;
3530 Some(*acc)
3531 }),
3532 )
3533 .send_bincode_external(&external);
3534
3535 let nodes = flow
3536 .with_process(&node, deployment.Localhost())
3537 .with_external(&external, deployment.Localhost())
3538 .deploy(&mut deployment);
3539
3540 deployment.deploy().await.unwrap();
3541
3542 let mut external_in = nodes.connect(input_port).await;
3543 let mut external_out = nodes.connect(out).await;
3544
3545 deployment.start().await.unwrap();
3546
3547 external_in.send(1).await.unwrap();
3548 assert_eq!(external_out.next().await.unwrap(), 1);
3549
3550 external_in.send(2).await.unwrap();
3551 assert_eq!(external_out.next().await.unwrap(), 3);
3552 }
3553
3554 #[cfg(feature = "deploy")]
3555 #[tokio::test]
3556 async fn unbounded_enumerate_remembers_state() {
3557 let mut deployment = Deployment::new();
3558
3559 let mut flow = FlowBuilder::new();
3560 let node = flow.process::<()>();
3561 let external = flow.external::<()>();
3562
3563 let (input_port, input) = node.source_external_bincode(&external);
3564 let out = input.enumerate().send_bincode_external(&external);
3565
3566 let nodes = flow
3567 .with_process(&node, deployment.Localhost())
3568 .with_external(&external, deployment.Localhost())
3569 .deploy(&mut deployment);
3570
3571 deployment.deploy().await.unwrap();
3572
3573 let mut external_in = nodes.connect(input_port).await;
3574 let mut external_out = nodes.connect(out).await;
3575
3576 deployment.start().await.unwrap();
3577
3578 external_in.send(1).await.unwrap();
3579 assert_eq!(external_out.next().await.unwrap(), (0, 1));
3580
3581 external_in.send(2).await.unwrap();
3582 assert_eq!(external_out.next().await.unwrap(), (1, 2));
3583 }
3584
3585 #[cfg(feature = "deploy")]
3586 #[tokio::test]
3587 async fn unbounded_unique_remembers_state() {
3588 let mut deployment = Deployment::new();
3589
3590 let mut flow = FlowBuilder::new();
3591 let node = flow.process::<()>();
3592 let external = flow.external::<()>();
3593
3594 let (input_port, input) =
3595 node.source_external_bincode::<_, _, TotalOrder, ExactlyOnce>(&external);
3596 let out = input.unique().send_bincode_external(&external);
3597
3598 let nodes = flow
3599 .with_process(&node, deployment.Localhost())
3600 .with_external(&external, deployment.Localhost())
3601 .deploy(&mut deployment);
3602
3603 deployment.deploy().await.unwrap();
3604
3605 let mut external_in = nodes.connect(input_port).await;
3606 let mut external_out = nodes.connect(out).await;
3607
3608 deployment.start().await.unwrap();
3609
3610 external_in.send(1).await.unwrap();
3611 assert_eq!(external_out.next().await.unwrap(), 1);
3612
3613 external_in.send(2).await.unwrap();
3614 assert_eq!(external_out.next().await.unwrap(), 2);
3615
3616 external_in.send(1).await.unwrap();
3617 external_in.send(3).await.unwrap();
3618 assert_eq!(external_out.next().await.unwrap(), 3);
3619 }
3620
3621 #[cfg(feature = "sim")]
3622 #[test]
3623 #[should_panic]
3624 fn sim_batch_nondet_size() {
3625 let mut flow = FlowBuilder::new();
3626 let node = flow.process::<()>();
3627
3628 let (in_send, input) = node.sim_input::<_, TotalOrder, _>();
3629
3630 let tick = node.tick();
3631 let out_recv = input
3632 .batch(&tick, nondet!(/** test */))
3633 .count()
3634 .all_ticks()
3635 .sim_output();
3636
3637 flow.sim().exhaustive(async || {
3638 in_send.send(());
3639 in_send.send(());
3640 in_send.send(());
3641
3642 assert_eq!(out_recv.next().await.unwrap(), 3); // fails with nondet batching
3643 });
3644 }
3645
3646 #[cfg(feature = "sim")]
3647 #[test]
3648 fn sim_batch_preserves_order() {
3649 let mut flow = FlowBuilder::new();
3650 let node = flow.process::<()>();
3651
3652 let (in_send, input) = node.sim_input();
3653
3654 let tick = node.tick();
3655 let out_recv = input
3656 .batch(&tick, nondet!(/** test */))
3657 .all_ticks()
3658 .sim_output();
3659
3660 flow.sim().exhaustive(async || {
3661 in_send.send(1);
3662 in_send.send(2);
3663 in_send.send(3);
3664
3665 out_recv.assert_yields_only([1, 2, 3]).await;
3666 });
3667 }
3668
3669 #[cfg(feature = "sim")]
3670 #[test]
3671 #[should_panic]
3672 fn sim_batch_unordered_shuffles() {
3673 let mut flow = FlowBuilder::new();
3674 let node = flow.process::<()>();
3675
3676 let (in_send, input) = node.sim_input::<_, NoOrder, _>();
3677
3678 let tick = node.tick();
3679 let batch = input.batch(&tick, nondet!(/** test */));
3680 let out_recv = batch
3681 .clone()
3682 .min()
3683 .zip(batch.max())
3684 .all_ticks()
3685 .sim_output();
3686
3687 flow.sim().exhaustive(async || {
3688 in_send.send_many_unordered([1, 2, 3]);
3689
3690 if out_recv.collect::<Vec<_>>().await == vec![(1, 3), (2, 2)] {
3691 panic!("saw both (1, 3) and (2, 2), so batching must have shuffled the order");
3692 }
3693 });
3694 }
3695
3696 #[cfg(feature = "sim")]
3697 #[test]
3698 fn sim_batch_unordered_shuffles_count() {
3699 let mut flow = FlowBuilder::new();
3700 let node = flow.process::<()>();
3701
3702 let (in_send, input) = node.sim_input::<_, NoOrder, _>();
3703
3704 let tick = node.tick();
3705 let batch = input.batch(&tick, nondet!(/** test */));
3706 let out_recv = batch.all_ticks().sim_output();
3707
3708 let instance_count = flow.sim().exhaustive(async || {
3709 in_send.send_many_unordered([1, 2, 3, 4]);
3710 out_recv.assert_yields_only_unordered([1, 2, 3, 4]).await;
3711 });
3712
3713 assert_eq!(
3714 instance_count,
3715 75 // ∑ (k=1 to 4) S(4,k) × k! = 75
3716 )
3717 }
3718
3719 #[cfg(feature = "sim")]
3720 #[test]
3721 #[should_panic]
3722 fn sim_observe_order_batched() {
3723 let mut flow = FlowBuilder::new();
3724 let node = flow.process::<()>();
3725
3726 let (in_send, input) = node.sim_input::<_, NoOrder, _>();
3727
3728 let tick = node.tick();
3729 let batch = input.batch(&tick, nondet!(/** test */));
3730 let out_recv = batch
3731 .assume_ordering::<TotalOrder>(nondet!(/** test */))
3732 .all_ticks()
3733 .sim_output();
3734
3735 flow.sim().exhaustive(async || {
3736 in_send.send_many_unordered([1, 2, 3, 4]);
3737 out_recv.assert_yields_only([1, 2, 3, 4]).await; // fails with assume_ordering
3738 });
3739 }
3740
3741 #[cfg(feature = "sim")]
3742 #[test]
3743 fn sim_observe_order_batched_count() {
3744 let mut flow = FlowBuilder::new();
3745 let node = flow.process::<()>();
3746
3747 let (in_send, input) = node.sim_input::<_, NoOrder, _>();
3748
3749 let tick = node.tick();
3750 let batch = input.batch(&tick, nondet!(/** test */));
3751 let out_recv = batch
3752 .assume_ordering::<TotalOrder>(nondet!(/** test */))
3753 .all_ticks()
3754 .sim_output();
3755
3756 let instance_count = flow.sim().exhaustive(async || {
3757 in_send.send_many_unordered([1, 2, 3, 4]);
3758 let _ = out_recv.collect::<Vec<_>>().await;
3759 });
3760
3761 assert_eq!(
3762 instance_count,
3763 192 // 4! * 2^{4 - 1}
3764 )
3765 }
3766
3767 #[cfg(feature = "sim")]
3768 #[test]
3769 fn sim_unordered_count_instance_count() {
3770 let mut flow = FlowBuilder::new();
3771 let node = flow.process::<()>();
3772
3773 let (in_send, input) = node.sim_input::<_, NoOrder, _>();
3774
3775 let tick = node.tick();
3776 let out_recv = input
3777 .count()
3778 .snapshot(&tick, nondet!(/** test */))
3779 .all_ticks()
3780 .sim_output();
3781
3782 let instance_count = flow.sim().exhaustive(async || {
3783 in_send.send_many_unordered([1, 2, 3, 4]);
3784 assert!(out_recv.collect::<Vec<_>>().await.last().unwrap() == &4);
3785 });
3786
3787 assert_eq!(
3788 instance_count,
3789 16 // 2^4, { 0, 1, 2, 3 } can be a snapshot and 4 is always included
3790 )
3791 }
3792
3793 #[cfg(feature = "sim")]
3794 #[test]
3795 fn sim_top_level_assume_ordering() {
3796 let mut flow = FlowBuilder::new();
3797 let node = flow.process::<()>();
3798
3799 let (in_send, input) = node.sim_input::<_, NoOrder, _>();
3800
3801 let out_recv = input
3802 .assume_ordering::<TotalOrder>(nondet!(/** test */))
3803 .sim_output();
3804
3805 let instance_count = flow.sim().exhaustive(async || {
3806 in_send.send_many_unordered([1, 2, 3]);
3807 let mut out = out_recv.collect::<Vec<_>>().await;
3808 out.sort();
3809 assert_eq!(out, vec![1, 2, 3]);
3810 });
3811
3812 assert_eq!(instance_count, 6)
3813 }
3814
3815 #[cfg(feature = "sim")]
3816 #[test]
3817 fn sim_top_level_assume_ordering_cycle_back() {
3818 let mut flow = FlowBuilder::new();
3819 let node = flow.process::<()>();
3820 let node2 = flow.process::<()>();
3821
3822 let (in_send, input) = node.sim_input::<_, NoOrder, _>();
3823
3824 let (complete_cycle_back, cycle_back) =
3825 node.forward_ref::<super::Stream<_, _, _, NoOrder>>();
3826 let ordered = input
3827 .merge_unordered(cycle_back)
3828 .assume_ordering::<TotalOrder>(nondet!(/** test */));
3829 complete_cycle_back.complete(
3830 ordered
3831 .clone()
3832 .map(q!(|v| v + 1))
3833 .filter(q!(|v| v % 2 == 1))
3834 .send(&node2, TCP.fail_stop().bincode())
3835 .send(&node, TCP.fail_stop().bincode()),
3836 );
3837
3838 let out_recv = ordered.sim_output();
3839
3840 let mut saw = false;
3841 let instance_count = flow.sim().exhaustive(async || {
3842 in_send.send_many_unordered([0, 2]);
3843 let out = out_recv.collect::<Vec<_>>().await;
3844
3845 if out.starts_with(&[0, 1, 2]) {
3846 saw = true;
3847 }
3848 });
3849
3850 assert!(saw, "did not see an instance with 0, 1, 2 in order");
3851 assert_eq!(instance_count, 6);
3852 }
3853
3854 #[cfg(feature = "sim")]
3855 #[test]
3856 fn sim_top_level_assume_ordering_cycle_back_tick() {
3857 let mut flow = FlowBuilder::new();
3858 let node = flow.process::<()>();
3859 let node2 = flow.process::<()>();
3860
3861 let (in_send, input) = node.sim_input::<_, NoOrder, _>();
3862
3863 let (complete_cycle_back, cycle_back) =
3864 node.forward_ref::<super::Stream<_, _, _, NoOrder>>();
3865 let ordered = input
3866 .merge_unordered(cycle_back)
3867 .assume_ordering::<TotalOrder>(nondet!(/** test */));
3868 complete_cycle_back.complete(
3869 ordered
3870 .clone()
3871 .batch(&node.tick(), nondet!(/** test */))
3872 .all_ticks()
3873 .map(q!(|v| v + 1))
3874 .filter(q!(|v| v % 2 == 1))
3875 .send(&node2, TCP.fail_stop().bincode())
3876 .send(&node, TCP.fail_stop().bincode()),
3877 );
3878
3879 let out_recv = ordered.sim_output();
3880
3881 let mut saw = false;
3882 let instance_count = flow.sim().exhaustive(async || {
3883 in_send.send_many_unordered([0, 2]);
3884 let out = out_recv.collect::<Vec<_>>().await;
3885
3886 if out.starts_with(&[0, 1, 2]) {
3887 saw = true;
3888 }
3889 });
3890
3891 assert!(saw, "did not see an instance with 0, 1, 2 in order");
3892 assert_eq!(instance_count, 58);
3893 }
3894
3895 #[cfg(feature = "sim")]
3896 #[test]
3897 fn sim_top_level_assume_ordering_multiple() {
3898 let mut flow = FlowBuilder::new();
3899 let node = flow.process::<()>();
3900 let node2 = flow.process::<()>();
3901
3902 let (in_send, input) = node.sim_input::<_, NoOrder, _>();
3903 let (_, input2) = node.sim_input::<_, NoOrder, _>();
3904
3905 let (complete_cycle_back, cycle_back) =
3906 node.forward_ref::<super::Stream<_, _, _, NoOrder>>();
3907 let input1_ordered = input
3908 .clone()
3909 .merge_unordered(cycle_back)
3910 .assume_ordering::<TotalOrder>(nondet!(/** test */));
3911 let foo = input1_ordered
3912 .clone()
3913 .map(q!(|v| v + 3))
3914 .weaken_ordering::<NoOrder>()
3915 .merge_unordered(input2)
3916 .assume_ordering::<TotalOrder>(nondet!(/** test */));
3917
3918 complete_cycle_back.complete(
3919 foo.filter(q!(|v| *v == 3))
3920 .send(&node2, TCP.fail_stop().bincode())
3921 .send(&node, TCP.fail_stop().bincode()),
3922 );
3923
3924 let out_recv = input1_ordered.sim_output();
3925
3926 let mut saw = false;
3927 let instance_count = flow.sim().exhaustive(async || {
3928 in_send.send_many_unordered([0, 1]);
3929 let out = out_recv.collect::<Vec<_>>().await;
3930
3931 if out.starts_with(&[0, 3, 1]) {
3932 saw = true;
3933 }
3934 });
3935
3936 assert!(saw, "did not see an instance with 0, 3, 1 in order");
3937 assert_eq!(instance_count, 24);
3938 }
3939
3940 #[cfg(feature = "sim")]
3941 #[test]
3942 fn sim_atomic_assume_ordering_cycle_back() {
3943 let mut flow = FlowBuilder::new();
3944 let node = flow.process::<()>();
3945 let node2 = flow.process::<()>();
3946
3947 let (in_send, input) = node.sim_input::<_, NoOrder, _>();
3948
3949 let (complete_cycle_back, cycle_back) =
3950 node.forward_ref::<super::Stream<_, _, _, NoOrder>>();
3951 let ordered = input
3952 .merge_unordered(cycle_back)
3953 .atomic()
3954 .assume_ordering::<TotalOrder>(nondet!(/** test */))
3955 .end_atomic();
3956 complete_cycle_back.complete(
3957 ordered
3958 .clone()
3959 .map(q!(|v| v + 1))
3960 .filter(q!(|v| v % 2 == 1))
3961 .send(&node2, TCP.fail_stop().bincode())
3962 .send(&node, TCP.fail_stop().bincode()),
3963 );
3964
3965 let out_recv = ordered.sim_output();
3966
3967 let instance_count = flow.sim().exhaustive(async || {
3968 in_send.send_many_unordered([0, 2]);
3969 let out = out_recv.collect::<Vec<_>>().await;
3970 assert_eq!(out.len(), 4);
3971 });
3972 assert_eq!(instance_count, 22);
3973 }
3974
3975 #[cfg(feature = "deploy")]
3976 #[tokio::test]
3977 async fn partition_evens_odds() {
3978 let mut deployment = Deployment::new();
3979
3980 let mut flow = FlowBuilder::new();
3981 let node = flow.process::<()>();
3982 let external = flow.external::<()>();
3983
3984 let numbers = node.source_iter(q!(vec![1i32, 2, 3, 4, 5, 6]));
3985 let (evens, odds) = numbers.partition(q!(|x: &i32| x % 2 == 0));
3986 let evens_port = evens.send_bincode_external(&external);
3987 let odds_port = odds.send_bincode_external(&external);
3988
3989 let nodes = flow
3990 .with_process(&node, deployment.Localhost())
3991 .with_external(&external, deployment.Localhost())
3992 .deploy(&mut deployment);
3993
3994 deployment.deploy().await.unwrap();
3995
3996 let mut evens_out = nodes.connect(evens_port).await;
3997 let mut odds_out = nodes.connect(odds_port).await;
3998
3999 deployment.start().await.unwrap();
4000
4001 let mut even_results = Vec::new();
4002 for _ in 0..3 {
4003 even_results.push(evens_out.next().await.unwrap());
4004 }
4005 even_results.sort();
4006 assert_eq!(even_results, vec![2, 4, 6]);
4007
4008 let mut odd_results = Vec::new();
4009 for _ in 0..3 {
4010 odd_results.push(odds_out.next().await.unwrap());
4011 }
4012 odd_results.sort();
4013 assert_eq!(odd_results, vec![1, 3, 5]);
4014 }
4015
4016 #[cfg(feature = "deploy")]
4017 #[tokio::test]
4018 async fn unconsumed_inspect_still_runs() {
4019 use crate::deploy::DeployCrateWrapper;
4020
4021 let mut deployment = Deployment::new();
4022
4023 let mut flow = FlowBuilder::new();
4024 let node = flow.process::<()>();
4025
4026 // The return value of .inspect() is intentionally dropped.
4027 // Before the Null-root fix, this would silently do nothing.
4028 node.source_iter(q!(0..5))
4029 .inspect(q!(|x| println!("inspect: {}", x)));
4030
4031 let nodes = flow
4032 .with_process(&node, deployment.Localhost())
4033 .deploy(&mut deployment);
4034
4035 deployment.deploy().await.unwrap();
4036
4037 let mut stdout = nodes.get_process(&node).stdout();
4038
4039 deployment.start().await.unwrap();
4040
4041 let mut lines = Vec::new();
4042 for _ in 0..5 {
4043 lines.push(stdout.recv().await.unwrap());
4044 }
4045 lines.sort();
4046 assert_eq!(
4047 lines,
4048 vec![
4049 "inspect: 0",
4050 "inspect: 1",
4051 "inspect: 2",
4052 "inspect: 3",
4053 "inspect: 4",
4054 ]
4055 );
4056 }
4057
4058 #[cfg(feature = "sim")]
4059 #[test]
4060 fn sim_limit() {
4061 let mut flow = FlowBuilder::new();
4062 let node = flow.process::<()>();
4063
4064 let (in_send, input) = node.sim_input();
4065
4066 let out_recv = input.limit(q!(3)).sim_output();
4067
4068 flow.sim().exhaustive(async || {
4069 in_send.send(1);
4070 in_send.send(2);
4071 in_send.send(3);
4072 in_send.send(4);
4073 in_send.send(5);
4074
4075 out_recv.assert_yields_only([1, 2, 3]).await;
4076 });
4077 }
4078
4079 #[cfg(feature = "sim")]
4080 #[test]
4081 fn sim_limit_zero() {
4082 let mut flow = FlowBuilder::new();
4083 let node = flow.process::<()>();
4084
4085 let (in_send, input) = node.sim_input();
4086
4087 let out_recv = input.limit(q!(0)).sim_output();
4088
4089 flow.sim().exhaustive(async || {
4090 in_send.send(1);
4091 in_send.send(2);
4092
4093 out_recv.assert_yields_only::<i32, _>([]).await;
4094 });
4095 }
4096
4097 #[cfg(feature = "sim")]
4098 #[test]
4099 fn sim_merge_ordered() {
4100 let mut flow = FlowBuilder::new();
4101 let node = flow.process::<()>();
4102
4103 let (in_send, input) = node.sim_input();
4104 let (in_send2, input2) = node.sim_input();
4105
4106 let out_recv = input
4107 .merge_ordered(input2, nondet!(/** test */))
4108 .sim_output();
4109
4110 let mut saw_out_of_order = false;
4111 let instances = flow.sim().exhaustive(async || {
4112 in_send.send(1);
4113 in_send.send(2);
4114 in_send2.send(3);
4115 in_send2.send(4);
4116
4117 let out = out_recv.collect::<Vec<_>>().await;
4118
4119 if out == [1, 3, 2, 4] {
4120 saw_out_of_order = true;
4121 }
4122
4123 // Assert ordering preservation: elements from each input must
4124 // appear in their original relative order.
4125 let mut first_elements = out.iter().filter(|v| **v <= 2).copied().collect::<Vec<_>>();
4126 let mut second_elements = out.iter().filter(|v| **v > 2).copied().collect::<Vec<_>>();
4127 assert_eq!(
4128 first_elements,
4129 vec![1, 2],
4130 "first input order violated: {:?}",
4131 out
4132 );
4133 assert_eq!(
4134 second_elements,
4135 vec![3, 4],
4136 "second input order violated: {:?}",
4137 out
4138 );
4139
4140 first_elements.append(&mut second_elements);
4141 first_elements.sort();
4142 assert_eq!(first_elements, vec![1, 2, 3, 4]);
4143 });
4144
4145 assert!(saw_out_of_order);
4146 assert_eq!(instances, 6);
4147 }
4148
4149 /// Tests that merge_ordered passes through elements when only one input
4150 /// has data.
4151 #[cfg(feature = "sim")]
4152 #[test]
4153 fn sim_merge_ordered_one_empty() {
4154 let mut flow = FlowBuilder::new();
4155 let node = flow.process::<()>();
4156
4157 let (in_send, input) = node.sim_input();
4158 let (_in_send2, input2) = node.sim_input();
4159
4160 let out_recv = input
4161 .merge_ordered(input2, nondet!(/** test */))
4162 .sim_output();
4163
4164 let instances = flow.sim().exhaustive(async || {
4165 in_send.send(1);
4166 in_send.send(2);
4167
4168 let out = out_recv.collect::<Vec<_>>().await;
4169 assert_eq!(out, vec![1, 2]);
4170 });
4171
4172 // Only one possible interleaving when one input is empty
4173 assert_eq!(instances, 1);
4174 }
4175
4176 /// Tests that merge_ordered correctly handles feedback cycles.
4177 /// An element output from merge_ordered is filtered and cycled back to
4178 /// one of its inputs. The one-at-a-time release must allow the cycled-back
4179 /// element to arrive and potentially be emitted before elements still
4180 /// waiting on the other input.
4181 #[cfg(feature = "sim")]
4182 #[test]
4183 fn sim_merge_ordered_cycle_back() {
4184 let mut flow = FlowBuilder::new();
4185 let node = flow.process::<()>();
4186
4187 let (in_send, input) = node.sim_input();
4188
4189 // Create a forward ref for the cycle back
4190 let (complete_cycle_back, cycle_back) =
4191 node.forward_ref::<super::Stream<_, _, _, TotalOrder>>();
4192
4193 // merge_ordered: input (external) with cycle_back
4194 let merged = input.merge_ordered(cycle_back, nondet!(/** test */));
4195
4196 // Cycle back: elements equal to 1 get mapped to 10 and fed back
4197 complete_cycle_back.complete(merged.clone().filter(q!(|v| *v == 1)).map(q!(|v| v * 10)));
4198
4199 let out_recv = merged.sim_output();
4200
4201 // Send 1 and 2. Element 1 should cycle back as 10.
4202 // Valid orderings must have 1 before 10 (since 10 depends on 1).
4203 let mut saw_cycle_before_second = false;
4204 flow.sim().exhaustive(async || {
4205 in_send.send(1);
4206 in_send.send(2);
4207
4208 let out = out_recv.collect::<Vec<_>>().await;
4209
4210 // 10 must always come after 1 (causal dependency)
4211 let pos_1 = out.iter().position(|v| *v == 1).unwrap();
4212 let pos_10 = out.iter().position(|v| *v == 10).unwrap();
4213 assert!(pos_1 < pos_10, "causal order violated: {:?}", out);
4214
4215 // Check if we see [1, 10, 2] — the cycled element beats the second input
4216 if out == [1, 10, 2] {
4217 saw_cycle_before_second = true;
4218 }
4219
4220 let mut sorted = out;
4221 sorted.sort();
4222 assert_eq!(sorted, vec![1, 2, 10]);
4223 });
4224
4225 assert!(
4226 saw_cycle_before_second,
4227 "never saw the cycled element arrive before the second input element"
4228 );
4229 }
4230
4231 /// Tests that merge_ordered correctly interleaves when one input has a
4232 /// delayed element. With a: [1, _delay_, 2] and b: [3, 4], the delayed
4233 /// element 2 should be able to appear after b's elements.
4234 #[cfg(feature = "sim")]
4235 #[test]
4236 fn sim_merge_ordered_delayed() {
4237 let mut flow = FlowBuilder::new();
4238 let node = flow.process::<()>();
4239
4240 let (in_send, input) = node.sim_input();
4241 let (in_send2, input2) = node.sim_input();
4242
4243 let out_recv = input
4244 .merge_ordered(input2, nondet!(/** test */))
4245 .sim_output();
4246
4247 let mut saw_delayed_interleaving = false;
4248 flow.sim().exhaustive(async || {
4249 // Send 1 from a, and 3, 4 from b
4250 in_send.send(1);
4251 in_send2.send(3);
4252 in_send2.send(4);
4253
4254 // Collect what's available so far
4255 let first_batch = out_recv.collect::<Vec<_>>().await;
4256
4257 // Now send the delayed element 2 from a
4258 in_send.send(2);
4259 let second_batch = out_recv.collect::<Vec<_>>().await;
4260
4261 let mut all: Vec<_> = first_batch
4262 .iter()
4263 .chain(second_batch.iter())
4264 .copied()
4265 .collect();
4266
4267 // Check if we saw [1, 3, 4, 2] — the delayed interleaving
4268 if all == [1, 3, 4, 2] {
4269 saw_delayed_interleaving = true;
4270 }
4271
4272 all.sort();
4273 assert_eq!(all, vec![1, 2, 3, 4]);
4274 });
4275
4276 assert!(saw_delayed_interleaving);
4277 }
4278
4279 /// Deploy test: merge_ordered with a delayed element on one input.
4280 /// Sends a=1, b=3, b=4, then after receiving those, sends a=2.
4281 /// Expects to see [1, 3, 4] first, then [2] — demonstrating that
4282 /// both inputs are pulled and the delayed element arrives later.
4283 #[cfg(feature = "deploy")]
4284 #[tokio::test]
4285 async fn deploy_merge_ordered_delayed() {
4286 let mut deployment = Deployment::new();
4287
4288 let mut flow = FlowBuilder::new();
4289 let node = flow.process::<()>();
4290 let external = flow.external::<()>();
4291
4292 let (input_a_port, input_a) = node.source_external_bincode(&external);
4293 let (input_b_port, input_b) = node.source_external_bincode(&external);
4294
4295 let out = input_a
4296 .assume_ordering(nondet!(/** test */))
4297 .merge_ordered(
4298 input_b.assume_ordering(nondet!(/** test */)),
4299 nondet!(/** test */),
4300 )
4301 .send_bincode_external(&external);
4302
4303 let nodes = flow
4304 .with_process(&node, deployment.Localhost())
4305 .with_external(&external, deployment.Localhost())
4306 .deploy(&mut deployment);
4307
4308 deployment.deploy().await.unwrap();
4309
4310 let mut ext_a = nodes.connect(input_a_port).await;
4311 let mut ext_b = nodes.connect(input_b_port).await;
4312 let mut ext_out = nodes.connect(out).await;
4313
4314 deployment.start().await.unwrap();
4315
4316 // Send a=1, b=3, b=4
4317 ext_a.send(1).await.unwrap();
4318 ext_b.send(3).await.unwrap();
4319 ext_b.send(4).await.unwrap();
4320
4321 // Collect the first 3 elements
4322 let mut received = Vec::new();
4323 for _ in 0..3 {
4324 received.push(ext_out.next().await.unwrap());
4325 }
4326
4327 // Now send the delayed a=2
4328 ext_a.send(2).await.unwrap();
4329 received.push(ext_out.next().await.unwrap());
4330
4331 // All elements should be present
4332 received.sort();
4333 assert_eq!(received, vec![1, 2, 3, 4]);
4334 }
4335
4336 #[cfg(feature = "deploy")]
4337 #[tokio::test]
4338 async fn monotone_fold_threshold() {
4339 use crate::properties::manual_proof;
4340
4341 let mut deployment = Deployment::new();
4342
4343 let mut flow = FlowBuilder::new();
4344 let node = flow.process::<()>();
4345 let external = flow.external::<()>();
4346
4347 let in_unbounded: super::Stream<_, _> =
4348 node.source_iter(q!(vec![1i32, 2, 3, 4, 5, 6])).into();
4349 let sum = in_unbounded.fold(
4350 q!(|| 0),
4351 q!(
4352 |sum, v| {
4353 *sum += v;
4354 },
4355 monotone = manual_proof!(/** test */)
4356 ),
4357 );
4358
4359 let threshold_out = sum
4360 .threshold_greater_or_equal(node.singleton(q!(7)))
4361 .send_bincode_external(&external);
4362
4363 let nodes = flow
4364 .with_process(&node, deployment.Localhost())
4365 .with_external(&external, deployment.Localhost())
4366 .deploy(&mut deployment);
4367
4368 deployment.deploy().await.unwrap();
4369
4370 let mut threshold_out = nodes.connect(threshold_out).await;
4371
4372 deployment.start().await.unwrap();
4373
4374 assert_eq!(threshold_out.next().await.unwrap(), 7);
4375 }
4376
4377 #[cfg(feature = "deploy")]
4378 #[tokio::test]
4379 async fn monotone_count_threshold() {
4380 let mut deployment = Deployment::new();
4381
4382 let mut flow = FlowBuilder::new();
4383 let node = flow.process::<()>();
4384 let external = flow.external::<()>();
4385
4386 let in_unbounded: super::Stream<_, _> =
4387 node.source_iter(q!(vec![1i32, 2, 3, 4, 5, 6])).into();
4388 let sum = in_unbounded.count();
4389
4390 let threshold_out = sum
4391 .threshold_greater_or_equal(node.singleton(q!(3)))
4392 .send_bincode_external(&external);
4393
4394 let nodes = flow
4395 .with_process(&node, deployment.Localhost())
4396 .with_external(&external, deployment.Localhost())
4397 .deploy(&mut deployment);
4398
4399 deployment.deploy().await.unwrap();
4400
4401 let mut threshold_out = nodes.connect(threshold_out).await;
4402
4403 deployment.start().await.unwrap();
4404
4405 assert_eq!(threshold_out.next().await.unwrap(), 3);
4406 }
4407
4408 #[cfg(feature = "deploy")]
4409 #[tokio::test]
4410 async fn monotone_map_order_preserving_threshold() {
4411 use crate::properties::manual_proof;
4412
4413 let mut deployment = Deployment::new();
4414
4415 let mut flow = FlowBuilder::new();
4416 let node = flow.process::<()>();
4417 let external = flow.external::<()>();
4418
4419 let in_unbounded: super::Stream<_, _> =
4420 node.source_iter(q!(vec![1i32, 2, 3, 4, 5, 6])).into();
4421 let sum = in_unbounded.fold(
4422 q!(|| 0),
4423 q!(
4424 |sum, v| {
4425 *sum += v;
4426 },
4427 monotone = manual_proof!(/** test */)
4428 ),
4429 );
4430
4431 // map with order_preserving should preserve monotonicity
4432 let doubled = sum.map(q!(
4433 |v| v * 2,
4434 order_preserving = manual_proof!(/** doubling preserves order */)
4435 ));
4436
4437 let threshold_out = doubled
4438 .threshold_greater_or_equal(node.singleton(q!(14)))
4439 .send_bincode_external(&external);
4440
4441 let nodes = flow
4442 .with_process(&node, deployment.Localhost())
4443 .with_external(&external, deployment.Localhost())
4444 .deploy(&mut deployment);
4445
4446 deployment.deploy().await.unwrap();
4447
4448 let mut threshold_out = nodes.connect(threshold_out).await;
4449
4450 deployment.start().await.unwrap();
4451
4452 assert_eq!(threshold_out.next().await.unwrap(), 14);
4453 }
4454
4455 // === Compile-time type tests for join/cross_product ordering ===
4456
4457 #[cfg(any(feature = "deploy", feature = "sim"))]
4458 mod join_ordering_type_tests {
4459 use crate::live_collections::boundedness::{Bounded, Unbounded};
4460 use crate::live_collections::stream::{ExactlyOnce, NoOrder, Stream, TotalOrder};
4461 use crate::location::{Location, Process};
4462
4463 #[expect(dead_code, reason = "compile-time type test")]
4464 fn join_unbounded_with_bounded_preserves_order<'a>(
4465 left: Stream<(i32, char), Process<'a>, Unbounded, TotalOrder, ExactlyOnce>,
4466 right: Stream<(i32, char), Process<'a>, Bounded, TotalOrder, ExactlyOnce>,
4467 ) -> Stream<(i32, (char, char)), Process<'a>, Unbounded, TotalOrder, ExactlyOnce> {
4468 left.join(right)
4469 }
4470
4471 #[expect(dead_code, reason = "compile-time type test")]
4472 fn join_unbounded_with_unbounded_is_no_order<'a>(
4473 left: Stream<(i32, char), Process<'a>, Unbounded, TotalOrder, ExactlyOnce>,
4474 right: Stream<(i32, char), Process<'a>, Unbounded, TotalOrder, ExactlyOnce>,
4475 ) -> Stream<(i32, (char, char)), Process<'a>, Unbounded, NoOrder, ExactlyOnce> {
4476 left.join(right)
4477 }
4478
4479 #[expect(dead_code, reason = "compile-time type test")]
4480 fn join_bounded_with_bounded_preserves_order<'a, L: Location<'a>>(
4481 left: Stream<(i32, char), L, Bounded, TotalOrder, ExactlyOnce>,
4482 right: Stream<(i32, char), L, Bounded, TotalOrder, ExactlyOnce>,
4483 ) -> Stream<(i32, (char, char)), L, Bounded, TotalOrder, ExactlyOnce> {
4484 left.join(right)
4485 }
4486
4487 #[expect(dead_code, reason = "compile-time type test")]
4488 fn join_unbounded_noorder_with_bounded<'a>(
4489 left: Stream<(i32, char), Process<'a>, Unbounded, NoOrder, ExactlyOnce>,
4490 right: Stream<(i32, char), Process<'a>, Bounded, NoOrder, ExactlyOnce>,
4491 ) -> Stream<(i32, (char, char)), Process<'a>, Unbounded, NoOrder, ExactlyOnce> {
4492 left.join(right)
4493 }
4494
4495 // === Compile-time type tests for cross_product ordering ===
4496
4497 #[expect(dead_code, reason = "compile-time type test")]
4498 fn cross_product_unbounded_with_bounded_preserves_order<'a>(
4499 left: Stream<i32, Process<'a>, Unbounded, TotalOrder, ExactlyOnce>,
4500 right: Stream<char, Process<'a>, Bounded, TotalOrder, ExactlyOnce>,
4501 ) -> Stream<(i32, char), Process<'a>, Unbounded, TotalOrder, ExactlyOnce> {
4502 left.cross_product(right)
4503 }
4504
4505 #[expect(dead_code, reason = "compile-time type test")]
4506 fn cross_product_bounded_with_bounded_preserves_order<'a>(
4507 left: Stream<i32, Process<'a>, Bounded, TotalOrder, ExactlyOnce>,
4508 right: Stream<char, Process<'a>, Bounded, TotalOrder, ExactlyOnce>,
4509 ) -> Stream<(i32, char), Process<'a>, Bounded, TotalOrder, ExactlyOnce> {
4510 left.cross_product(right)
4511 }
4512
4513 #[expect(dead_code, reason = "compile-time type test")]
4514 fn cross_product_unbounded_with_unbounded_is_no_order<'a>(
4515 left: Stream<i32, Process<'a>, Unbounded, TotalOrder, ExactlyOnce>,
4516 right: Stream<char, Process<'a>, Unbounded, TotalOrder, ExactlyOnce>,
4517 ) -> Stream<(i32, char), Process<'a>, Unbounded, NoOrder, ExactlyOnce> {
4518 left.cross_product(right)
4519 }
4520 } // mod join_ordering_type_tests
4521
4522 // === Runtime correctness tests for bounded join/cross_product ===
4523
4524 #[cfg(feature = "sim")]
4525 #[test]
4526 fn cross_product_mixed_boundedness_correctness() {
4527 use stageleft::q;
4528
4529 use crate::compile::builder::FlowBuilder;
4530 use crate::nondet::nondet;
4531
4532 let mut flow = FlowBuilder::new();
4533 let process = flow.process::<()>();
4534 let tick = process.tick();
4535
4536 let left = process.source_iter(q!(vec![1, 2]));
4537 let right = process
4538 .source_iter(q!(vec!['a', 'b']))
4539 .batch(&tick, nondet!(/** test */))
4540 .all_ticks();
4541
4542 let out = left.cross_product(right).sim_output();
4543
4544 flow.sim().exhaustive(async || {
4545 out.assert_yields_only_unordered(vec![(1, 'a'), (1, 'b'), (2, 'a'), (2, 'b')])
4546 .await;
4547 });
4548 }
4549
4550 #[cfg(feature = "sim")]
4551 #[test]
4552 fn join_mixed_boundedness_correctness() {
4553 use stageleft::q;
4554
4555 use crate::compile::builder::FlowBuilder;
4556 use crate::nondet::nondet;
4557
4558 let mut flow = FlowBuilder::new();
4559 let process = flow.process::<()>();
4560 let tick = process.tick();
4561
4562 let left = process.source_iter(q!(vec![(1, 'a'), (2, 'b')]));
4563 let right = process
4564 .source_iter(q!(vec![(1, 'x'), (2, 'y')]))
4565 .batch(&tick, nondet!(/** test */))
4566 .all_ticks();
4567
4568 let out = left.join(right).sim_output();
4569
4570 flow.sim().exhaustive(async || {
4571 out.assert_yields_only_unordered(vec![(1, ('a', 'x')), (2, ('b', 'y'))])
4572 .await;
4573 });
4574 }
4575
4576 #[cfg(feature = "sim")]
4577 #[test]
4578 fn sim_merge_unordered_independent_atomics() {
4579 let mut flow = FlowBuilder::new();
4580 let node = flow.process::<()>();
4581
4582 let (in1_send, input1) = node.sim_input::<_, TotalOrder, _>();
4583 let (in2_send, input2) = node.sim_input::<_, TotalOrder, _>();
4584
4585 let out = input1
4586 .atomic()
4587 .merge_unordered(input2.atomic())
4588 .end_atomic()
4589 .sim_output();
4590
4591 flow.sim().exhaustive(async || {
4592 in1_send.send(1);
4593 in2_send.send(2);
4594
4595 out.assert_yields_only_unordered(vec![1, 2]).await;
4596 });
4597 }
4598}