laminar-core 0.26.0

Core streaming engine for LaminarDB - operators, checkpoint barriers, and streaming primitives
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
//! Cross-node shuffle transport over Tonic gRPC client-streaming.
//!
//! Each sender opens a client-streaming `Shuffle` RPC per peer and pushes a
//! forward-only stream of `ShuffleFrame`s; the receiver runs the
//! `ShuffleTransport` service, attributes every stream to the peer announced in
//! its leading `Hello`, and surfaces decoded [`ShuffleMessage`]s on a bounded
//! crossfire MPSC queue. Backpressure is the HTTP/2 flow-control window plus
//! that bounded queue. See [`super::message`] for the per-frame payloads and
//! [`crate::serialization`] for the Arrow IPC (de)serialization of `VnodeData`.
//!
//! The real gRPC path is compiled under the `cluster` feature (which
//! pulls in `tonic`/`prost`). A default build keeps the same public API via a
//! networking-free shim so the types referenced by `laminar-db`/`laminar-server`
//! signatures still compile without the cluster dependencies.

use super::message::ShuffleMessage;
use crate::checkpoint::barrier::CheckpointBarrier;

/// Bounded capacity for the inbound shuffle queue. One consumer per
/// [`ShuffleReceiver`] (the cluster repartition dispatcher) drains it; a slow
/// consumer parks the per-stream service handler on the bounded `send`, so
/// backpressure flows back over HTTP/2 to the sender.
const SHUFFLE_RECV_QUEUE: usize = 1024;

/// Peer-local identifier on the wire. Matches `cluster::discovery::NodeId`'s
/// inner type for seamless conversion.
pub type ShufflePeerId = u64;

/// Gossip KV key used by [`ShuffleReceiver::bind_with_kv`] to publish the
/// listener's socket address, and by [`ShuffleSender`] to discover peer
/// addresses on first contact. Value: the bound socket address formatted via
/// `SocketAddr::to_string()`.
#[cfg(feature = "cluster")]
pub const SHUFFLE_ADDR_KEY: &str = "shuffle:addr";

#[cfg(feature = "cluster")]
#[allow(
    clippy::doc_markdown,
    clippy::default_trait_access,
    clippy::missing_const_for_fn,
    clippy::must_use_candidate,
    clippy::too_many_lines,
    missing_docs
)]
pub(crate) mod shuffle_v1 {
    tonic::include_proto!("laminar.shuffle.v1");
}

// ---------------------------------------------------------------------------
// Per-stage / per-barrier holdover shared by both builds.
// ---------------------------------------------------------------------------

/// Inbound-side holdover state lifted out of [`ShuffleReceiver`] so both the
/// gRPC and default builds share the staging semantics that barrier alignment
/// depends on: frames pulled for another stage are bucketed for that stage's own
/// drainer, and barriers pulled mid-cycle are stashed (never dropped) for the
/// aligning checkpoint.
#[derive(Default)]
struct Holdover {
    staged: parking_lot::Mutex<rustc_hash::FxHashMap<String, Vec<arrow_array::RecordBatch>>>,
    staged_barriers: parking_lot::Mutex<Vec<(ShufflePeerId, CheckpointBarrier)>>,
}

// ===========================================================================
// gRPC implementation (cluster).
// ===========================================================================

#[cfg(feature = "cluster")]
mod grpc {
    use std::collections::hash_map::Entry;
    use std::io;
    use std::net::SocketAddr;
    use std::sync::atomic::{AtomicBool, Ordering};
    use std::sync::Arc;

    use arrow_array::RecordBatch;
    use crossfire::{mpsc, AsyncRx, MAsyncTx};
    use futures::StreamExt as _;
    use parking_lot::Mutex;
    use rustc_hash::FxHashMap;
    use tokio::task::JoinHandle;
    use tonic::transport::{Channel, Server};
    use tonic::Request;

    use super::shuffle_v1::shuffle_frame;
    use super::shuffle_v1::shuffle_transport_client::ShuffleTransportClient;
    use super::shuffle_v1::shuffle_transport_server::{ShuffleTransport, ShuffleTransportServer};
    use super::shuffle_v1::{Barrier, Close, Hello, ShuffleFrame, ShuffleSummary, VnodeData};
    use super::{Holdover, ShuffleMessage, ShufflePeerId, SHUFFLE_ADDR_KEY, SHUFFLE_RECV_QUEUE};
    use crate::checkpoint::barrier::CheckpointBarrier;
    use crate::cluster::control::ClusterKv;
    use crate::serialization::{BatchStreamDecoder, BatchStreamEncoder};

    /// Outbound queue capacity per peer. Bounds per-peer buffering before the
    /// HTTP/2 window applies its own backpressure.
    const SHUFFLE_SEND_QUEUE: usize = 1024;

    /// Inbound queue item flavor (kept as a `type` so the parked-behind-mutex
    /// receiver field doesn't trip clippy's `type_complexity`).
    type InboundRx = AsyncRx<mpsc::Array<(ShufflePeerId, ShuffleMessage)>>;
    type InboundTx = MAsyncTx<mpsc::Array<(ShufflePeerId, ShuffleMessage)>>;

    /// Map a `tonic::Status` / `tonic::transport::Error` (or any `Display`) into
    /// `io::Error` so the public API keeps its `io::Result` shape.
    fn io_err<E: std::fmt::Display>(e: E) -> io::Error {
        io::Error::other(e.to_string())
    }

