hydro_lang 0.16.0

A Rust framework for correct and performant distributed systems
Documentation
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549
1550
1551
1552
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
//! Type definitions for distributed locations, which specify where pieces of a Hydro
//! program will be executed.
//!
//! Hydro is a **global**, **distributed** programming model. This means that the data
//! and computation in a Hydro program can be spread across multiple machines, data
//! centers, and even continents. To achieve this, Hydro uses the concept of
//! **locations** to keep track of _where_ data is located and computation is executed.
//!
//! Each live collection type (in [`crate::live_collections`]) has a type parameter `L`
//! which will always be a type that implements the [`Location`] trait (e.g. [`Process`]
//! and [`Cluster`]). To create distributed programs, Hydro provides a variety of APIs
//! to allow live collections to be _moved_ between locations via network send/receive.
//!
//! See [the Hydro docs](https://hydro.run/docs/hydro/reference/locations/) for more information.

use std::fmt::Debug;
use std::future::Future;
use std::marker::PhantomData;
use std::num::ParseIntError;
use std::time::Duration;

use bytes::{Bytes, BytesMut};
use futures::stream::Stream as FuturesStream;
use proc_macro2::Span;
use quote::quote;
use serde::de::DeserializeOwned;
use serde::{Deserialize, Serialize};
use slotmap::{Key, new_key_type};
use stageleft::runtime_support::{FreeVariableWithContextWithProps, QuoteTokens};
use stageleft::{QuotedWithContext, q, quote_type};
use syn::parse_quote;
use tokio_util::codec::{Decoder, Encoder, LengthDelimitedCodec};

use crate::compile::ir::{
    ClusterMembersState, DebugInstantiate, HydroIrOpMetadata, HydroNode, HydroRoot, HydroSource,
};
use crate::forward_handle::ForwardRef;
#[cfg(stageleft_runtime)]
use crate::forward_handle::{CycleCollection, ForwardHandle};
use crate::live_collections::boundedness::{Bounded, Unbounded};
use crate::live_collections::keyed_stream::KeyedStream;
use crate::live_collections::singleton::Singleton;
use crate::live_collections::stream::{
    ExactlyOnce, NoOrder, Ordering, Retries, Stream, TotalOrder,
};
use crate::location::dynamic::LocationId;
use crate::location::external_process::{
    ExternalBincodeBidi, ExternalBincodeSink, ExternalBytesPort, Many, NotMany,
};
use crate::nondet::NonDet;
#[cfg(feature = "sim")]
use crate::sim::SimSender;
use crate::staging_util::get_this_crate;

pub mod dynamic;

pub mod external_process;
pub use external_process::External;

pub mod process;
pub use process::Process;

pub mod cluster;
pub use cluster::Cluster;

pub mod member_id;
pub use member_id::{MemberId, TaglessMemberId};

pub mod tick;
pub use tick::{Atomic, NoTick, Tick};

/// An event indicating a change in membership status of a location in a group
/// (e.g. a node in a [`Cluster`] or an external client connection).
#[derive(PartialEq, Eq, Clone, Debug, Hash, Serialize, Deserialize)]
pub enum MembershipEvent {
    /// The member has joined the group and is now active.
    Joined,
    /// The member has left the group and is no longer active.
    Left,
}

/// A hint for configuring the network transport used by an external connection.
///
/// This controls how the underlying TCP listener is set up when binding
/// external client connections via methods like [`Location::bind_single_client`]
/// or [`Location::bidi_external_many_bytes`].
#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)]
pub enum NetworkHint {
    /// Automatically select the network configuration (e.g. an ephemeral port).
    Auto,
    /// Use a TCP port, optionally specifying a fixed port number.
    ///
    /// If `None`, an available port will be chosen automatically.
    /// If `Some(port)`, the given port number will be used.
    TcpPort(Option<u16>),
}

pub(crate) fn check_matching_location<'a, L: Location<'a>>(l1: &L, l2: &L) {
    assert_eq!(Location::id(l1), Location::id(l2), "locations do not match");
}

#[stageleft::export(LocationKey)]
new_key_type! {
    /// A unique identifier for a clock tick.
    pub struct LocationKey;
}

impl std::fmt::Display for LocationKey {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        write!(f, "loc{:?}", self.data()) // `"loc1v1"``
    }
}

/// This is used for the ECS membership stream.
/// TODO(mingwei): Make this more robust?
impl std::str::FromStr for LocationKey {
    type Err = Option<ParseIntError>;

    fn from_str(s: &str) -> Result<Self, Self::Err> {
        let nvn = s.strip_prefix("loc").ok_or(None)?;
        let (idx, ver) = nvn.split_once("v").ok_or(None)?;
        let idx: u64 = idx.parse()?;
        let ver: u64 = ver.parse()?;
        Ok(slotmap::KeyData::from_ffi((ver << 32) | idx).into())
    }
}

impl LocationKey {
    /// TODO(minwgei): Remove this and avoid magic key for simulator external.
    /// The first location key, used by the simulator as the default external location.
    pub const FIRST: Self = Self(slotmap::KeyData::from_ffi(0x0000000100000001)); // `1v1`

    /// A key for testing with index 1.
    #[cfg(test)]
    pub const TEST_KEY_1: Self = Self(slotmap::KeyData::from_ffi(0x000000FF00000001)); // `1v255`

    /// A key for testing with index 2.
    #[cfg(test)]
    pub const TEST_KEY_2: Self = Self(slotmap::KeyData::from_ffi(0x000000FF00000002)); // `2v255`
}

/// This is used within `q!` code in docker and ECS.
impl<Ctx> FreeVariableWithContextWithProps<Ctx, ()> for LocationKey {
    type O = LocationKey;

    fn to_tokens(self, _ctx: &Ctx) -> (QuoteTokens, ())
    where
        Self: Sized,
    {
        let root = get_this_crate();
        let n = Key::data(&self).as_ffi();
        (
            QuoteTokens {
                prelude: None,
                expr: Some(quote! {
                    #root::location::LocationKey::from(#root::runtime_support::slotmap::KeyData::from_ffi(#n))
                }),
            },
            (),
        )
    }
}

/// A simple enum for the type of a root location.
#[derive(Clone, Copy, Debug, Eq, Hash, PartialEq, Serialize)]
pub enum LocationType {
    /// A process (single node).
    Process,
    /// A cluster (multiple nodes).
    Cluster,
    /// An external client.
    External,
}