    /// Encode a [`ShuffleMessage`] into the wire [`ShuffleFrame`]. The per-stage
    /// [`BatchStreamEncoder`] writes the Arrow schema only on a stage's first
    /// `VnodeData`; later batches are schema-less. Runs in the connection driver
    /// task, off the Ring 0 compute thread.
    fn encode_message(
        msg: &ShuffleMessage,
        encoders: &mut FxHashMap<String, BatchStreamEncoder>,
    ) -> Result<ShuffleFrame, tonic::Status> {
        let kind = match msg {
            ShuffleMessage::Hello(node_id) => {
                shuffle_frame::Kind::Hello(Hello { node_id: *node_id })
            }
            ShuffleMessage::Barrier(b) => shuffle_frame::Kind::Barrier(Barrier {
                checkpoint_id: b.checkpoint_id,
                epoch: b.epoch,
                flags: b.flags,
            }),
            ShuffleMessage::VnodeData(stage, vnode, batch) => {
                let encoder = match encoders.entry(stage.clone()) {
                    Entry::Occupied(e) => {
                        let enc = e.into_mut();
                        // Fail loudly rather than desync the peer's IPC decoder.
                        let schema = batch.schema();
                        if !Arc::ptr_eq(enc.schema(), &schema) && *enc.schema() != schema {
                            return Err(tonic::Status::internal(format!(
                                "shuffle stage '{stage}' changed schema mid-connection",
                            )));
                        }
                        enc
                    }
                    Entry::Vacant(v) => {
                        v.insert(BatchStreamEncoder::new(&batch.schema()).map_err(|e| {
                            tonic::Status::internal(format!("shuffle ipc encoder init: {e}"))
                        })?)
                    }
                };
                let arrow_ipc = encoder
                    .encode(batch)
                    .map_err(|e| tonic::Status::internal(format!("shuffle ipc encode: {e}")))?;
                shuffle_frame::Kind::VnodeData(VnodeData {
                    stage: stage.clone(),
                    vnode: *vnode,
                    arrow_ipc,
                })
            }
            ShuffleMessage::Close(reason) => shuffle_frame::Kind::Close(Close {
                reason: reason.clone(),
            }),
        };
        Ok(ShuffleFrame { kind: Some(kind) })
    }

    /// One lazily-opened client-streaming call to a peer. The driver task pulls
    /// [`ShuffleMessage`]s from `tx`'s queue, serializes them to wire frames, and
    /// feeds the gRPC request stream; it flips `alive=false` on the first transport
    /// error (or connect failure), so the next `send_to` purges this entry and
    /// reconnects. Buffering messages (not frames) keeps the CPU-heavy Arrow IPC
    /// serialization off the caller's (Ring 0 compute) thread.
    struct PeerConn {
        tx: MAsyncTx<mpsc::Array<ShuffleMessage>>,
        alive: Arc<AtomicBool>,
        driver: JoinHandle<()>,
    }

    impl PeerConn {
        fn is_alive(&self) -> bool {
            self.alive.load(Ordering::Acquire)
        }
    }

    impl Drop for PeerConn {
        fn drop(&mut self) {
            self.driver.abort();
        }
    }

    /// Lazy pool of outbound client-streaming calls, keyed by peer id.
    pub struct ShuffleSender {
        local_id: ShufflePeerId,
        peers: Mutex<FxHashMap<ShufflePeerId, SocketAddr>>,
        pool: Mutex<FxHashMap<ShufflePeerId, Arc<PeerConn>>>,
        kv: Option<Arc<dyn ClusterKv>>,
    }

    impl std::fmt::Debug for ShuffleSender {
        fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
            f.debug_struct("ShuffleSender")
                .field("local_id", &self.local_id)
                .finish_non_exhaustive()
        }
    }

    impl ShuffleSender {
        /// Empty sender. Peers are added via [`Self::register_peer`] or discovered
        /// via the KV (in [`Self::with_kv`]) before any `send_to`.
        #[must_use]
        pub fn new(local_id: ShufflePeerId) -> Self {
            Self {
                local_id,
                peers: Mutex::new(FxHashMap::default()),
                pool: Mutex::new(FxHashMap::default()),
                kv: None,
            }
        }

        /// Sender that falls back to `kv` (key [`SHUFFLE_ADDR_KEY`] on the peer's
        /// own state) when `send_to` targets a peer not previously registered.
        #[must_use]
        pub fn with_kv(local_id: ShufflePeerId, kv: Arc<dyn ClusterKv>) -> Self {
            let mut s = Self::new(local_id);
            s.kv = Some(kv);
            s
        }

        /// Register (or update) a peer's shuffle address.
        // Body is sync, but the signature stays async to match the contract
        // callers `.await`.
        #[allow(clippy::unused_async)]
        pub async fn register_peer(&self, peer: ShufflePeerId, addr: SocketAddr) {
            self.peers.lock().insert(peer, addr);
        }

        /// Send `msg` to `peer`, opening a client-streaming call if necessary.
        ///
        /// # Errors
        /// Returns `io::Error` when the peer is unregistered/undiscoverable, the
        /// endpoint cannot be built, or the per-peer stream has shut down.
        pub async fn send_to(&self, peer: ShufflePeerId, msg: &ShuffleMessage) -> io::Result<()> {
            let conn = self.connection_for(peer).await?;
            // The clone is cheap (`RecordBatch` is an Arc bump); the driver
            // task serializes to Arrow IPC off this thread.
            conn.tx.send(msg.clone()).await.map_err(|_| {
                io::Error::new(
                    io::ErrorKind::BrokenPipe,
                    format!("shuffle stream to peer {peer} closed"),
                )
            })
        }

        /// Ship `barrier` to every peer in order, short-circuiting on the first
        /// failure (the gossip side-channel is authoritative, so a partial
        /// fan-out is tolerable).
        ///
        /// # Errors
        /// Returns the first `io::Error` from any peer's `send_to`.
        pub async fn fan_out_barrier(
            &self,
            peers: &[ShufflePeerId],
            barrier: CheckpointBarrier,
        ) -> io::Result<()> {
            let msg = ShuffleMessage::Barrier(barrier);
            for &peer in peers {
                self.send_to(peer, &msg).await?;
            }
            Ok(())
        }

        /// Resolve `peer`'s address from the KV (`SHUFFLE_ADDR_KEY` on the peer's
        /// own state) and cache it. `None` when no KV, no entry, or unparseable.
        async fn discover_peer(&self, peer: ShufflePeerId) -> Option<SocketAddr> {
            let kv = self.kv.as_ref()?;
            let raw = kv
                .read_from(crate::cluster::discovery::NodeId(peer), SHUFFLE_ADDR_KEY)
                .await?;
            let addr: SocketAddr = raw.parse().ok()?;
            self.peers.lock().insert(peer, addr);
            Some(addr)
        }

        async fn connection_for(&self, peer: ShufflePeerId) -> io::Result<Arc<PeerConn>> {
            if let Some(existing) = self.pool.lock().get(&peer).cloned() {
                if existing.is_alive() {
                    return Ok(existing);
                }
            }
            // Purge a dead entry so we reopen the call below.
            self.pool.lock().retain(|p, c| *p != peer || c.is_alive());

            // Re-resolve on reconnect (peers may restart on a new port); fall
            // back to a statically registered address when there's no KV.
            let addr = match self.discover_peer(peer).await {
                Some(addr) => addr,
                None => self.peers.lock().get(&peer).copied().ok_or_else(|| {
                    io::Error::new(
                        io::ErrorKind::NotFound,
                        format!("peer {peer} has no registered shuffle address"),
                    )
                })?,
            };

            let conn = Arc::new(open_call(self.local_id, addr)?);

            // Race: another task may have opened a live call meanwhile.
            let mut pool = self.pool.lock();
            if let Some(winner) = pool.get(&peer).cloned() {
                if winner.is_alive() {
                    return Ok(winner);
                }
            }
            pool.insert(peer, Arc::clone(&conn));
            Ok(conn)
        }
    }

    /// Open a client-streaming `Shuffle` call to `addr`, sending `Hello(local_id)`
    /// as the first frame. Connecting happens inside the driver task so this stays
    /// non-blocking; a connect failure flips `alive` so the next `send_to` retries.
    fn open_call(local_id: ShufflePeerId, addr: SocketAddr) -> io::Result<PeerConn> {
        let endpoint = crate::cluster::control::tls::client_endpoint(&addr.to_string())
            .map_err(io_err)?
            .tcp_nodelay(true);
        let (tx, rx) = mpsc::bounded_async::<ShuffleMessage>(SHUFFLE_SEND_QUEUE);
        let alive = Arc::new(AtomicBool::new(true));
        let alive_for_driver = Arc::clone(&alive);

        // Request stream: a single `Hello` chained onto an unfold over the
        // per-peer crossfire receiver (no `async-stream` dependency needed).
        // The unfold serializes dequeued messages here, in the driver task.
        let hello = ShuffleFrame {
            kind: Some(shuffle_frame::Kind::Hello(Hello { node_id: local_id })),
        };
        let encoders: FxHashMap<String, BatchStreamEncoder> = FxHashMap::default();
        let outbound = futures::stream::once(async move { hello }).chain(futures::stream::unfold(
            (rx, encoders),
            |(rx, mut encoders)| async move {
                let msg = rx.recv().await.ok()?;
                match encode_message(&msg, &mut encoders) {
                    Ok(frame) => Some((frame, (rx, encoders))),
                    Err(e) => {
                        // An unencodable batch would desync the stage's IPC
                        // stream; half-close so the peer reconnects fresh.
                        tracing::warn!(error = %e, "shuffle frame encode failed; closing stream");
                        None
                    }
                }
            },
        ));

        let driver = tokio::spawn(async move {
            let Ok(channel) = endpoint.connect().await else {
                alive_for_driver.store(false, Ordering::Release);
                return;
            };
            let mut client = ShuffleTransportClient::<Channel>::new(channel);
            // The call returns when the server responds to half-close or the
            // transport breaks; either way the peer connection is finished.
            let _ = client.shuffle(Request::new(outbound)).await;
            alive_for_driver.store(false, Ordering::Release);
        });

        Ok(PeerConn { tx, alive, driver })
    }

    /// Inbound side of the shuffle fabric: a Tonic `ShuffleTransport` server
    /// surfacing every received frame, attributed to its sending peer, on the
    /// bounded crossfire queue.
    pub struct ShuffleReceiver {
        local_id: ShufflePeerId,
        local_addr: SocketAddr,
        // crossfire's `AsyncRx` is `Send` but `!Sync` (it holds a
        // `PhantomData<Cell<()>>`), yet `Arc<ShuffleReceiver>` must be `Sync` — it
        // is embedded in DataFusion's `ClusterRepartitionExec`, whose
        // `ExecutionPlan` impl requires `Send + Sync`. Park the receiver behind a
        // `Mutex<Option<_>>` (which is `Sync` for any `Send` inner) and hand it out
        // via a take/return guard so the single async consumer never holds the
        // guard across `.await`. `rx_returned` wakes the next waiter; the guard
        // restores the receiver on drop so a cancelled `recv` can't strand it.
        rx: Mutex<Option<InboundRx>>,
        rx_returned: Arc<tokio::sync::Notify>,
        server: JoinHandle<()>,
        holdover: Arc<Holdover>,
    }

    impl Drop for ShuffleReceiver {
        fn drop(&mut self) {
            // Abort the server task so the listener closes and in-flight peer
            // streams break — senders then observe the error and reconnect.
            self.server.abort();
        }
    }

    impl std::fmt::Debug for ShuffleReceiver {
        fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
            f.debug_struct("ShuffleReceiver")
                .field("local_id", &self.local_id)
                .field("local_addr", &self.local_addr)
                .finish_non_exhaustive()
        }
    }

    impl ShuffleReceiver {
        /// Bind on `addr` and start serving. The bound address (with any ephemeral
        /// port resolved) is exposed via [`Self::local_addr`].
        ///
        /// # Errors
        /// Returns `io::Error` on bind failure.
        pub async fn bind(local_id: ShufflePeerId, addr: SocketAddr) -> io::Result<Self> {
            let listener = tokio::net::TcpListener::bind(addr).await?;
            let local_addr = listener.local_addr()?;
            let (tx, rx) =
                mpsc::bounded_async::<(ShufflePeerId, ShuffleMessage)>(SHUFFLE_RECV_QUEUE);

            let service = ShuffleService { tx };
            // Accept loop as a stream of `Result<TcpStream, io::Error>` for
            // `serve_with_incoming` — avoids the tokio-stream `net` feature.
            // nodelay is set per accepted connection.
            let incoming = futures::stream::unfold(listener, |listener| async move {
                let item = match listener.accept().await {
                    Ok((stream, _)) => {
                        let _ = stream.set_nodelay(true);
                        Ok(stream)
                    }
                    Err(e) => Err(e),
                };
                Some((item, listener))
            });
            // Apply TLS synchronously so a bad cert fails bind() rather than
            // silently never serving.
            let mut builder = Server::builder();
            if let Some(tls) = crate::cluster::control::tls::server_tls() {
                builder = builder
                    .tls_config(tls.clone())
                    .map_err(|e| io::Error::other(format!("cluster shuffle TLS config: {e}")))?;
            }
            let router = builder.add_service(ShuffleTransportServer::new(service));
            let server = tokio::spawn(async move {
                let _ = router.serve_with_incoming(incoming).await;
            });

            Ok(Self {
                local_id,
                local_addr,
                rx: Mutex::new(Some(rx)),
                rx_returned: Arc::new(tokio::sync::Notify::new()),
                server,
                holdover: Arc::new(Holdover::default()),
            })
        }

        /// Bind and publish the listener's address into `kv` under
        /// [`SHUFFLE_ADDR_KEY`] for peer discovery.
        ///
        /// # Errors
        /// Returns `io::Error` on bind failure.
        pub async fn bind_with_kv(
            local_id: ShufflePeerId,
            addr: SocketAddr,
            kv: Arc<dyn ClusterKv>,
        ) -> io::Result<Self> {
            let recv = Self::bind(local_id, addr).await?;
            kv.write(SHUFFLE_ADDR_KEY, recv.local_addr.to_string())
                .await;
            Ok(recv)
        }

        /// Local socket address the server is bound to.
        #[must_use]
        pub fn local_addr(&self) -> SocketAddr {
            self.local_addr
        }

        /// Await the next `(peer_id, msg)`. `None` once the server task has stopped
        /// and every queued item is drained. Single-owner; concurrent callers
        /// serialise via `rx_returned`. Cancellation-safe — a dropped `recv()`
        /// future returns the receiver to its slot via the RAII guard.
        pub async fn recv(&self) -> Option<(ShufflePeerId, ShuffleMessage)> {
            loop {
                // Take the receiver out under a short lock dropped before `.await`.
                let taken = { self.rx.lock().take() };
                let Some(rx) = taken else {
                    self.rx_returned.notified().await;
                    continue;
                };
                let mut guard = RxReturnGuard {
                    slot: &self.rx,
                    notify: &self.rx_returned,
                    rx: Some(rx),
                };
                let rx = guard.rx.as_mut()?;
                return rx.recv().await.ok();
            }
        }

        /// Drain every currently-available `(peer_id, msg)` without blocking. Empty
        /// when the queue is empty or a `recv()` currently holds the receiver.
        #[must_use]
        pub fn drain_available(&self) -> Vec<(ShufflePeerId, ShuffleMessage)> {
            let mut out = Vec::new();
            let slot = self.rx.lock();
            if let Some(rx) = slot.as_ref() {
                while let Ok(item) = rx.try_recv() {
                    out.push(item);
                }
            }
            out
        }

        /// Drain the inbound queue into `staged`: bucket `VnodeData` by stage,
        /// stash `Barrier`s for the aligning checkpoint (never dropped — see
        /// `Holdover`), discard `Hello`/`Close`.
        fn drain_inbound_into(&self, staged: &mut FxHashMap<String, Vec<RecordBatch>>) {
            let slot = self.rx.lock();
            if let Some(rx) = slot.as_ref() {
                while let Ok((from, msg)) = rx.try_recv() {
                    match msg {
                        ShuffleMessage::VnodeData(s, _vnode, batch) => {
                            staged.entry(s).or_default().push(batch);
                        }
                        ShuffleMessage::Barrier(b) => {
                            self.holdover.staged_barriers.lock().push((from, b));
                        }
                        _ => {} // Hello / Close
                    }
                }
            }
        }

        /// Non-blocking drain of the [`ShuffleMessage::VnodeData`] batches for
        /// `stage`; other stages stay bucketed for their own drainer. Empty if the
        /// queue is empty or a `recv()` holds it.
        #[must_use]
        pub fn drain_vnode_data_for(&self, stage: &str) -> Vec<RecordBatch> {
            let mut staged = self.holdover.staged.lock();
            self.drain_inbound_into(&mut staged);
            staged.remove(stage).unwrap_or_default()
        }

        /// Single lock-cycle drain of every staged stage whose key starts with
        /// `prefix`, lifting those out and leaving operator stages untouched. Lets
        /// the subscription router pull all `__sub::` batches in one pass.
        #[must_use]
        pub fn drain_staged_with_prefix(
            &self,
            prefix: &str,
        ) -> FxHashMap<String, Vec<RecordBatch>> {
            let mut staged = self.holdover.staged.lock();
            self.drain_inbound_into(&mut staged);
            let mut out: FxHashMap<String, Vec<RecordBatch>> = FxHashMap::default();
            staged.retain(|stage, batches| {
                if stage.starts_with(prefix) {
                    out.insert(stage.clone(), std::mem::take(batches));
                    false
                } else {
                    true
                }
            });
            out
        }

        /// Stage `batch` under `stage` for a later [`Self::drain_vnode_data_for`] /
        /// [`Self::drain_all_staged`] — used when no operator for `stage` exists yet
        /// at drain time.
        pub fn stage_batch(&self, stage: String, batch: RecordBatch) {
            self.holdover
                .staged
                .lock()
                .entry(stage)
                .or_default()
                .push(batch);
        }

        /// Take the barriers stashed by [`Self::drain_vnode_data_for`] (peers that
        /// fanned out before this node began aligning).
        #[must_use]
        pub fn drain_staged_barriers(&self) -> Vec<(ShufflePeerId, CheckpointBarrier)> {
            std::mem::take(&mut self.holdover.staged_barriers.lock())
        }

        /// Empty the per-stage holdover, returning every buffered `(stage, batch)`.
        #[must_use]
        pub fn drain_all_staged(&self) -> Vec<(String, RecordBatch)> {
            let mut staged = self.holdover.staged.lock();
            staged
                .drain()
                .flat_map(|(stage, batches)| batches.into_iter().map(move |b| (stage.clone(), b)))
                .collect()
        }
    }

    /// Returns the receiver to the slot on drop so a cancelled `recv()` future
    /// doesn't strand it; wakes the next parked waiter.
    struct RxReturnGuard<'a> {
        slot: &'a Mutex<Option<InboundRx>>,
        notify: &'a tokio::sync::Notify,
        rx: Option<InboundRx>,
    }

    impl Drop for RxReturnGuard<'_> {
        fn drop(&mut self) {
            if let Some(rx) = self.rx.take() {
                *self.slot.lock() = Some(rx);
                // notify_one stores a permit; notify_waiters can lose wakeups.
                self.notify.notify_one();
            }
        }
    }

    /// The `ShuffleTransport` service object: holds the producer end of the inbound
    /// queue shared by every peer stream.
    struct ShuffleService {
        tx: InboundTx,
    }

    #[tonic::async_trait]
    impl ShuffleTransport for ShuffleService {
        async fn shuffle(
            &self,
            request: Request<tonic::Streaming<ShuffleFrame>>,
        ) -> Result<tonic::Response<ShuffleSummary>, tonic::Status> {
            let summary = run_stream(self.tx.clone(), request.into_inner()).await?;
            Ok(tonic::Response::new(summary))
        }
    }

    /// Read the leading `Hello`, then forward each decoded frame onto the bounded
    /// inbound queue, returning a summary when the client half-closes. `VnodeData`
    /// is decoded with per-stage [`BatchStreamDecoder`]s mirroring the sender's
    /// per-stage encoders (schema on a stage's first chunk only).
    async fn run_stream(
        tx: InboundTx,
        mut stream: tonic::Streaming<ShuffleFrame>,
    ) -> Result<ShuffleSummary, tonic::Status> {
        let first = stream
            .message()
            .await?
            .ok_or_else(|| tonic::Status::invalid_argument("shuffle stream closed before Hello"))?;
        let peer = match first.kind {
            Some(shuffle_frame::Kind::Hello(h)) => h.node_id,
            _ => {
                return Err(tonic::Status::invalid_argument(
                    "first shuffle frame must be Hello",
                ))
            }
        };

        let mut decoders: FxHashMap<String, BatchStreamDecoder> = FxHashMap::default();
        let mut frames_received = 0u64;
        while let Some(frame) = stream.message().await? {
            let kind = frame
                .kind
                .ok_or_else(|| tonic::Status::invalid_argument("empty shuffle frame"))?;
            match kind {
                shuffle_frame::Kind::Close(_) => break,
                shuffle_frame::Kind::Hello(h) => {
                    frames_received += 1;
                    if tx
                        .send((peer, ShuffleMessage::Hello(h.node_id)))
                        .await
                        .is_err()
                    {
                        break;
                    }
                }
                shuffle_frame::Kind::Barrier(b) => {
                    frames_received += 1;
                    let msg = ShuffleMessage::Barrier(CheckpointBarrier {
                        checkpoint_id: b.checkpoint_id,
                        epoch: b.epoch,
                        flags: b.flags,
                    });
                    if tx.send((peer, msg)).await.is_err() {
                        break;
                    }
                }
                shuffle_frame::Kind::VnodeData(v) => {
                    frames_received += 1;
                    let batches = decoders
                        .entry(v.stage.clone())
                        .or_default()
                        .decode_chunk(v.arrow_ipc)
                        .map_err(|e| {
                            tonic::Status::invalid_argument(format!("shuffle ipc: {e}"))
                        })?;
                    let mut stream_broken = false;
                    for batch in batches {
                        if !forward_vnode_batch(&tx, peer, &v.stage, v.vnode, batch).await? {
                            stream_broken = true;
                            break;
                        }
                    }
                    if stream_broken {
                        break;
                    }
                }
            }
        }
        Ok(ShuffleSummary { frames_received })
    }

    /// Forward one decoded `stage` batch onto the inbound queue. If the batch
    /// carries the `__laminar_vnode` metadata column, split it per vnode and emit
    /// a slice each; otherwise emit it whole under `default_vnode`. Returns
    /// `Ok(false)` when the inbound queue has closed (the caller stops reading).
    async fn forward_vnode_batch(
        tx: &InboundTx,
        peer: ShufflePeerId,
        stage: &str,
        default_vnode: u32,
        batch: RecordBatch,
    ) -> Result<bool, tonic::Status> {
        let schema = batch.schema();
        let Some((col_idx, _field)) = schema.column_with_name("__laminar_vnode") else {
            let msg = ShuffleMessage::VnodeData(stage.to_string(), default_vnode, batch);
            return Ok(tx.send((peer, msg)).await.is_ok());
        };

        let vnode_array = batch
            .column(col_idx)
            .as_any()
            .downcast_ref::<arrow_array::UInt32Array>()
            .ok_or_else(|| {
                tonic::Status::invalid_argument("vnode metadata column is not UInt32Array")
            })?;
        let row_vnodes: Vec<u32> = vnode_array.values().to_vec();

        let mut projection: Vec<usize> = (0..schema.fields().len()).collect();
        projection.remove(col_idx);
        let batch_without_vnode = batch.project(&projection).map_err(|e| {
            tonic::Status::internal(format!("Failed to project out vnode metadata: {e}"))
        })?;

        let slices =
            crate::shuffle::routing::slice_batch_by_vnodes(&batch_without_vnode, &row_vnodes);
        for (v, slice) in slices {
            let sub_msg = ShuffleMessage::VnodeData(stage.to_string(), v, slice);
            if tx.send((peer, sub_msg)).await.is_err() {
                return Ok(false);
            }
        }
        Ok(true)
    }

    #[cfg(test)]
    mod encode_tests {
        use super::*;
        use arrow_array::Int64Array;
        use arrow_schema::{DataType, Field, Schema};

        #[test]
        fn schema_change_on_a_stage_is_rejected() {
            let batch = |name: &str| {
                let schema = Arc::new(Schema::new(vec![Field::new(name, DataType::Int64, false)]));
                arrow_array::RecordBatch::try_new(schema, vec![Arc::new(Int64Array::from(vec![1]))])
                    .unwrap()
            };
            let mut encoders = FxHashMap::default();
            let msg = ShuffleMessage::VnodeData("s".into(), 0, batch("a"));
            encode_message(&msg, &mut encoders).unwrap();

            let changed = ShuffleMessage::VnodeData("s".into(), 0, batch("b"));
            let err = encode_message(&changed, &mut encoders).unwrap_err();
            assert!(err.message().contains("changed schema"), "{err}");

            // A different stage with its own schema is fine.
            let other = ShuffleMessage::VnodeData("t".into(), 0, batch("b"));
            encode_message(&other, &mut encoders).unwrap();
        }
    }
}