/// A location where data can be materialized and computation can be executed.
///
/// Hydro is a **global**, **distributed** programming model. This means that the data
/// and computation in a Hydro program can be spread across multiple machines, data
/// centers, and even continents. To achieve this, Hydro uses the concept of
/// **locations** to keep track of _where_ data is located and computation is executed.
///
/// Each live collection type (in [`crate::live_collections`]) has a type parameter `L`
/// which will always be a type that implements the [`Location`] trait (e.g. [`Process`]
/// and [`Cluster`]). To create distributed programs, Hydro provides a variety of APIs
/// to allow live collections to be _moved_ between locations via network send/receive.
///
/// See [the Hydro docs](https://hydro.run/docs/hydro/reference/locations/) for more information.
#[expect(
    private_bounds,
    reason = "only internal Hydro code can define location types"
)]
pub trait Location<'a>: dynamic::DynLocation {
    /// The root location type for this location.
    ///
    /// For top-level locations like [`Process`] and [`Cluster`], this is `Self`.
    /// For nested locations like [`Tick`], this is the root location that contains it.
    type Root: Location<'a>;

    /// Returns the root location for this location.
    ///
    /// For top-level locations like [`Process`] and [`Cluster`], this returns `self`.
    /// For nested locations like [`Tick`], this returns the root location that contains it.
    fn root(&self) -> Self::Root;

    /// Attempts to create a new [`Tick`] clock domain at this location.
    ///
    /// Returns `Some(Tick)` if this is a top-level location (like [`Process`] or [`Cluster`]),
    /// or `None` if this location is already inside a tick (nested ticks are not supported).
    ///
    /// Prefer using [`Location::tick`] when you know the location is top-level.
    fn try_tick(&self) -> Option<Tick<Self>> {
        if Self::is_top_level() {
            let id = self.flow_state().borrow_mut().next_clock_id();
            Some(Tick {
                id,
                l: self.clone(),
            })
        } else {
            None
        }
    }

    /// Returns the unique identifier for this location.
    fn id(&self) -> LocationId {
        dynamic::DynLocation::id(self)
    }

    /// Creates a new [`Tick`] clock domain at this location.
    ///
    /// A tick represents a logical clock that can be used to batch streaming data
    /// into discrete time steps. This is useful for implementing iterative algorithms
    /// or for synchronizing data across multiple streams.
    ///
    /// # Example
    /// ```rust
    /// # #[cfg(feature = "deploy")] {
    /// # use hydro_lang::prelude::*;
    /// # use futures::StreamExt;
    /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
    /// let tick = process.tick();
    /// let inside_tick = process
    ///     .source_iter(q!(vec![1, 2, 3, 4]))
    ///     .batch(&tick, nondet!(/** test */));
    /// inside_tick.all_ticks()
    /// # }, |mut stream| async move {
    /// // 1, 2, 3, 4
    /// # for w in vec![1, 2, 3, 4] {
    /// #     assert_eq!(stream.next().await.unwrap(), w);
    /// # }
    /// # }));
    /// # }
    /// ```
    fn tick(&self) -> Tick<Self>
    where
        Self: NoTick,
    {
        let id = self.flow_state().borrow_mut().next_clock_id();
        Tick {
            id,
            l: self.clone(),
        }
    }

    /// Creates an unbounded stream that continuously emits unit values `()`.
    ///
    /// This is useful for driving computations that need to run continuously,
    /// such as polling or heartbeat mechanisms.
    ///
    /// # Example
    /// ```rust
    /// # #[cfg(feature = "deploy")] {
    /// # use hydro_lang::prelude::*;
    /// # use futures::StreamExt;
    /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
    /// let tick = process.tick();
    /// process.spin()
    ///     .batch(&tick, nondet!(/** test */))
    ///     .map(q!(|_| 42))
    ///     .all_ticks()
    /// # }, |mut stream| async move {
    /// // 42, 42, 42, ...
    /// # assert_eq!(stream.next().await.unwrap(), 42);
    /// # assert_eq!(stream.next().await.unwrap(), 42);
    /// # assert_eq!(stream.next().await.unwrap(), 42);
    /// # }));
    /// # }
    /// ```
    fn spin(&self) -> Stream<(), Self, Unbounded, TotalOrder, ExactlyOnce>
    where
        Self: Sized + NoTick,
    {
        Stream::new(
            self.clone(),
            HydroNode::Source {
                source: HydroSource::Spin(),
                metadata: self.new_node_metadata(Stream::<
                    (),
                    Self,
                    Unbounded,
                    TotalOrder,
                    ExactlyOnce,
                >::collection_kind()),
            },
        )
    }

    /// Creates a stream from an async [`FuturesStream`].
    ///
    /// This is useful for integrating with external async data sources,
    /// such as network connections or file readers.
    ///
    /// # Example
    /// ```rust
    /// # #[cfg(feature = "deploy")] {
    /// # use hydro_lang::prelude::*;
    /// # use futures::StreamExt;
    /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
    /// process.source_stream(q!(futures::stream::iter(vec![1, 2, 3])))
    /// # }, |mut stream| async move {
    /// // 1, 2, 3
    /// # for w in vec![1, 2, 3] {
    /// #     assert_eq!(stream.next().await.unwrap(), w);
    /// # }
    /// # }));
    /// # }
    /// ```
    fn source_stream<T, E>(
        &self,
        e: impl QuotedWithContext<'a, E, Self>,
    ) -> Stream<T, Self, Unbounded, TotalOrder, ExactlyOnce>
    where
        E: FuturesStream<Item = T> + Unpin,
        Self: Sized + NoTick,
    {
        let e = e.splice_untyped_ctx(self);

        Stream::new(
            self.clone(),
            HydroNode::Source {
                source: HydroSource::Stream(e.into()),
                metadata: self.new_node_metadata(Stream::<
                    T,
                    Self,
                    Unbounded,
                    TotalOrder,
                    ExactlyOnce,
                >::collection_kind()),
            },
        )
    }

    /// Creates a bounded stream from an iterator.
    ///
    /// The iterator is evaluated once at runtime, and all elements are emitted
    /// in order. This is useful for creating streams from static data or
    /// for testing.
    ///
    /// # Example
    /// ```rust
    /// # #[cfg(feature = "deploy")] {
    /// # use hydro_lang::prelude::*;
    /// # use futures::StreamExt;
    /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
    /// process.source_iter(q!(vec![1, 2, 3, 4]))
    /// # }, |mut stream| async move {
    /// // 1, 2, 3, 4
    /// # for w in vec![1, 2, 3, 4] {
    /// #     assert_eq!(stream.next().await.unwrap(), w);
    /// # }
    /// # }));
    /// # }
    /// ```
    fn source_iter<T, E>(
        &self,
        e: impl QuotedWithContext<'a, E, Self>,
    ) -> Stream<T, Self, Bounded, TotalOrder, ExactlyOnce>
    where
        E: IntoIterator<Item = T>,
        Self: Sized,
    {
        let e = e.splice_typed_ctx(self);

        Stream::new(
            self.clone(),
            HydroNode::Source {
                source: HydroSource::Iter(e.into()),
                metadata: self.new_node_metadata(
                    Stream::<T, Self, Bounded, TotalOrder, ExactlyOnce>::collection_kind(),
                ),
            },
        )
    }

    /// Creates a stream of membership events for a cluster.
    ///
    /// This stream emits [`MembershipEvent::Joined`] when a cluster member joins
    /// and [`MembershipEvent::Left`] when a cluster member leaves. The stream is
    /// keyed by the [`MemberId`] of the cluster member.
    ///
    /// This is useful for implementing protocols that need to track cluster membership,
    /// such as broadcasting to all members or detecting failures.
    ///
    /// # Example
    /// ```rust
    /// # #[cfg(feature = "deploy")] {
    /// # use hydro_lang::prelude::*;
    /// # use futures::StreamExt;
    /// # tokio_test::block_on(hydro_lang::test_util::multi_location_test(|flow, p2| {
    /// let p1 = flow.process::<()>();
    /// let workers: Cluster<()> = flow.cluster::<()>();
    /// # // do nothing on each worker
    /// # workers.source_iter(q!(vec![])).for_each(q!(|_: ()| {}));
    /// let cluster_members = p1.source_cluster_members(&workers);
    /// # cluster_members.entries().send(&p2, TCP.fail_stop().bincode())
    /// // if there are 4 members in the cluster, we would see a join event for each
    /// // { MemberId::<Worker>(0): [MembershipEvent::Join], MemberId::<Worker>(2): [MembershipEvent::Join], ... }
    /// # }, |mut stream| async move {
    /// # let mut results = Vec::new();
    /// # for w in 0..4 {
    /// #     results.push(format!("{:?}", stream.next().await.unwrap()));
    /// # }
    /// # results.sort();
    /// # assert_eq!(results, vec!["(MemberId::<()>(0), Joined)", "(MemberId::<()>(1), Joined)", "(MemberId::<()>(2), Joined)", "(MemberId::<()>(3), Joined)"]);
    /// # }));
    /// # }
    /// ```
    fn source_cluster_members<C: 'a>(
        &self,
        cluster: &Cluster<'a, C>,
    ) -> KeyedStream<MemberId<C>, MembershipEvent, Self, Unbounded>
    where
        Self: Sized + NoTick,
    {
        Stream::new(
            self.clone(),
            HydroNode::Source {
                source: HydroSource::ClusterMembers(cluster.id(), ClusterMembersState::Uninit),
                metadata: self.new_node_metadata(Stream::<
                    (TaglessMemberId, MembershipEvent),
                    Self,
                    Unbounded,
                    TotalOrder,
                    ExactlyOnce,
                >::collection_kind()),
            },
        )
        .map(q!(|(k, v)| (MemberId::from_tagless(k), v)))
        .into_keyed()
    }

    /// Creates a one-way connection from an external process to receive raw bytes.
    ///
    /// Returns a port handle for the external process to connect to, and a stream
    /// of received byte buffers.
    ///
    /// For bidirectional communication or typed data, see [`Location::bind_single_client`]
    /// or [`Location::source_external_bincode`].
    fn source_external_bytes<L>(
        &self,
        from: &External<L>,
    ) -> (
        ExternalBytesPort,
        Stream<BytesMut, Self, Unbounded, TotalOrder, ExactlyOnce>,
    )
    where
        Self: Sized + NoTick,
    {
        let (port, stream, sink) =
            self.bind_single_client::<_, Bytes, LengthDelimitedCodec>(from, NetworkHint::Auto);

        sink.complete(self.source_iter(q!([])));

        (port, stream)
    }

    /// Creates a one-way connection from an external process to receive bincode-serialized data.
    ///
    /// Returns a sink handle for the external process to send data to, and a stream
    /// of received values.
    ///
    /// For bidirectional communication, see [`Location::bind_single_client_bincode`].
    #[expect(clippy::type_complexity, reason = "stream markers")]
    fn source_external_bincode<L, T, O: Ordering, R: Retries>(
        &self,
        from: &External<L>,
    ) -> (
        ExternalBincodeSink<T, NotMany, O, R>,
        Stream<T, Self, Unbounded, O, R>,
    )
    where
        Self: Sized + NoTick,
        T: Serialize + DeserializeOwned,
    {
        let (port, stream, sink) = self.bind_single_client_bincode::<_, T, ()>(from);
        sink.complete(self.source_iter(q!([])));

        (
            ExternalBincodeSink {
                process_key: from.key,
                port_id: port.port_id,
                _phantom: PhantomData,
            },
            stream.weaken_ordering().weaken_retries(),
        )
    }

    /// Sets up a simulated input port on this location for testing.
    ///
    /// Returns a handle to send messages to the location as well as a stream
    /// of received messages. This is only available when the `sim` feature is enabled.
    #[cfg(feature = "sim")]
    #[expect(clippy::type_complexity, reason = "stream markers")]
    fn sim_input<T, O: Ordering, R: Retries>(
        &self,
    ) -> (SimSender<T, O, R>, Stream<T, Self, Unbounded, O, R>)
    where
        Self: Sized + NoTick,
        T: Serialize + DeserializeOwned,
    {
        let external_location: External<'a, ()> = External {
            key: LocationKey::FIRST,
            flow_state: self.flow_state().clone(),
            _phantom: PhantomData,
        };

        let (external, stream) = self.source_external_bincode(&external_location);

        (SimSender(external.port_id, PhantomData), stream)
    }

    /// Creates an external input stream for embedded deployment mode.
    ///
    /// The `name` parameter specifies the name of the generated function parameter
    /// that will supply data to this stream at runtime. The generated function will
    /// accept an `impl Stream<Item = T> + Unpin` argument with this name.
    fn embedded_input<T>(
        &self,
        name: impl Into<String>,
    ) -> Stream<T, Self, Unbounded, TotalOrder, ExactlyOnce>
    where
        Self: Sized + NoTick,
    {
        let ident = syn::Ident::new(&name.into(), Span::call_site());

        Stream::new(
            self.clone(),
            HydroNode::Source {
                source: HydroSource::Embedded(ident),
                metadata: self.new_node_metadata(Stream::<
                    T,
                    Self,
                    Unbounded,
                    TotalOrder,
                    ExactlyOnce,
                >::collection_kind()),
            },
        )
    }

    /// Creates an embedded singleton input for embedded deployment mode.
    ///
    /// The `name` parameter specifies the name of the generated function parameter
    /// that will supply data to this singleton at runtime. The generated function will
    /// accept a plain `T` parameter with this name.
    fn embedded_singleton_input<T>(&self, name: impl Into<String>) -> Singleton<T, Self, Bounded>
    where
        Self: Sized + NoTick,
    {
        let ident = syn::Ident::new(&name.into(), Span::call_site());

        Singleton::new(
            self.clone(),
            HydroNode::Source {
                source: HydroSource::EmbeddedSingleton(ident),
                metadata: self.new_node_metadata(Singleton::<T, Self, Bounded>::collection_kind()),
            },
        )
    }

    /// Establishes a server on this location to receive a bidirectional connection from a single
    /// client, identified by the given `External` handle. Returns a port handle for the external
    /// process to connect to, a stream of incoming messages, and a handle to send outgoing
    /// messages.
    ///
    /// # Example
    /// ```rust
    /// # #[cfg(feature = "deploy")] {
    /// # use hydro_lang::prelude::*;
    /// # use hydro_deploy::Deployment;
    /// # use futures::{SinkExt, StreamExt};
    /// # tokio_test::block_on(async {
    /// # use bytes::Bytes;
    /// # use hydro_lang::location::NetworkHint;
    /// # use tokio_util::codec::LengthDelimitedCodec;
    /// # let mut flow = FlowBuilder::new();
    /// let node = flow.process::<()>();
    /// let external = flow.external::<()>();
    /// let (port, incoming, outgoing) =
    ///     node.bind_single_client::<_, Bytes, LengthDelimitedCodec>(&external, NetworkHint::Auto);
    /// outgoing.complete(incoming.map(q!(|data /* : Bytes */| {
    ///     let mut resp: Vec<u8> = data.into();
    ///     resp.push(42);
    ///     resp.into() // : Bytes
    /// })));
    ///
    /// # let mut deployment = Deployment::new();
    /// let nodes = flow // ... with_process and with_external
    /// #     .with_process(&node, deployment.Localhost())
    /// #     .with_external(&external, deployment.Localhost())
    /// #     .deploy(&mut deployment);
    ///
    /// deployment.deploy().await.unwrap();
    /// deployment.start().await.unwrap();
    ///
    /// let (mut external_out, mut external_in) = nodes.connect(port).await;
    /// external_in.send(vec![1, 2, 3].into()).await.unwrap();
    /// assert_eq!(
    ///     external_out.next().await.unwrap().unwrap(),
    ///     vec![1, 2, 3, 42]
    /// );
    /// # });
    /// # }
    /// ```
    #[expect(clippy::type_complexity, reason = "stream markers")]
    fn bind_single_client<L, T, Codec: Encoder<T> + Decoder>(
        &self,
        from: &External<L>,
        port_hint: NetworkHint,
    ) -> (
        ExternalBytesPort<NotMany>,
        Stream<<Codec as Decoder>::Item, Self, Unbounded, TotalOrder, ExactlyOnce>,
        ForwardHandle<'a, Stream<T, Self, Unbounded, TotalOrder, ExactlyOnce>>,
    )
    where
        Self: Sized + NoTick,
    {
        let next_external_port_id = from.flow_state.borrow_mut().next_external_port();

        let (fwd_ref, to_sink) =
            self.forward_ref::<Stream<T, Self, Unbounded, TotalOrder, ExactlyOnce>>();
        let mut flow_state_borrow = self.flow_state().borrow_mut();

        flow_state_borrow.push_root(HydroRoot::SendExternal {
            to_external_key: from.key,
            to_port_id: next_external_port_id,
            to_many: false,
            unpaired: false,
            serialize_fn: None,
            instantiate_fn: DebugInstantiate::Building,
            input: Box::new(to_sink.ir_node.replace(HydroNode::Placeholder)),
            op_metadata: HydroIrOpMetadata::new(),
        });

        let raw_stream: Stream<
            Result<<Codec as Decoder>::Item, <Codec as Decoder>::Error>,
            Self,
            Unbounded,
            TotalOrder,
            ExactlyOnce,
        > = Stream::new(
            self.clone(),
            HydroNode::ExternalInput {
                from_external_key: from.key,
                from_port_id: next_external_port_id,
                from_many: false,
                codec_type: quote_type::<Codec>().into(),
                port_hint,
                instantiate_fn: DebugInstantiate::Building,
                deserialize_fn: None,
                metadata: self.new_node_metadata(Stream::<
                    Result<<Codec as Decoder>::Item, <Codec as Decoder>::Error>,
                    Self,
                    Unbounded,
                    TotalOrder,
                    ExactlyOnce,
                >::collection_kind()),
            },
        );

        (
            ExternalBytesPort {
                process_key: from.key,
                port_id: next_external_port_id,
                _phantom: PhantomData,
            },
            raw_stream.flatten_ordered(),
            fwd_ref,
        )
    }

    /// Establishes a bidirectional connection from a single external client using bincode serialization.
    ///
    /// Returns a port handle for the external process to connect to, a stream of incoming messages,
    /// and a handle to send outgoing messages. This is a convenience wrapper around
    /// [`Location::bind_single_client`] that uses bincode for serialization.
    ///
    /// # Type Parameters
    /// - `InT`: The type of incoming messages (must implement [`DeserializeOwned`])
    /// - `OutT`: The type of outgoing messages (must implement [`Serialize`])
    #[expect(clippy::type_complexity, reason = "stream markers")]
    fn bind_single_client_bincode<L, InT: DeserializeOwned, OutT: Serialize>(
        &self,
        from: &External<L>,
    ) -> (
        ExternalBincodeBidi<InT, OutT, NotMany>,
        Stream<InT, Self, Unbounded, TotalOrder, ExactlyOnce>,
        ForwardHandle<'a, Stream<OutT, Self, Unbounded, TotalOrder, ExactlyOnce>>,
    )
    where
        Self: Sized + NoTick,
    {
        let next_external_port_id = from.flow_state.borrow_mut().next_external_port();

        let (fwd_ref, to_sink) =
            self.forward_ref::<Stream<OutT, Self, Unbounded, TotalOrder, ExactlyOnce>>();
        let mut flow_state_borrow = self.flow_state().borrow_mut();

        let root = get_this_crate();

        let out_t_type = quote_type::<OutT>();
        let ser_fn: syn::Expr = syn::parse_quote! {
            #root::runtime_support::stageleft::runtime_support::fn1_type_hint::<#out_t_type, _>(
                |b| #root::runtime_support::bincode::serialize(&b).unwrap().into()
            )
        };

        flow_state_borrow.push_root(HydroRoot::SendExternal {
            to_external_key: from.key,
            to_port_id: next_external_port_id,
            to_many: false,
            unpaired: false,
            serialize_fn: Some(ser_fn.into()),
            instantiate_fn: DebugInstantiate::Building,
            input: Box::new(to_sink.ir_node.replace(HydroNode::Placeholder)),
            op_metadata: HydroIrOpMetadata::new(),
        });

        let in_t_type = quote_type::<InT>();

        let deser_fn: syn::Expr = syn::parse_quote! {
            |res| {
                let b = res.unwrap();
                #root::runtime_support::bincode::deserialize::<#in_t_type>(&b).unwrap()
            }
        };

        let raw_stream: Stream<InT, Self, Unbounded, TotalOrder, ExactlyOnce> = Stream::new(
            self.clone(),
            HydroNode::ExternalInput {
                from_external_key: from.key,
                from_port_id: next_external_port_id,
                from_many: false,
                codec_type: quote_type::<LengthDelimitedCodec>().into(),
                port_hint: NetworkHint::Auto,
                instantiate_fn: DebugInstantiate::Building,
                deserialize_fn: Some(deser_fn.into()),
                metadata: self.new_node_metadata(Stream::<
                    InT,
                    Self,
                    Unbounded,
                    TotalOrder,
                    ExactlyOnce,
                >::collection_kind()),
            },
        );

        (
            ExternalBincodeBidi {
                process_key: from.key,
                port_id: next_external_port_id,
                _phantom: PhantomData,
            },
            raw_stream,
            fwd_ref,
        )
    }

    /// Establishes a server on this location to receive bidirectional connections from multiple
    /// external clients using raw bytes.
    ///
    /// Unlike [`Location::bind_single_client`], this method supports multiple concurrent client
    /// connections. Each client is assigned a unique `u64` identifier.
    ///
    /// Returns:
    /// - A port handle for external processes to connect to
    /// - A keyed stream of incoming messages, keyed by client ID
    /// - A keyed stream of membership events (client joins/leaves), keyed by client ID
    /// - A handle to send outgoing messages, keyed by client ID
    #[expect(clippy::type_complexity, reason = "stream markers")]
    fn bidi_external_many_bytes<L, T, Codec: Encoder<T> + Decoder>(
        &self,
        from: &External<L>,
        port_hint: NetworkHint,
    ) -> (
        ExternalBytesPort<Many>,
        KeyedStream<u64, <Codec as Decoder>::Item, Self, Unbounded, TotalOrder, ExactlyOnce>,
        KeyedStream<u64, MembershipEvent, Self, Unbounded, TotalOrder, ExactlyOnce>,
        ForwardHandle<'a, KeyedStream<u64, T, Self, Unbounded, NoOrder, ExactlyOnce>>,
    )
    where
        Self: Sized + NoTick,
    {
        let next_external_port_id = from.flow_state.borrow_mut().next_external_port();

        let (fwd_ref, to_sink) =
            self.forward_ref::<KeyedStream<u64, T, Self, Unbounded, NoOrder, ExactlyOnce>>();
        let mut flow_state_borrow = self.flow_state().borrow_mut();

        flow_state_borrow.push_root(HydroRoot::SendExternal {
            to_external_key: from.key,
            to_port_id: next_external_port_id,
            to_many: true,
            unpaired: false,
            serialize_fn: None,
            instantiate_fn: DebugInstantiate::Building,
            input: Box::new(to_sink.entries().ir_node.replace(HydroNode::Placeholder)),
            op_metadata: HydroIrOpMetadata::new(),
        });

        let raw_stream: Stream<
            Result<(u64, <Codec as Decoder>::Item), <Codec as Decoder>::Error>,
            Self,
            Unbounded,
            TotalOrder,
            ExactlyOnce,
        > = Stream::new(
            self.clone(),
            HydroNode::ExternalInput {
                from_external_key: from.key,
                from_port_id: next_external_port_id,
                from_many: true,
                codec_type: quote_type::<Codec>().into(),
                port_hint,
                instantiate_fn: DebugInstantiate::Building,
                deserialize_fn: None,
                metadata: self.new_node_metadata(Stream::<
                    Result<(u64, <Codec as Decoder>::Item), <Codec as Decoder>::Error>,
                    Self,
                    Unbounded,
                    TotalOrder,
                    ExactlyOnce,
                >::collection_kind()),
            },
        );

        let membership_stream_ident = syn::Ident::new(
            &format!(
                "__hydro_deploy_many_{}_{}_membership",
                from.key, next_external_port_id
            ),
            Span::call_site(),
        );
        let membership_stream_expr: syn::Expr = parse_quote!(#membership_stream_ident);
        let raw_membership_stream: KeyedStream<
            u64,
            bool,
            Self,
            Unbounded,
            TotalOrder,
            ExactlyOnce,
        > = KeyedStream::new(
            self.clone(),
            HydroNode::Source {
                source: HydroSource::Stream(membership_stream_expr.into()),
                metadata: self.new_node_metadata(KeyedStream::<
                    u64,
                    bool,
                    Self,
                    Unbounded,
                    TotalOrder,
                    ExactlyOnce,
                >::collection_kind()),
            },
        );

        (
            ExternalBytesPort {
                process_key: from.key,
                port_id: next_external_port_id,
                _phantom: PhantomData,
            },
            raw_stream
                .flatten_ordered() // TODO(shadaj): this silently drops framing errors, decide on right defaults
                .into_keyed(),
            raw_membership_stream.map(q!(|join| {
                if join {
                    MembershipEvent::Joined
                } else {
                    MembershipEvent::Left
                }
            })),
            fwd_ref,
        )
    }

    /// Establishes a server on this location to receive bidirectional connections from multiple
    /// external clients using bincode serialization.
    ///
    /// Unlike [`Location::bind_single_client_bincode`], this method supports multiple concurrent
    /// client connections. Each client is assigned a unique `u64` identifier.
    ///
    /// Returns:
    /// - A port handle for external processes to connect to
    /// - A keyed stream of incoming messages, keyed by client ID
    /// - A keyed stream of membership events (client joins/leaves), keyed by client ID
    /// - A handle to send outgoing messages, keyed by client ID
    ///
    /// # Type Parameters
    /// - `InT`: The type of incoming messages (must implement [`DeserializeOwned`])
    /// - `OutT`: The type of outgoing messages (must implement [`Serialize`])
    #[expect(clippy::type_complexity, reason = "stream markers")]
    fn bidi_external_many_bincode<L, InT: DeserializeOwned, OutT: Serialize>(
        &self,
        from: &External<L>,
    ) -> (
        ExternalBincodeBidi<InT, OutT, Many>,
        KeyedStream<u64, InT, Self, Unbounded, TotalOrder, ExactlyOnce>,
        KeyedStream<u64, MembershipEvent, Self, Unbounded, TotalOrder, ExactlyOnce>,
        ForwardHandle<'a, KeyedStream<u64, OutT, Self, Unbounded, NoOrder, ExactlyOnce>>,
    )
    where
        Self: Sized + NoTick,
    {
        let next_external_port_id = from.flow_state.borrow_mut().next_external_port();

        let (fwd_ref, to_sink) =
            self.forward_ref::<KeyedStream<u64, OutT, Self, Unbounded, NoOrder, ExactlyOnce>>();
        let mut flow_state_borrow = self.flow_state().borrow_mut();

        let root = get_this_crate();

        let out_t_type = quote_type::<OutT>();
        let ser_fn: syn::Expr = syn::parse_quote! {
            #root::runtime_support::stageleft::runtime_support::fn1_type_hint::<(u64, #out_t_type), _>(
                |(id, b)| (id, #root::runtime_support::bincode::serialize(&b).unwrap().into())
            )
        };

        flow_state_borrow.push_root(HydroRoot::SendExternal {
            to_external_key: from.key,
            to_port_id: next_external_port_id,
            to_many: true,
            unpaired: false,
            serialize_fn: Some(ser_fn.into()),
            instantiate_fn: DebugInstantiate::Building,
            input: Box::new(to_sink.entries().ir_node.replace(HydroNode::Placeholder)),
            op_metadata: HydroIrOpMetadata::new(),
        });

        let in_t_type = quote_type::<InT>();

        let deser_fn: syn::Expr = syn::parse_quote! {
            |res| {
                let (id, b) = res.unwrap();
                (id, #root::runtime_support::bincode::deserialize::<#in_t_type>(&b).unwrap())
            }
        };

        let raw_stream: KeyedStream<u64, InT, Self, Unbounded, TotalOrder, ExactlyOnce> =
            KeyedStream::new(
                self.clone(),
                HydroNode::ExternalInput {
                    from_external_key: from.key,
                    from_port_id: next_external_port_id,
                    from_many: true,
                    codec_type: quote_type::<LengthDelimitedCodec>().into(),
                    port_hint: NetworkHint::Auto,
                    instantiate_fn: DebugInstantiate::Building,
                    deserialize_fn: Some(deser_fn.into()),
                    metadata: self.new_node_metadata(KeyedStream::<
                        u64,
                        InT,
                        Self,
                        Unbounded,
                        TotalOrder,
                        ExactlyOnce,
                    >::collection_kind()),
                },
            );

        let membership_stream_ident = syn::Ident::new(
            &format!(
                "__hydro_deploy_many_{}_{}_membership",
                from.key, next_external_port_id
            ),
            Span::call_site(),
        );
        let membership_stream_expr: syn::Expr = parse_quote!(#membership_stream_ident);
        let raw_membership_stream: KeyedStream<
            u64,
            bool,
            Self,
            Unbounded,
            TotalOrder,
            ExactlyOnce,
        > = KeyedStream::new(
            self.clone(),
            HydroNode::Source {
                source: HydroSource::Stream(membership_stream_expr.into()),
                metadata: self.new_node_metadata(KeyedStream::<
                    u64,
                    bool,
                    Self,
                    Unbounded,
                    TotalOrder,
                    ExactlyOnce,
                >::collection_kind()),
            },
        );

        (
            ExternalBincodeBidi {
                process_key: from.key,
                port_id: next_external_port_id,
                _phantom: PhantomData,
            },
            raw_stream,
            raw_membership_stream.map(q!(|join| {
                if join {
                    MembershipEvent::Joined
                } else {
                    MembershipEvent::Left
                }
            })),
            fwd_ref,
        )
    }

    /// Constructs a [`Singleton`] materialized at this location with the given static value.
    ///
    /// See also: [`Tick::singleton`], for creating a singleton _within_ a tick, which requires
    /// `T: Clone`.
    ///
    /// # Example
    /// ```rust
    /// # #[cfg(feature = "deploy")] {
    /// # use hydro_lang::prelude::*;
    /// # use futures::StreamExt;
    /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
    /// let singleton = process.singleton(q!(5));
    /// # singleton.into_stream()
    /// # }, |mut stream| async move {
    /// // 5
    /// # assert_eq!(stream.next().await.unwrap(), 5);
    /// # }));
    /// # }
    /// ```
    fn singleton<T>(&self, e: impl QuotedWithContext<'a, T, Self>) -> Singleton<T, Self, Bounded>
    where
        Self: Sized + NoTick,
    {
        let e = e.splice_untyped_ctx(self);

        Singleton::new(
            self.clone(),
            HydroNode::SingletonSource {
                value: e.into(),
                first_tick_only: false,
                metadata: self.new_node_metadata(Singleton::<T, Self, Bounded>::collection_kind()),
            },
        )
    }

    /// Constructs a [`Singleton`] by resolving an async [`Future`] to completion.
    ///
    /// This is a convenience method equivalent to
    /// `self.singleton(future_expr).resolve_future_blocking()`, which is a common
    /// pattern when initializing a singleton from an async computation.
    ///
    /// # Example
    /// ```rust
    /// # #[cfg(feature = "deploy")] {
    /// # use hydro_lang::prelude::*;
    /// # use futures::StreamExt;
    /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
    /// let singleton = process.singleton_future(q!(async { 42 }));
    /// singleton.into_stream()
    /// # }, |mut stream| async move {
    /// // 42
    /// # assert_eq!(stream.next().await.unwrap(), 42);
    /// # }));
    /// # }
    /// ```
    ///
    /// [`Future`]: std::future::Future
    fn singleton_future<F>(
        &self,
        e: impl QuotedWithContext<'a, F, Self>,
    ) -> Singleton<F::Output, Self, Bounded>
    where
        F: Future,
        Self: Sized + NoTick,
    {
        self.singleton(e).resolve_future_blocking()
    }

    /// Generates a stream with values emitted at a fixed interval, with
    /// each value being the current time (as an [`tokio::time::Instant`]).
    ///
    /// The clock source used is monotonic, so elements will be emitted in
    /// increasing order.
    ///
    /// # Non-Determinism
    /// Because this stream is generated by an OS timer, it will be
    /// non-deterministic because each timestamp will be arbitrary.
    fn source_interval(
        &self,
        interval: impl QuotedWithContext<'a, Duration, Self> + Copy + 'a,
        _nondet: NonDet,
    ) -> Stream<tokio::time::Instant, Self, Unbounded, TotalOrder, ExactlyOnce>
    where
        Self: Sized + NoTick,
    {
        self.source_stream(q!(tokio_stream::wrappers::IntervalStream::new(
            tokio::time::interval(interval)
        )))
    }

    /// Generates a stream with values emitted at a fixed interval (with an
    /// initial delay), with each value being the current time
    /// (as an [`tokio::time::Instant`]).
    ///
    /// The clock source used is monotonic, so elements will be emitted in
    /// increasing order.
    ///
    /// # Non-Determinism
    /// Because this stream is generated by an OS timer, it will be
    /// non-deterministic because each timestamp will be arbitrary.
    fn source_interval_delayed(
        &self,
        delay: impl QuotedWithContext<'a, Duration, Self> + Copy + 'a,
        interval: impl QuotedWithContext<'a, Duration, Self> + Copy + 'a,
        _nondet: NonDet,
    ) -> Stream<tokio::time::Instant, Self, Unbounded, TotalOrder, ExactlyOnce>
    where
        Self: Sized + NoTick,
    {
        self.source_stream(q!(tokio_stream::wrappers::IntervalStream::new(
            tokio::time::interval_at(tokio::time::Instant::now() + delay, interval)
        )))
    }

    /// Creates a forward reference, allowing a stream to be used before its source is defined.
    ///
    /// Returns a `(handle, placeholder)` pair. Use the placeholder in the dataflow graph,
    /// then call `handle.complete(actual_stream)` to wire in the real source.
    ///
    /// This is useful for mutually-dependent dataflows or when the definition order
    /// doesn't match the data flow direction. For feedback loops, prefer [`Tick::cycle`]
    /// instead, which automatically defers values by one tick.
    ///
    /// # Panics
    /// Panics if the forward reference creates a synchronous cycle (i.e., the completed
    /// stream transitively depends on the placeholder without a `defer_tick` or network
    /// hop in between).
    ///
    /// # Example
    /// ```rust
    /// # #[cfg(feature = "deploy")] {
    /// # use hydro_lang::prelude::*;
    /// # use hydro_lang::live_collections::stream::NoOrder;
    /// # use futures::StreamExt;
    /// # tokio_test::block_on(hydro_lang::test_util::stream_transform_test(|process| {
    /// // Create a forward reference to define a stream that will be completed later
    /// let (complete, forward_stream) = process.forward_ref::<Stream<i32, _, _, NoOrder>>();
    ///
    /// // Use the forward reference as input to another computation
    /// let output: Stream<_, _, _, NoOrder> = forward_stream.map(q!(|x| x * 2));
    ///
    /// // Complete the forward reference with the actual source
    /// let source: Stream<_, _, Unbounded> = process.source_iter(q!([1, 2, 3])).into();
    /// complete.complete(source);
    /// output
    /// # }, |mut stream| async move {
    /// // 2, 4, 6
    /// # assert_eq!(stream.next().await.unwrap(), 2);
    /// # assert_eq!(stream.next().await.unwrap(), 4);
    /// # assert_eq!(stream.next().await.unwrap(), 6);
    /// # }));
    /// # }
    /// ```
    fn forward_ref<S>(&self) -> (ForwardHandle<'a, S>, S)
    where
        S: CycleCollection<'a, ForwardRef, Location = Self>,
    {
        let cycle_id = self.flow_state().borrow_mut().next_cycle_id();
        (
            ForwardHandle::new(cycle_id, Location::id(self)),
            S::create_source(cycle_id, self.clone()),
        )
    }
}

#[cfg(feature = "deploy")]
#[cfg(test)]
mod tests {
    use std::collections::HashSet;

    use futures::{SinkExt, StreamExt};
    use hydro_deploy::Deployment;
    use stageleft::q;
    use tokio_util::codec::LengthDelimitedCodec;

    use crate::compile::builder::FlowBuilder;
    use crate::live_collections::stream::{ExactlyOnce, TotalOrder};
    use crate::location::{Location, NetworkHint};
    use crate::nondet::nondet;

    #[tokio::test]
    async fn top_level_singleton_replay_cardinality() {
        let mut deployment = Deployment::new();

        let mut flow = FlowBuilder::new();
        let node = flow.process::<()>();
        let external = flow.external::<()>();

        let (in_port, input) =
            node.source_external_bincode::<_, _, TotalOrder, ExactlyOnce>(&external);
        let singleton = node.singleton(q!(123));
        let tick = node.tick();
        let out = input
            .batch(&tick, nondet!(/** test */))
            .cross_singleton(singleton.clone().snapshot(&tick, nondet!(/** test */)))
            .cross_singleton(
                singleton
                    .snapshot(&tick, nondet!(/** test */))
                    .into_stream()
                    .count(),
            )
            .all_ticks()
            .send_bincode_external(&external);

        let nodes = flow
            .with_process(&node, deployment.Localhost())
            .with_external(&external, deployment.Localhost())
            .deploy(&mut deployment);

        deployment.deploy().await.unwrap();

        let mut external_in = nodes.connect(in_port).await;
        let mut external_out = nodes.connect(out).await;

        deployment.start().await.unwrap();

        external_in.send(1).await.unwrap();
        assert_eq!(external_out.next().await.unwrap(), ((1, 123), 1));

        external_in.send(2).await.unwrap();
        assert_eq!(external_out.next().await.unwrap(), ((2, 123), 1));
    }

    #[tokio::test]
    async fn tick_singleton_replay_cardinality() {
        let mut deployment = Deployment::new();

        let mut flow = FlowBuilder::new();
        let node = flow.process::<()>();
        let external = flow.external::<()>();

        let (in_port, input) =
            node.source_external_bincode::<_, _, TotalOrder, ExactlyOnce>(&external);
        let tick = node.tick();
        let singleton = tick.singleton(q!(123));
        let out = input
            .batch(&tick, nondet!(/** test */))
            .cross_singleton(singleton.clone())
            .cross_singleton(singleton.into_stream().count())
            .all_ticks()
            .send_bincode_external(&external);

        let nodes = flow
            .with_process(&node, deployment.Localhost())
            .with_external(&external, deployment.Localhost())
            .deploy(&mut deployment);

        deployment.deploy().await.unwrap();

        let mut external_in = nodes.connect(in_port).await;
        let mut external_out = nodes.connect(out).await;

        deployment.start().await.unwrap();

        external_in.send(1).await.unwrap();
        assert_eq!(external_out.next().await.unwrap(), ((1, 123), 1));

        external_in.send(2).await.unwrap();
        assert_eq!(external_out.next().await.unwrap(), ((2, 123), 1));
    }

    #[tokio::test]
    async fn external_bytes() {
        let mut deployment = Deployment::new();

        let mut flow = FlowBuilder::new();
        let first_node = flow.process::<()>();
        let external = flow.external::<()>();

        let (in_port, input) = first_node.source_external_bytes(&external);
        let out = input.send_bincode_external(&external);

        let nodes = flow
            .with_process(&first_node, deployment.Localhost())
            .with_external(&external, deployment.Localhost())
            .deploy(&mut deployment);

        deployment.deploy().await.unwrap();

        let mut external_in = nodes.connect(in_port).await.1;
        let mut external_out = nodes.connect(out).await;

        deployment.start().await.unwrap();

        external_in.send(vec![1, 2, 3].into()).await.unwrap();

        assert_eq!(external_out.next().await.unwrap(), vec![1, 2, 3]);
    }

    #[tokio::test]
    async fn multi_external_source() {
        let mut deployment = Deployment::new();

        let mut flow = FlowBuilder::new();
        let first_node = flow.process::<()>();
        let external = flow.external::<()>();

        let (in_port, input, _membership, complete_sink) =
            first_node.bidi_external_many_bincode(&external);
        let out = input.entries().send_bincode_external(&external);
        complete_sink.complete(
            first_node
                .source_iter::<(u64, ()), _>(q!([]))
                .into_keyed()
                .weaken_ordering(),
        );

        let nodes = flow
            .with_process(&first_node, deployment.Localhost())
            .with_external(&external, deployment.Localhost())
            .deploy(&mut deployment);

        deployment.deploy().await.unwrap();

        let (_, mut external_in_1) = nodes.connect_bincode(in_port.clone()).await;
        let (_, mut external_in_2) = nodes.connect_bincode(in_port).await;
        let external_out = nodes.connect(out).await;

        deployment.start().await.unwrap();

        external_in_1.send(123).await.unwrap();
        external_in_2.send(456).await.unwrap();

        assert_eq!(
            external_out.take(2).collect::<HashSet<_>>().await,
            vec![(0, 123), (1, 456)].into_iter().collect()
        );
    }

    #[tokio::test]
    async fn second_connection_only_multi_source() {
        let mut deployment = Deployment::new();

        let mut flow = FlowBuilder::new();
        let first_node = flow.process::<()>();
        let external = flow.external::<()>();

        let (in_port, input, _membership, complete_sink) =
            first_node.bidi_external_many_bincode(&external);
        let out = input.entries().send_bincode_external(&external);
        complete_sink.complete(
            first_node
                .source_iter::<(u64, ()), _>(q!([]))
                .into_keyed()
                .weaken_ordering(),
        );

        let nodes = flow
            .with_process(&first_node, deployment.Localhost())
            .with_external(&external, deployment.Localhost())
            .deploy(&mut deployment);

        deployment.deploy().await.unwrap();

        // intentionally skipped to test stream waking logic
        let (_, mut _external_in_1) = nodes.connect_bincode(in_port.clone()).await;
        let (_, mut external_in_2) = nodes.connect_bincode(in_port).await;
        let mut external_out = nodes.connect(out).await;

        deployment.start().await.unwrap();

        external_in_2.send(456).await.unwrap();

        assert_eq!(external_out.next().await.unwrap(), (1, 456));
    }

    #[tokio::test]
    async fn multi_external_bytes() {
        let mut deployment = Deployment::new();

        let mut flow = FlowBuilder::new();
        let first_node = flow.process::<()>();
        let external = flow.external::<()>();

        let (in_port, input, _membership, complete_sink) = first_node
            .bidi_external_many_bytes::<_, _, LengthDelimitedCodec>(&external, NetworkHint::Auto);
        let out = input.entries().send_bincode_external(&external);
        complete_sink.complete(
            first_node
                .source_iter(q!([]))
                .into_keyed()
                .weaken_ordering(),
        );

        let nodes = flow
            .with_process(&first_node, deployment.Localhost())
            .with_external(&external, deployment.Localhost())
            .deploy(&mut deployment);

        deployment.deploy().await.unwrap();

        let mut external_in_1 = nodes.connect(in_port.clone()).await.1;
        let mut external_in_2 = nodes.connect(in_port).await.1;
        let external_out = nodes.connect(out).await;

        deployment.start().await.unwrap();

        external_in_1.send(vec![1, 2, 3].into()).await.unwrap();
        external_in_2.send(vec![4, 5].into()).await.unwrap();

        assert_eq!(
            external_out.take(2).collect::<HashSet<_>>().await,
            vec![
                (0, (&[1u8, 2, 3] as &[u8]).into()),
                (1, (&[4u8, 5] as &[u8]).into())
            ]
            .into_iter()
            .collect()
        );
    }

    #[tokio::test]
    async fn single_client_external_bytes() {
        let mut deployment = Deployment::new();
        let mut flow = FlowBuilder::new();
        let first_node = flow.process::<()>();
        let external = flow.external::<()>();
        let (port, input, complete_sink) = first_node
            .bind_single_client::<_, _, LengthDelimitedCodec>(&external, NetworkHint::Auto);
        complete_sink.complete(input.map(q!(|data| {
            let mut resp: Vec<u8> = data.into();
            resp.push(42);
            resp.into() // : Bytes
        })));

        let nodes = flow
            .with_process(&first_node, deployment.Localhost())
            .with_external(&external, deployment.Localhost())
            .deploy(&mut deployment);

        deployment.deploy().await.unwrap();
        deployment.start().await.unwrap();

        let (mut external_out, mut external_in) = nodes.connect(port).await;

        external_in.send(vec![1, 2, 3].into()).await.unwrap();
        assert_eq!(
            external_out.next().await.unwrap().unwrap(),
            vec![1, 2, 3, 42]
        );
    }

    #[tokio::test]
    async fn echo_external_bytes() {
        let mut deployment = Deployment::new();

        let mut flow = FlowBuilder::new();
        let first_node = flow.process::<()>();
        let external = flow.external::<()>();

        let (port, input, _membership, complete_sink) = first_node
            .bidi_external_many_bytes::<_, _, LengthDelimitedCodec>(&external, NetworkHint::Auto);
        complete_sink
            .complete(input.map(q!(|bytes| { bytes.into_iter().map(|x| x + 1).collect() })));

        let nodes = flow
            .with_process(&first_node, deployment.Localhost())
            .with_external(&external, deployment.Localhost())
            .deploy(&mut deployment);

        deployment.deploy().await.unwrap();

        let (mut external_out_1, mut external_in_1) = nodes.connect(port.clone()).await;
        let (mut external_out_2, mut external_in_2) = nodes.connect(port).await;

        deployment.start().await.unwrap();

        external_in_1.send(vec![1, 2, 3].into()).await.unwrap();
        external_in_2.send(vec![4, 5].into()).await.unwrap();

        assert_eq!(external_out_1.next().await.unwrap().unwrap(), vec![2, 3, 4]);
        assert_eq!(external_out_2.next().await.unwrap().unwrap(), vec![5, 6]);
    }

    #[tokio::test]
    async fn echo_external_bincode() {
        let mut deployment = Deployment::new();

        let mut flow = FlowBuilder::new();
        let first_node = flow.process::<()>();
        let external = flow.external::<()>();

        let (port, input, _membership, complete_sink) =
            first_node.bidi_external_many_bincode(&external);
        complete_sink.complete(input.map(q!(|text: String| { text.to_uppercase() })));

        let nodes = flow
            .with_process(&first_node, deployment.Localhost())
            .with_external(&external, deployment.Localhost())
            .deploy(&mut deployment);

        deployment.deploy().await.unwrap();

        let (mut external_out_1, mut external_in_1) = nodes.connect_bincode(port.clone()).await;
        let (mut external_out_2, mut external_in_2) = nodes.connect_bincode(port).await;

        deployment.start().await.unwrap();

        external_in_1.send("hi".to_owned()).await.unwrap();
        external_in_2.send("hello".to_owned()).await.unwrap();

        assert_eq!(external_out_1.next().await.unwrap(), "HI");
        assert_eq!(external_out_2.next().await.unwrap(), "HELLO");
    }

    #[tokio::test]
    async fn closure_location_name() {
        let mut deployment = Deployment::new();
        let mut flow = FlowBuilder::new();

        enum ClosureProcess {}

        let node = flow.process::<ClosureProcess>();
        let external = flow.external::<()>();

        let (in_port, input) =
            node.source_external_bincode::<_, i32, TotalOrder, ExactlyOnce>(&external);
        let out = input.send_bincode_external(&external);

        let nodes = flow
            .with_process(&node, deployment.Localhost())
            .with_external(&external, deployment.Localhost())
            .deploy(&mut deployment);

        deployment.deploy().await.unwrap();

        let mut external_in = nodes.connect(in_port).await;
        let mut external_out = nodes.connect(out).await;

        deployment.start().await.unwrap();

        external_in.send(42).await.unwrap();
        assert_eq!(external_out.next().await.unwrap(), 42);
    }
}