#[cfg(feature = "cluster")]
pub use grpc::{ShuffleReceiver, ShuffleSender};

// ===========================================================================
// Default build: networking-free shim preserving the public API.
//
// The cluster shuffle is only exercised under `cluster`; a default
// build references these types only in signatures. The shim keeps the inbound
// crossfire queue + holdover staging so the surface compiles and behaves sanely
// (local-only) without pulling in tonic.
// ===========================================================================

#[cfg(not(feature = "cluster"))]
mod shim {
    use std::io;
    use std::net::SocketAddr;
    use std::sync::Arc;

    use arrow_array::RecordBatch;
    use crossfire::{mpsc, AsyncRx, MAsyncTx};
    use parking_lot::Mutex;
    use rustc_hash::FxHashMap;

    use super::{Holdover, ShuffleMessage, ShufflePeerId, SHUFFLE_RECV_QUEUE};
    use crate::checkpoint::barrier::CheckpointBarrier;

    type InboundRx = AsyncRx<mpsc::Array<(ShufflePeerId, ShuffleMessage)>>;
    type InboundTx = MAsyncTx<mpsc::Array<(ShufflePeerId, ShuffleMessage)>>;

    /// Outbound shuffle handle. Without the cluster feature there is no peer
    /// fabric, so sends to a non-local peer report the peer as unregistered.
    pub struct ShuffleSender {
        local_id: ShufflePeerId,
        peers: Mutex<FxHashMap<ShufflePeerId, SocketAddr>>,
    }

    impl std::fmt::Debug for ShuffleSender {
        fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
            f.debug_struct("ShuffleSender")
                .field("local_id", &self.local_id)
                .finish_non_exhaustive()
        }
    }

    impl ShuffleSender {
        /// Empty sender (no peer fabric without the cluster feature).
        #[must_use]
        pub fn new(local_id: ShufflePeerId) -> Self {
            Self {
                local_id,
                peers: Mutex::new(FxHashMap::default()),
            }
        }

        /// Register (or update) a peer's shuffle address.
        #[allow(clippy::unused_async)] // async to match the cluster build's API.
        pub async fn register_peer(&self, peer: ShufflePeerId, addr: SocketAddr) {
            self.peers.lock().insert(peer, addr);
        }

        /// # Errors
        /// Errors for an unregistered peer; the no-cluster build has no transport,
        /// so registered peers are accepted as a no-op delivery.
        #[allow(clippy::unused_async)] // async to match the cluster build's API.
        pub async fn send_to(&self, peer: ShufflePeerId, _msg: &ShuffleMessage) -> io::Result<()> {
            if self.peers.lock().contains_key(&peer) {
                Ok(())
            } else {
                Err(io::Error::new(
                    io::ErrorKind::NotFound,
                    format!("peer {peer} has no registered shuffle address"),
                ))
            }
        }

        /// # Errors
        /// Returns the first `io::Error` from any peer's `send_to`.
        pub async fn fan_out_barrier(
            &self,
            peers: &[ShufflePeerId],
            barrier: CheckpointBarrier,
        ) -> io::Result<()> {
            let msg = ShuffleMessage::Barrier(barrier);
            for &peer in peers {
                self.send_to(peer, &msg).await?;
            }
            Ok(())
        }
    }

    /// Inbound shuffle handle. Holds the bounded crossfire queue + holdover so the
    /// drain/stage API compiles and behaves locally without a network. The
    /// receiver is parked behind a `Mutex<Option<_>>` for the same `Sync` reason as
    /// the gRPC build.
    pub struct ShuffleReceiver {
        local_id: ShufflePeerId,
        local_addr: SocketAddr,
        #[allow(dead_code)]
        tx: InboundTx,
        rx: Mutex<Option<InboundRx>>,
        rx_returned: Arc<tokio::sync::Notify>,
        holdover: Arc<Holdover>,
    }

    impl std::fmt::Debug for ShuffleReceiver {
        fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
            f.debug_struct("ShuffleReceiver")
                .field("local_id", &self.local_id)
                .field("local_addr", &self.local_addr)
                .finish_non_exhaustive()
        }
    }

    impl ShuffleReceiver {
        /// # Errors
        /// Returns `io::Error` on bind failure.
        pub async fn bind(local_id: ShufflePeerId, addr: SocketAddr) -> io::Result<Self> {
            // Resolve the address (incl. ephemeral port) by binding momentarily.
            let listener = tokio::net::TcpListener::bind(addr).await?;
            let local_addr = listener.local_addr()?;
            drop(listener);
            let (tx, rx) =
                mpsc::bounded_async::<(ShufflePeerId, ShuffleMessage)>(SHUFFLE_RECV_QUEUE);
            Ok(Self {
                local_id,
                local_addr,
                tx,
                rx: Mutex::new(Some(rx)),
                rx_returned: Arc::new(tokio::sync::Notify::new()),
                holdover: Arc::new(Holdover::default()),
            })
        }

        /// Local socket address resolved at bind time.
        #[must_use]
        pub fn local_addr(&self) -> SocketAddr {
            self.local_addr
        }

        /// Await the next `(peer_id, msg)`. `None` once all senders drop.
        pub async fn recv(&self) -> Option<(ShufflePeerId, ShuffleMessage)> {
            loop {
                let taken = { self.rx.lock().take() };
                let Some(rx) = taken else {
                    self.rx_returned.notified().await;
                    continue;
                };
                let mut guard = RxReturnGuard {
                    slot: &self.rx,
                    notify: &self.rx_returned,
                    rx: Some(rx),
                };
                let rx = guard.rx.as_mut()?;
                return rx.recv().await.ok();
            }
        }

        /// Drain every currently-available `(peer_id, msg)` without blocking.
        #[must_use]
        pub fn drain_available(&self) -> Vec<(ShufflePeerId, ShuffleMessage)> {
            let mut out = Vec::new();
            let slot = self.rx.lock();
            if let Some(rx) = slot.as_ref() {
                while let Ok(item) = rx.try_recv() {
                    out.push(item);
                }
            }
            out
        }

        /// Non-blocking drain of the `VnodeData` batches for `stage`; other-stage
        /// frames are bucketed and barriers stashed (never dropped).
        #[must_use]
        pub fn drain_vnode_data_for(&self, stage: &str) -> Vec<RecordBatch> {
            let mut staged = self.holdover.staged.lock();
            {
                let slot = self.rx.lock();
                if let Some(rx) = slot.as_ref() {
                    while let Ok((from, msg)) = rx.try_recv() {
                        match msg {
                            ShuffleMessage::VnodeData(s, _vnode, batch) => {
                                staged.entry(s).or_default().push(batch);
                            }
                            ShuffleMessage::Barrier(b) => {
                                self.holdover.staged_barriers.lock().push((from, b));
                            }
                            _ => {}
                        }
                    }
                }
            }
            staged.remove(stage).unwrap_or_default()
        }

        /// Single lock-cycle drain of every staged stage whose key starts with
        /// `prefix`; other-stage frames are bucketed and barriers stashed (never
        /// dropped), matching [`Self::drain_vnode_data_for`]. Operator stages are
        /// left in `staged`; only the matching entries are returned.
        #[must_use]
        pub fn drain_staged_with_prefix(
            &self,
            prefix: &str,
        ) -> FxHashMap<String, Vec<RecordBatch>> {
            let mut staged = self.holdover.staged.lock();
            {
                let slot = self.rx.lock();
                if let Some(rx) = slot.as_ref() {
                    while let Ok((from, msg)) = rx.try_recv() {
                        match msg {
                            ShuffleMessage::VnodeData(s, _vnode, batch) => {
                                staged.entry(s).or_default().push(batch);
                            }
                            ShuffleMessage::Barrier(b) => {
                                self.holdover.staged_barriers.lock().push((from, b));
                            }
                            _ => {}
                        }
                    }
                }
            }
            let mut out: FxHashMap<String, Vec<RecordBatch>> = FxHashMap::default();
            staged.retain(|stage, batches| {
                if stage.starts_with(prefix) {
                    out.insert(stage.clone(), std::mem::take(batches));
                    false
                } else {
                    true
                }
            });
            out
        }

        /// Stage `batch` under `stage` for a later drain.
        pub fn stage_batch(&self, stage: String, batch: RecordBatch) {
            self.holdover
                .staged
                .lock()
                .entry(stage)
                .or_default()
                .push(batch);
        }

        /// Take the barriers stashed by [`Self::drain_vnode_data_for`].
        #[must_use]
        pub fn drain_staged_barriers(&self) -> Vec<(ShufflePeerId, CheckpointBarrier)> {
            std::mem::take(&mut self.holdover.staged_barriers.lock())
        }

        /// Empty the per-stage holdover, returning every buffered `(stage, batch)`.
        #[must_use]
        pub fn drain_all_staged(&self) -> Vec<(String, RecordBatch)> {
            let mut staged = self.holdover.staged.lock();
            staged
                .drain()
                .flat_map(|(stage, batches)| batches.into_iter().map(move |b| (stage.clone(), b)))
                .collect()
        }
    }

    /// Returns the receiver to the slot on drop so a cancelled `recv()` future
    /// doesn't strand it; wakes the next parked waiter.
    struct RxReturnGuard<'a> {
        slot: &'a Mutex<Option<InboundRx>>,
        notify: &'a tokio::sync::Notify,
        rx: Option<InboundRx>,
    }

    impl Drop for RxReturnGuard<'_> {
        fn drop(&mut self) {
            if let Some(rx) = self.rx.take() {
                *self.slot.lock() = Some(rx);
                self.notify.notify_one();
            }
        }
    }
}

#[cfg(not(feature = "cluster"))]
pub use shim::{ShuffleReceiver, ShuffleSender};

#[cfg(all(test, feature = "cluster"))]
mod tests {
    use std::io;
    use std::sync::Arc;

    use super::*;

    async fn bind_on_loopback(local_id: ShufflePeerId) -> ShuffleReceiver {
        ShuffleReceiver::bind(local_id, "127.0.0.1:0".parse().unwrap())
            .await
            .expect("bind")
    }

    #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
    async fn sender_to_receiver_delivers_with_peer_attribution() {
        let recv = bind_on_loopback(2).await;
        let recv_addr = recv.local_addr();

        let sender = ShuffleSender::new(1);
        sender.register_peer(2, recv_addr).await;
        sender
            .send_to(2, &ShuffleMessage::Hello(1234))
            .await
            .unwrap();

        let (from, msg) = recv.recv().await.unwrap();
        assert_eq!(from, 1, "receiver attributes frame to sender id");
        assert_eq!(msg, ShuffleMessage::Hello(1234));
    }

    #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
    async fn sender_reuses_stream_across_sends() {
        let recv = bind_on_loopback(2).await;
        let sender = ShuffleSender::new(1);
        sender.register_peer(2, recv.local_addr()).await;

        for delta in [10u64, 20, 30, 40] {
            sender
                .send_to(2, &ShuffleMessage::Hello(delta))
                .await
                .unwrap();
        }

        let mut got = Vec::new();
        for _ in 0..4 {
            got.push(recv.recv().await.unwrap().1);
        }
        assert_eq!(
            got,
            vec![
                ShuffleMessage::Hello(10),
                ShuffleMessage::Hello(20),
                ShuffleMessage::Hello(30),
                ShuffleMessage::Hello(40),
            ]
        );
    }

    #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
    async fn send_to_unregistered_peer_errors() {
        let sender = ShuffleSender::new(1);
        let err = sender
            .send_to(99, &ShuffleMessage::Hello(1))
            .await
            .unwrap_err();
        assert_eq!(err.kind(), io::ErrorKind::NotFound);
    }

    #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
    async fn send_discovers_peer_address_from_kv() {
        use crate::cluster::control::{ClusterKv, InMemoryKv};
        use crate::cluster::discovery::NodeId;

        // Peer 2 binds for real; its address is seeded into peer 1's KV so the
        // KV-backed sender resolves it on first send without an explicit
        // `register_peer`. End-to-end delivery proves the discovery glue.
        let recv = bind_on_loopback(2).await;
        let kv = Arc::new(InMemoryKv::new(NodeId(1)));
        kv.seed(NodeId(2), SHUFFLE_ADDR_KEY, recv.local_addr().to_string());
        let sender = ShuffleSender::with_kv(1, kv as Arc<dyn ClusterKv>);

        sender.send_to(2, &ShuffleMessage::Hello(7)).await.unwrap();
        let (from, msg) = recv.recv().await.unwrap();
        assert_eq!(from, 1);
        assert_eq!(msg, ShuffleMessage::Hello(7));
    }

    /// A peer restarting at a new address: the cached stream breaks, the next
    /// `send_to` reconnects against the freshly-registered address. Windows-only
    /// skip — the FIN-after-abort wakeup chain is not time-bounded under nextest
    /// parallelism there.
    #[cfg(not(windows))]
    #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
    async fn send_reconnects_after_peer_restart_at_new_address() {
        let recv_v1 = bind_on_loopback(2).await;
        let addr_v1 = recv_v1.local_addr();

        let sender = ShuffleSender::new(1);
        sender.register_peer(2, addr_v1).await;
        sender
            .send_to(2, &ShuffleMessage::Hello(111))
            .await
            .unwrap();
        let (from, msg) = recv_v1.recv().await.unwrap();
        assert_eq!(from, 1);
        assert_eq!(msg, ShuffleMessage::Hello(111));

        // Crash the peer.
        drop(recv_v1);

        // Peer restarts on a fresh ephemeral port.
        let recv_v2 = bind_on_loopback(2).await;
        let addr_v2 = recv_v2.local_addr();
        assert_ne!(addr_v1, addr_v2, "ephemeral rebind must pick a new port");
        sender.register_peer(2, addr_v2).await;

        // Reconnect + deliver to the restarted peer. Retry to absorb the time it
        // takes the old stream to flip dead after the server aborted.
        let deadline = std::time::Instant::now() + std::time::Duration::from_secs(30);
        loop {
            let _ = sender.send_to(2, &ShuffleMessage::Hello(222)).await;
            if let Some((from, ShuffleMessage::Hello(222))) =
                tokio::time::timeout(std::time::Duration::from_millis(200), recv_v2.recv())
                    .await
                    .ok()
                    .flatten()
            {
                assert_eq!(from, 1);
                return;
            }
            assert!(
                std::time::Instant::now() < deadline,
                "did not deliver to restarted peer within 30s",
            );
        }
    }

    /// `drain_staged_with_prefix` lifts `__sub::` stages in one pass while
    /// leaving operator stages staged for their own drainer.
    #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
    async fn drain_staged_with_prefix_lifts_subs_and_keeps_operator_stages() {
        use arrow_array::{Int64Array, RecordBatch};
        use arrow_schema::{DataType, Field, Schema};
        use rustc_hash::FxHashMap;

        use crate::checkpoint::barrier::CheckpointBarrier;

        fn batch(values: Vec<i64>) -> RecordBatch {
            let schema = Arc::new(Schema::new(vec![Field::new("x", DataType::Int64, false)]));
            RecordBatch::try_new(schema, vec![Arc::new(Int64Array::from(values))]).unwrap()
        }
        fn col(b: &RecordBatch) -> Vec<i64> {
            b.column(0)
                .as_any()
                .downcast_ref::<Int64Array>()
                .unwrap()
                .values()
                .to_vec()
        }

        let recv = bind_on_loopback(2).await;
        let sender = ShuffleSender::new(1);
        sender.register_peer(2, recv.local_addr()).await;

        // FIFO over one stream: two subscription stages, one operator stage, then
        // a trailing barrier. Once the barrier is observed, every prior frame has
        // been received and bucketed.
        for (stage, vals) in [
            ("__sub::alpha", vec![1, 2, 3]),
            ("__sub::beta", vec![4, 5, 6]),
            ("op_stage", vec![7, 8, 9]),
        ] {
            sender
                .send_to(2, &ShuffleMessage::VnodeData(stage.into(), 0, batch(vals)))
                .await
                .unwrap();
        }
        sender
            .send_to(
                2,
                &ShuffleMessage::Barrier(CheckpointBarrier {
                    checkpoint_id: 7,
                    epoch: 3,
                    flags: 0,
                }),
            )
            .await
            .unwrap();

        // Poll the single-lock-cycle drain until both sub stages and the trailing
        // barrier have arrived (loopback is near-instant; 2s is a wide margin).
        let mut subs: FxHashMap<String, Vec<RecordBatch>> = FxHashMap::default();
        let mut barriers = Vec::new();
        let deadline = std::time::Instant::now() + std::time::Duration::from_secs(2);
        while subs.len() < 2 || barriers.is_empty() {
            for (k, v) in recv.drain_staged_with_prefix("__sub::") {
                subs.entry(k).or_default().extend(v);
            }
            barriers.extend(recv.drain_staged_barriers());
            assert!(
                std::time::Instant::now() < deadline,
                "frames not delivered within 2s",
            );
            tokio::time::sleep(std::time::Duration::from_millis(5)).await;
        }

        // Both subscription stages lifted, with their batches intact.
        assert_eq!(subs.len(), 2, "only the two __sub:: stages are returned");
        assert_eq!(col(&subs["__sub::alpha"][0]), vec![1, 2, 3]);
        assert_eq!(col(&subs["__sub::beta"][0]), vec![4, 5, 6]);

        // The barrier was stashed, not dropped, and attributed to its sender.
        assert_eq!(barriers.len(), 1);
        assert_eq!(barriers[0].0, 1, "barrier attributed to sender peer 1");
        assert_eq!(barriers[0].1.checkpoint_id, 7);

        // The operator stage was left intact for its own drainer.
        let op = recv.drain_vnode_data_for("op_stage");
        assert_eq!(op.len(), 1);
        assert_eq!(col(&op[0]), vec![7, 8, 9]);

        // A second prefix drain finds nothing new.
        assert!(recv.drain_staged_with_prefix("__sub::").is_empty());
    }
}