openraft 0.9.24

Advanced Raft consensus
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
//! Provide a default chunked snapshot transport implementation for SnapshotData that implements
//! AsyncWrite + AsyncRead + AsyncSeek + Unpin.

use std::future::Future;
use std::io::SeekFrom;
use std::time::Duration;

use anyerror::AnyError;
use futures::FutureExt;
use openraft_macros::add_async_trait;
use tokio::io::AsyncReadExt;
use tokio::io::AsyncSeekExt;
use tokio::io::AsyncWriteExt;

use crate::error::Fatal;
use crate::error::InstallSnapshotError;
use crate::error::NetworkError;
use crate::error::RPCError;
use crate::error::RaftError;
use crate::error::RemoteError;
use crate::error::ReplicationClosed;
use crate::error::StreamingError;
use crate::network::Backoff;
use crate::network::RPCOption;
use crate::raft::InstallSnapshotRequest;
use crate::raft::SnapshotResponse;
use crate::type_config::TypeConfigExt;
use crate::ErrorSubject;
use crate::ErrorVerb;
use crate::OptionalSend;
use crate::Raft;
use crate::RaftNetwork;
use crate::RaftTypeConfig;
use crate::Snapshot;
use crate::SnapshotId;
use crate::StorageError;
use crate::StorageIOError;
use crate::ToStorageResult;
use crate::Vote;

/// Defines the sending and receiving API for snapshot transport.
#[add_async_trait]
pub trait SnapshotTransport<C: RaftTypeConfig> {
    /// Send a snapshot to a target node via `Net`.
    ///
    /// This function is for backward compatibility and provides a default implement for
    /// `RaftNetwork::full_snapshot()` upon `RafNetwork::install_snapshot()`.
    ///
    /// The argument `vote` is the leader's(the caller's) vote,
    /// which is used to check if the leader is still valid by a follower.
    ///
    /// `cancel` is a future that is polled by this function to check if the caller decides to
    /// cancel.
    /// It return `Ready` if the caller decide to cancel this snapshot transmission.
    // TODO: consider removing dependency on RaftNetwork
    async fn send_snapshot<Net>(
        net: &mut Net,
        vote: Vote<C::NodeId>,
        snapshot: Snapshot<C>,
        cancel: impl Future<Output = ReplicationClosed> + OptionalSend + 'static,
        option: RPCOption,
    ) -> Result<SnapshotResponse<C::NodeId>, StreamingError<C, Fatal<C::NodeId>>>
    where
        Net: RaftNetwork<C> + ?Sized;

    /// Receive a chunk of snapshot. If the snapshot is done receiving, return the snapshot.
    ///
    /// This method provide a default implementation for chunk based snapshot transport,
    /// and requires the caller to provide two things:
    ///
    /// - The receiving state `streaming` is maintained by the caller.
    /// - And it depends on `Raft::begin_receiving_snapshot()` to create a `SnapshotData` for
    /// receiving data.
    ///
    /// Example usage:
    /// ```ignore
    /// struct App<C> {
    ///     raft: Raft<C>
    ///     streaming: Option<Streaming<C>>,
    /// }
    ///
    /// impl<C> App<C> {
    ///     fn handle_install_snapshot_request(&mut self, req: InstallSnapshotRequest<C>) {
    ///         let res = Chunked::receive_snapshot(&mut self.streaming, &self.raft, req).await?;
    ///         if let Some(snapshot) = res {
    ///             self.raft.install_snapshot(snapshot).await?;
    ///         }
    ///     }
    /// }
    /// ```
    async fn receive_snapshot(
        streaming: &mut Option<Streaming<C>>,
        raft: &Raft<C>,
        req: InstallSnapshotRequest<C>,
    ) -> Result<Option<Snapshot<C>>, RaftError<C::NodeId, InstallSnapshotError>>;
}

/// Send and Receive snapshot by chunks.
pub struct Chunked {}

// Retry policy for chunked snapshot transport.
//
// `Timeout` and `Network` retries share a short exponential backoff local to
// this module: these errors typically clear within a packet-loss burst, and
// the caller's `Backoff` — intended for node-level unreachability — is too
// coarse for them.
//
// `Unreachable` uses the caller-supplied `RaftNetwork::backoff()` iterator
// because an unreachable target usually stays unreachable for seconds to
// minutes, a duration the application is better placed to pick.
//
// In all cases we cap at `SNAPSHOT_CHUNK_MAX_RETRIES` consecutive failures
// before surfacing the error; a successful chunk resets the counter and the
// backoff iterator so sporadic flakiness does not accumulate.
const SNAPSHOT_CHUNK_MAX_RETRIES: u64 = 5;
const SNAPSHOT_CHUNK_RETRY_BASE: Duration = Duration::from_millis(10);
const SNAPSHOT_CHUNK_RETRY_MAX: Duration = Duration::from_millis(200);

/// Fallback delay used when a caller-supplied [`Backoff`] iterator has been
/// exhausted. Matches the constant used by the default
/// [`RaftNetwork::backoff()`] implementation.
const SNAPSHOT_CHUNK_UNREACHABLE_FALLBACK: Duration = Duration::from_millis(500);

fn snapshot_chunk_retry_delay(consecutive_failures: u64) -> Duration {
    debug_assert!(consecutive_failures > 0);

    let shift = consecutive_failures.saturating_sub(1).min(4) as u32;
    let multiplier = 2u32.saturating_pow(shift);
    SNAPSHOT_CHUNK_RETRY_BASE.saturating_mul(multiplier).min(SNAPSHOT_CHUNK_RETRY_MAX)
}

/// This chunk based implementation requires `SnapshotData` to be `AsyncRead + AsyncSeek`.
impl<C: RaftTypeConfig> SnapshotTransport<C> for Chunked
where C::SnapshotData: tokio::io::AsyncRead + tokio::io::AsyncWrite + tokio::io::AsyncSeek + Unpin
{
    async fn send_snapshot<Net>(
        net: &mut Net,
        vote: Vote<C::NodeId>,
        mut snapshot: Snapshot<C>,
        cancel: impl Future<Output = ReplicationClosed> + OptionalSend + 'static,
        option: RPCOption,
    ) -> Result<SnapshotResponse<C::NodeId>, StreamingError<C, Fatal<C::NodeId>>>
    where
        Net: RaftNetwork<C> + ?Sized,
    {
        let subject_verb = || (ErrorSubject::Snapshot(Some(snapshot.meta.signature())), ErrorVerb::Read);

        let mut offset = 0;
        let end = snapshot.snapshot.seek(SeekFrom::End(0)).await.sto_res(subject_verb)?;
        let mut consecutive_failures = 0;
        let mut unreachable_backoff = None::<Backoff>;

        let mut c = std::pin::pin!(cancel);
        loop {
            // If canceled, return at once
            if let Some(err) = c.as_mut().now_or_never() {
                return Err(err.into());
            }

            // Sleep a short time otherwise in test environment it is a dead-loop that never
            // yields.
            // Because network implementation does not yield.
            C::sleep(Duration::from_millis(1)).await;

            snapshot.snapshot.seek(SeekFrom::Start(offset)).await.sto_res(subject_verb)?;

            // Safe unwrap(): this function is called only by default implementation of
            // `RaftNetwork::full_snapshot()` and it is always set.
            let chunk_size = option.snapshot_chunk_size().unwrap();
            let mut buf = Vec::with_capacity(chunk_size);
            while buf.capacity() > buf.len() {
                let n = snapshot.snapshot.read_buf(&mut buf).await.sto_res(subject_verb)?;
                if n == 0 {
                    break;
                }
            }

            let n_read = buf.len();

            let done = (offset + n_read as u64) == end;
            let req = InstallSnapshotRequest {
                vote: vote.clone(),
                meta: snapshot.meta.clone(),
                offset,
                data: buf,
                done,
            };

            // Send the RPC over to the target.
            tracing::debug!(
                snapshot_size = req.data.len(),
                req.offset,
                end,
                req.done,
                "sending snapshot chunk"
            );

            #[allow(deprecated)]
            let res = C::timeout(option.hard_ttl(), net.install_snapshot(req, option.clone())).await;

            let resp = match res {
                Err(outer_err) => {
                    // Outer `hard_ttl` expired before `install_snapshot` returned. Don't retry
                    // here: the replication layer drives the next attempt on its own timer with
                    // a fresh snapshot, while looping here just stacks in-flight RPCs under the
                    // same deadline.
                    tracing::warn!(error=%outer_err, "InstallSnapshot RPC timed out");
                    let any_err = AnyError::error(format!("InstallSnapshot RPC timed out: {outer_err}"));
                    return Err(NetworkError::new(&any_err).into());
                }
                Ok(Ok(resp)) => {
                    consecutive_failures = 0;
                    unreachable_backoff = None;
                    resp
                }
                Ok(Err(err)) => {
                    tracing::warn!(error=%err, "error sending InstallSnapshot RPC");

                    // Handle terminal and stream-reset variants up front — they skip the retry
                    // budget.
                    let err = match err {
                        RPCError::PayloadTooLarge(payload) => {
                            // Retrying the same chunk at the same size cannot make progress;
                            // the append-entries shrink path in `ReplicationCore` is not
                            // reusable here yet.
                            let any_err = AnyError::error(format!("snapshot chunk rejected as too large: {payload}"));
                            return Err(NetworkError::new(&any_err).into());
                        }
                        RPCError::RemoteError(RemoteError {
                            target,
                            target_node,
                            source: RaftError::Fatal(fatal),
                        }) => {
                            return Err(RemoteError {
                                target,
                                target_node,
                                source: fatal,
                            }
                            .into());
                        }
                        RPCError::RemoteError(RemoteError {
                            source: RaftError::APIError(InstallSnapshotError::SnapshotMismatch(mismatch)),
                            ..
                        }) => {
                            tracing::warn!(
                                mismatch = display(&mismatch),
                                "snapshot mismatch, reset offset and retry"
                            );
                            offset = 0;
                            consecutive_failures = 0;
                            unreachable_backoff = None;
                            continue;
                        }
                        err @ (RPCError::Timeout(_) | RPCError::Network(_) | RPCError::Unreachable(_)) => err,
                    };

                    // Strict rule: any transient error that exceeds the budget returns
                    // immediately, regardless of which variant it is.
                    consecutive_failures += 1;
                    if consecutive_failures >= SNAPSHOT_CHUNK_MAX_RETRIES {
                        return Err(match err {
                            RPCError::Timeout(e) => e.into(),
                            RPCError::Network(e) => e.into(),
                            RPCError::Unreachable(e) => e.into(),
                            _ => unreachable!("non-transient variants handled above"),
                        });
                    }

                    let delay = if matches!(err, RPCError::Unreachable(_)) {
                        unreachable_backoff
                            .get_or_insert_with(|| net.backoff())
                            .next()
                            .unwrap_or(SNAPSHOT_CHUNK_UNREACHABLE_FALLBACK)
                    } else {
                        snapshot_chunk_retry_delay(consecutive_failures)
                    };

                    C::sleep(delay).await;
                    continue;
                }
            };

            if resp.vote > vote {
                // Unfinished, return a response with a higher vote.
                // The caller checks the vote and return a HigherVote error.
                return Ok(SnapshotResponse::new(resp.vote));
            }

            if done {
                return Ok(SnapshotResponse::new(resp.vote));
            }

            offset += n_read as u64;
        }
    }

    async fn receive_snapshot(
        streaming: &mut Option<Streaming<C>>,
        raft: &Raft<C>,
        req: InstallSnapshotRequest<C>,
    ) -> Result<Option<Snapshot<C>>, RaftError<C::NodeId, InstallSnapshotError>> {
        let snapshot_id = &req.meta.snapshot_id;
        let snapshot_meta = req.meta.clone();
        let done = req.done;

        tracing::info!(req = display(&req), "{}", func_name!());

        let curr_id = streaming.as_ref().map(|s| s.snapshot_id());

        if curr_id != Some(snapshot_id) {
            if req.offset != 0 {
                let mismatch = InstallSnapshotError::SnapshotMismatch(crate::error::SnapshotMismatch {
                    expect: crate::SnapshotSegmentId {
                        id: snapshot_id.clone(),
                        offset: 0,
                    },
                    got: crate::SnapshotSegmentId {
                        id: snapshot_id.clone(),
                        offset: req.offset,
                    },
                });
                return Err(RaftError::APIError(mismatch));
            }

            // Changed to another stream. re-init snapshot state.
            let snapshot_data = raft.begin_receiving_snapshot().await.map_err(|e| {
                // Safe unwrap: `RaftError<Infallible>` is always a Fatal.
                RaftError::Fatal(e.into_fatal().unwrap())
            })?;

            *streaming = Some(Streaming::new(snapshot_id.clone(), snapshot_data));
        }

        {
            let s = streaming.as_mut().unwrap();
            s.receive(req).await?;
        }

        tracing::info!("Done received snapshot chunk");

        if done {
            let streaming = streaming.take().unwrap();
            let mut data = streaming.into_snapshot_data();

            data.as_mut().shutdown().await.map_err(|e| {
                let io_err = StorageIOError::write_snapshot(Some(snapshot_meta.signature()), &e);
                StorageError::from(io_err)
            })?;

            tracing::info!("finished streaming snapshot: {:?}", snapshot_meta);
            return Ok(Some(Snapshot::new(snapshot_meta, data)));
        }

        Ok(None)
    }
}

/// The Raft node is streaming in a snapshot from the leader.
pub struct Streaming<C>
where C: RaftTypeConfig
{
    /// The offset of the last byte written to the snapshot.
    offset: u64,

    /// The ID of the snapshot being written.
    snapshot_id: SnapshotId,

    /// A handle to the snapshot writer.
    snapshot_data: Box<C::SnapshotData>,
}

impl<C> Streaming<C>
where C: RaftTypeConfig
{
    pub fn new(snapshot_id: SnapshotId, snapshot_data: Box<C::SnapshotData>) -> Self {
        Self {
            offset: 0,
            snapshot_id,
            snapshot_data,
        }
    }

    pub fn snapshot_id(&self) -> &SnapshotId {
        &self.snapshot_id
    }

    /// Consumes the `Streaming` and returns the snapshot data.
    pub fn into_snapshot_data(self) -> Box<C::SnapshotData> {
        self.snapshot_data
    }
}

impl<C> Streaming<C>
where
    C: RaftTypeConfig,
    C::SnapshotData: tokio::io::AsyncWrite + tokio::io::AsyncSeek + Unpin,
{
    /// Receive a chunk of snapshot data.
    pub async fn receive(&mut self, req: InstallSnapshotRequest<C>) -> Result<bool, StorageError<C::NodeId>> {
        // TODO: check id?

        // Always seek to the target offset if not an exact match.
        if req.offset != self.offset {
            if let Err(err) = self.snapshot_data.as_mut().seek(SeekFrom::Start(req.offset)).await {
                return Err(StorageError::from_io_error(
                    ErrorSubject::Snapshot(Some(req.meta.signature())),
                    ErrorVerb::Seek,
                    err,
                ));
            }
            self.offset = req.offset;
        }

        // Write the next segment & update offset.
        let res = self.snapshot_data.as_mut().write_all(&req.data).await;
        if let Err(err) = res {
            return Err(StorageError::from_io_error(
                ErrorSubject::Snapshot(Some(req.meta.signature())),
                ErrorVerb::Write,
                err,
            ));
        }
        self.offset += req.data.len() as u64;
        Ok(req.done)
    }
}

#[cfg(feature = "generic-snapshot-data")]
#[cfg(test)]
mod tests {
    use std::collections::HashMap;
    use std::future::Future;
    use std::io::Cursor;
    use std::sync::Arc;
    use std::sync::Mutex;
    use std::time::Duration;

    use anyerror::AnyError;
    use tokio::time::sleep;

    use super::SNAPSHOT_CHUNK_MAX_RETRIES;
    use crate::engine::testing::UTConfig;
    use crate::error::Fatal;
    use crate::error::InstallSnapshotError;
    use crate::error::NetworkError;
    use crate::error::PayloadTooLarge;
    use crate::error::RPCError;
    use crate::error::RaftError;
    use crate::error::RemoteError;
    use crate::error::ReplicationClosed;
    use crate::error::SnapshotMismatch;
    use crate::error::StreamingError;
    use crate::error::Timeout;
    use crate::error::Unreachable;
    use crate::network::snapshot_transport::Chunked;
    use crate::network::snapshot_transport::SnapshotTransport;
    use crate::network::Backoff;
    use crate::network::RPCOption;
    use crate::raft::AppendEntriesRequest;
    use crate::raft::AppendEntriesResponse;
    use crate::raft::InstallSnapshotRequest;
    use crate::raft::InstallSnapshotResponse;
    use crate::raft::SnapshotResponse;
    use crate::raft::VoteRequest;
    use crate::raft::VoteResponse;
    use crate::OptionalSend;
    use crate::RPCTypes;
    use crate::RaftNetwork;
    use crate::RaftTypeConfig;
    use crate::Snapshot;
    use crate::SnapshotMeta;
    use crate::StoredMembership;
    use crate::Vote;

    struct Network {
        received_offset: Vec<u64>,
        match_cnt: u64,
    }

    impl<C> RaftNetwork<C> for Network
    where C: RaftTypeConfig<NodeId = u64>
    {
        async fn append_entries(
            &mut self,
            _rpc: AppendEntriesRequest<C>,
            _option: RPCOption,
        ) -> Result<AppendEntriesResponse<C::NodeId>, RPCError<C::NodeId, C::Node, RaftError<C::NodeId>>> {
            unimplemented!()
        }

        async fn vote(
            &mut self,
            _rpc: VoteRequest<C::NodeId>,
            _option: RPCOption,
        ) -> Result<VoteResponse<C::NodeId>, RPCError<C::NodeId, C::Node, RaftError<C::NodeId>>> {
            unimplemented!()
        }

        async fn full_snapshot(
            &mut self,
            _vote: Vote<C::NodeId>,
            _snapshot: Snapshot<C>,
            _cancel: impl Future<Output = ReplicationClosed> + OptionalSend + 'static,
            _option: RPCOption,
        ) -> Result<SnapshotResponse<C::NodeId>, StreamingError<C, Fatal<C::NodeId>>> {
            unimplemented!()
        }

        async fn install_snapshot(
            &mut self,
            rpc: InstallSnapshotRequest<C>,
            _option: RPCOption,
        ) -> Result<
            InstallSnapshotResponse<C::NodeId>,
            RPCError<C::NodeId, C::Node, RaftError<C::NodeId, InstallSnapshotError>>,
        > {
            // A fake implementation to test the Chunked::send_snapshot.

            self.received_offset.push(rpc.offset);

            // For the second last time, return a mismatch error.
            // Then return Ok for the reset of the time.
            self.match_cnt = self.match_cnt.saturating_sub(1);
            if self.match_cnt == 1 {
                let mismatch = SnapshotMismatch {
                    expect: crate::SnapshotSegmentId {
                        id: rpc.meta.snapshot_id.clone(),
                        offset: 0,
                    },
                    got: crate::SnapshotSegmentId {
                        id: rpc.meta.snapshot_id.clone(),
                        offset: rpc.offset,
                    },
                };
                let err = RaftError::APIError(InstallSnapshotError::SnapshotMismatch(mismatch));
                Err(RPCError::RemoteError(RemoteError::new(0, err)))
            } else {
                Ok(InstallSnapshotResponse { vote: rpc.vote })
            }
        }
    }

    /// Test that `Chunked` should reset the offset to 0 to re-send all data,
    /// if a [`SnapshotMismatch`] error is received.
    #[tokio::test]
    async fn test_chunked_reset_offset_if_snapshot_id_mismatch() {
        let mut net = Network {
            received_offset: vec![],
            // When match_cnt == 1, return a mismatch error.
            // For other times, return Ok.
            match_cnt: 4,
        };

        let mut opt = RPCOption::new(Duration::from_millis(100));
        opt.snapshot_chunk_size = Some(1);
        let cancel = futures::future::pending();

        Chunked::send_snapshot(
            &mut net,
            Vote::new(1, 0),
            Snapshot::<UTConfig>::new(
                SnapshotMeta {
                    last_log_id: None,
                    last_membership: StoredMembership::default(),
                    snapshot_id: "1-1-1-1".to_string(),
                },
                Box::new(Cursor::new(vec![1, 2, 3])),
            ),
            cancel,
            opt,
        )
        .await
        .unwrap();

        assert_eq!(net.received_offset, vec![0, 1, 2, 0, 1, 2]);
    }

    struct RetryNetwork {
        received_offset: Vec<u64>,
        fail_offset: u64,
        remaining_network_failures: u64,
    }

    impl<C> RaftNetwork<C> for RetryNetwork
    where C: RaftTypeConfig<NodeId = u64>
    {
        async fn append_entries(
            &mut self,
            _rpc: AppendEntriesRequest<C>,
            _option: RPCOption,
        ) -> Result<AppendEntriesResponse<C::NodeId>, RPCError<C::NodeId, C::Node, RaftError<C::NodeId>>> {
            unimplemented!()
        }

        async fn vote(
            &mut self,
            _rpc: VoteRequest<C::NodeId>,
            _option: RPCOption,
        ) -> Result<VoteResponse<C::NodeId>, RPCError<C::NodeId, C::Node, RaftError<C::NodeId>>> {
            unimplemented!()
        }

        async fn full_snapshot(
            &mut self,
            _vote: Vote<C::NodeId>,
            _snapshot: Snapshot<C>,
            _cancel: impl Future<Output = ReplicationClosed> + OptionalSend + 'static,
            _option: RPCOption,
        ) -> Result<SnapshotResponse<C::NodeId>, StreamingError<C, Fatal<C::NodeId>>> {
            unimplemented!()
        }

        async fn install_snapshot(
            &mut self,
            rpc: InstallSnapshotRequest<C>,
            _option: RPCOption,
        ) -> Result<
            InstallSnapshotResponse<C::NodeId>,
            RPCError<C::NodeId, C::Node, RaftError<C::NodeId, InstallSnapshotError>>,
        > {
            self.received_offset.push(rpc.offset);

            if rpc.offset == self.fail_offset && self.remaining_network_failures > 0 {
                self.remaining_network_failures -= 1;
                let any_err = AnyError::error("inject snapshot chunk network error");
                return Err(RPCError::Network(NetworkError::new(&any_err)));
            }

            Ok(InstallSnapshotResponse { vote: rpc.vote })
        }
    }

    #[tokio::test]
    async fn test_chunked_retry_resumes_from_current_offset_after_network_error() {
        let mut net = RetryNetwork {
            received_offset: vec![],
            fail_offset: 1,
            remaining_network_failures: 1,
        };

        let mut opt = RPCOption::new(Duration::from_millis(100));
        opt.snapshot_chunk_size = Some(1);
        let cancel = futures::future::pending();

        Chunked::send_snapshot(
            &mut net,
            Vote::new(1, 0),
            Snapshot::<UTConfig>::new(
                SnapshotMeta {
                    last_log_id: None,
                    last_membership: StoredMembership::default(),
                    snapshot_id: "1-1-1-2".to_string(),
                },
                Box::new(Cursor::new(vec![1, 2, 3])),
            ),
            cancel,
            opt,
        )
        .await
        .unwrap();

        assert_eq!(net.received_offset, vec![0, 1, 1, 2]);
    }

    #[tokio::test]
    async fn test_chunked_retry_budget_bails_out_after_consecutive_network_errors() {
        let mut net = RetryNetwork {
            received_offset: vec![],
            fail_offset: 1,
            remaining_network_failures: u64::MAX,
        };

        let mut opt = RPCOption::new(Duration::from_millis(100));
        opt.snapshot_chunk_size = Some(1);
        let cancel = futures::future::pending();

        let err = Chunked::send_snapshot(
            &mut net,
            Vote::new(1, 0),
            Snapshot::<UTConfig>::new(
                SnapshotMeta {
                    last_log_id: None,
                    last_membership: StoredMembership::default(),
                    snapshot_id: "1-1-1-3".to_string(),
                },
                Box::new(Cursor::new(vec![1, 2, 3])),
            ),
            cancel,
            opt,
        )
        .await
        .unwrap_err();

        assert!(matches!(err, StreamingError::Network(_)));
        let mut expected = vec![0];
        expected.extend(std::iter::repeat_n(1, SNAPSHOT_CHUNK_MAX_RETRIES as usize));
        assert_eq!(net.received_offset, expected);
    }

    struct SlowNetwork {
        received_offset: Vec<u64>,
        delay: Duration,
    }

    impl<C> RaftNetwork<C> for SlowNetwork
    where C: RaftTypeConfig<NodeId = u64>
    {
        async fn append_entries(
            &mut self,
            _rpc: AppendEntriesRequest<C>,
            _option: RPCOption,
        ) -> Result<AppendEntriesResponse<C::NodeId>, RPCError<C::NodeId, C::Node, RaftError<C::NodeId>>> {
            unimplemented!()
        }

        async fn vote(
            &mut self,
            _rpc: VoteRequest<C::NodeId>,
            _option: RPCOption,
        ) -> Result<VoteResponse<C::NodeId>, RPCError<C::NodeId, C::Node, RaftError<C::NodeId>>> {
            unimplemented!()
        }

        async fn full_snapshot(
            &mut self,
            _vote: Vote<C::NodeId>,
            _snapshot: Snapshot<C>,
            _cancel: impl Future<Output = ReplicationClosed> + OptionalSend + 'static,
            _option: RPCOption,
        ) -> Result<SnapshotResponse<C::NodeId>, StreamingError<C, Fatal<C::NodeId>>> {
            unimplemented!()
        }

        async fn install_snapshot(
            &mut self,
            rpc: InstallSnapshotRequest<C>,
            _option: RPCOption,
        ) -> Result<
            InstallSnapshotResponse<C::NodeId>,
            RPCError<C::NodeId, C::Node, RaftError<C::NodeId, InstallSnapshotError>>,
        > {
            self.received_offset.push(rpc.offset);
            sleep(self.delay).await;
            Ok(InstallSnapshotResponse { vote: rpc.vote })
        }
    }

    #[tokio::test]
    async fn test_chunked_outer_timeout_does_not_retry() {
        let mut net = SlowNetwork {
            received_offset: vec![],
            delay: Duration::from_millis(20),
        };

        let mut opt = RPCOption::new(Duration::from_millis(1));
        opt.snapshot_chunk_size = Some(1);
        let cancel = futures::future::pending();

        let err = Chunked::send_snapshot(
            &mut net,
            Vote::new(1, 0),
            Snapshot::<UTConfig>::new(
                SnapshotMeta {
                    last_log_id: None,
                    last_membership: StoredMembership::default(),
                    snapshot_id: "1-1-1-4".to_string(),
                },
                Box::new(Cursor::new(vec![1, 2, 3])),
            ),
            cancel,
            opt,
        )
        .await
        .unwrap_err();

        assert!(matches!(err, StreamingError::Network(_)));
        assert_eq!(net.received_offset, vec![0]);
    }

    struct PayloadTooLargeNetwork {
        received_offset: Vec<u64>,
    }

    impl<C> RaftNetwork<C> for PayloadTooLargeNetwork
    where C: RaftTypeConfig<NodeId = u64>
    {
        async fn append_entries(
            &mut self,
            _rpc: AppendEntriesRequest<C>,
            _option: RPCOption,
        ) -> Result<AppendEntriesResponse<C::NodeId>, RPCError<C::NodeId, C::Node, RaftError<C::NodeId>>> {
            unimplemented!()
        }

        async fn vote(
            &mut self,
            _rpc: VoteRequest<C::NodeId>,
            _option: RPCOption,
        ) -> Result<VoteResponse<C::NodeId>, RPCError<C::NodeId, C::Node, RaftError<C::NodeId>>> {
            unimplemented!()
        }

        async fn full_snapshot(
            &mut self,
            _vote: Vote<C::NodeId>,
            _snapshot: Snapshot<C>,
            _cancel: impl Future<Output = ReplicationClosed> + OptionalSend + 'static,
            _option: RPCOption,
        ) -> Result<SnapshotResponse<C::NodeId>, StreamingError<C, Fatal<C::NodeId>>> {
            unimplemented!()
        }

        async fn install_snapshot(
            &mut self,
            rpc: InstallSnapshotRequest<C>,
            _option: RPCOption,
        ) -> Result<
            InstallSnapshotResponse<C::NodeId>,
            RPCError<C::NodeId, C::Node, RaftError<C::NodeId, InstallSnapshotError>>,
        > {
            self.received_offset.push(rpc.offset);
            Err(RPCError::PayloadTooLarge(PayloadTooLarge::new_bytes_hint(1)))
        }
    }

    #[tokio::test]
    async fn test_chunked_payload_too_large_fails_fast() {
        let mut net = PayloadTooLargeNetwork {
            received_offset: vec![],
        };

        let mut opt = RPCOption::new(Duration::from_millis(100));
        opt.snapshot_chunk_size = Some(1);
        let cancel = futures::future::pending();

        let err = Chunked::send_snapshot(
            &mut net,
            Vote::new(1, 0),
            Snapshot::<UTConfig>::new(
                SnapshotMeta {
                    last_log_id: None,
                    last_membership: StoredMembership::default(),
                    snapshot_id: "1-1-1-5".to_string(),
                },
                Box::new(Cursor::new(vec![1, 2, 3])),
            ),
            cancel,
            opt,
        )
        .await
        .unwrap_err();

        assert!(matches!(err, StreamingError::Network(_)));
        assert_eq!(net.received_offset, vec![0], "must not retry the same chunk");
    }

    struct FatalNetwork {
        received_offset: Vec<u64>,
    }

    impl<C> RaftNetwork<C> for FatalNetwork
    where C: RaftTypeConfig<NodeId = u64>
    {
        async fn append_entries(
            &mut self,
            _rpc: AppendEntriesRequest<C>,
            _option: RPCOption,
        ) -> Result<AppendEntriesResponse<C::NodeId>, RPCError<C::NodeId, C::Node, RaftError<C::NodeId>>> {
            unimplemented!()
        }

        async fn vote(
            &mut self,
            _rpc: VoteRequest<C::NodeId>,
            _option: RPCOption,
        ) -> Result<VoteResponse<C::NodeId>, RPCError<C::NodeId, C::Node, RaftError<C::NodeId>>> {
            unimplemented!()
        }

        async fn full_snapshot(
            &mut self,
            _vote: Vote<C::NodeId>,
            _snapshot: Snapshot<C>,
            _cancel: impl Future<Output = ReplicationClosed> + OptionalSend + 'static,
            _option: RPCOption,
        ) -> Result<SnapshotResponse<C::NodeId>, StreamingError<C, Fatal<C::NodeId>>> {
            unimplemented!()
        }

        async fn install_snapshot(
            &mut self,
            rpc: InstallSnapshotRequest<C>,
            _option: RPCOption,
        ) -> Result<
            InstallSnapshotResponse<C::NodeId>,
            RPCError<C::NodeId, C::Node, RaftError<C::NodeId, InstallSnapshotError>>,
        > {
            self.received_offset.push(rpc.offset);
            Err(RPCError::RemoteError(RemoteError::new(
                2,
                RaftError::Fatal(Fatal::Panicked),
            )))
        }
    }

    #[tokio::test]
    async fn test_chunked_remote_fatal_is_propagated() {
        let mut net = FatalNetwork {
            received_offset: vec![],
        };

        let mut opt = RPCOption::new(Duration::from_millis(100));
        opt.snapshot_chunk_size = Some(1);
        let cancel = futures::future::pending();

        let err = Chunked::send_snapshot(
            &mut net,
            Vote::new(1, 0),
            Snapshot::<UTConfig>::new(
                SnapshotMeta {
                    last_log_id: None,
                    last_membership: StoredMembership::default(),
                    snapshot_id: "1-1-1-6".to_string(),
                },
                Box::new(Cursor::new(vec![1, 2, 3])),
            ),
            cancel,
            opt,
        )
        .await
        .unwrap_err();

        assert!(
            matches!(err, StreamingError::RemoteError(_)),
            "remote Fatal must surface, got {err:?}"
        );
        assert_eq!(net.received_offset, vec![0], "remote Fatal must not be retried");
    }

    struct UnreachableNetwork {
        received_offset: Vec<u64>,
        /// Offsets at which one `Unreachable` error should be injected (consumed on first hit).
        unreachable_at: Vec<u64>,
        backoff_calls: Arc<Mutex<u64>>,
    }

    impl<C> RaftNetwork<C> for UnreachableNetwork
    where C: RaftTypeConfig<NodeId = u64>
    {
        async fn append_entries(
            &mut self,
            _rpc: AppendEntriesRequest<C>,
            _option: RPCOption,
        ) -> Result<AppendEntriesResponse<C::NodeId>, RPCError<C::NodeId, C::Node, RaftError<C::NodeId>>> {
            unimplemented!()
        }

        async fn vote(
            &mut self,
            _rpc: VoteRequest<C::NodeId>,
            _option: RPCOption,
        ) -> Result<VoteResponse<C::NodeId>, RPCError<C::NodeId, C::Node, RaftError<C::NodeId>>> {
            unimplemented!()
        }

        async fn full_snapshot(
            &mut self,
            _vote: Vote<C::NodeId>,
            _snapshot: Snapshot<C>,
            _cancel: impl Future<Output = ReplicationClosed> + OptionalSend + 'static,
            _option: RPCOption,
        ) -> Result<SnapshotResponse<C::NodeId>, StreamingError<C, Fatal<C::NodeId>>> {
            unimplemented!()
        }

        fn backoff(&self) -> Backoff {
            *self.backoff_calls.lock().unwrap() += 1;
            // Use a short, infinite iterator so the test is fast but the iterator never exhausts.
            Backoff::new(std::iter::repeat(Duration::from_millis(1)))
        }

        async fn install_snapshot(
            &mut self,
            rpc: InstallSnapshotRequest<C>,
            _option: RPCOption,
        ) -> Result<
            InstallSnapshotResponse<C::NodeId>,
            RPCError<C::NodeId, C::Node, RaftError<C::NodeId, InstallSnapshotError>>,
        > {
            self.received_offset.push(rpc.offset);

            if let Some(idx) = self.unreachable_at.iter().position(|&o| o == rpc.offset) {
                self.unreachable_at.remove(idx);
                let any_err = AnyError::error("target unreachable");
                return Err(RPCError::Unreachable(Unreachable::new(&any_err)));
            }
            Ok(InstallSnapshotResponse { vote: rpc.vote })
        }
    }

    /// `Unreachable` errors pull the next delay from `net.backoff()`, and the cached iterator
    /// is dropped on the next successful chunk so a fresh outage gets a fresh backoff.
    #[tokio::test]
    async fn test_chunked_retry_on_unreachable_uses_caller_backoff() {
        let backoff_calls = Arc::new(Mutex::new(0u64));
        let mut net = UnreachableNetwork {
            received_offset: vec![],
            // Two separate outages: one at offset 1, one at offset 2.
            unreachable_at: vec![1, 2],
            backoff_calls: backoff_calls.clone(),
        };

        let mut opt = RPCOption::new(Duration::from_millis(100));
        opt.snapshot_chunk_size = Some(1);
        let cancel = futures::future::pending();

        Chunked::send_snapshot(
            &mut net,
            Vote::new(1, 0),
            Snapshot::<UTConfig>::new(
                SnapshotMeta {
                    last_log_id: None,
                    last_membership: StoredMembership::default(),
                    snapshot_id: "1-1-1-7".to_string(),
                },
                Box::new(Cursor::new(vec![1, 2, 3])),
            ),
            cancel,
            opt,
        )
        .await
        .unwrap();

        assert_eq!(net.received_offset, vec![0, 1, 1, 2, 2]);
        assert_eq!(
            *backoff_calls.lock().unwrap(),
            2,
            "net.backoff() must be called once per outage — the cached iterator is dropped on success"
        );
    }

    struct BurstyNetwork {
        received_offset: Vec<u64>,
        /// For each offset, how many `Network` failures remain to inject before it succeeds.
        fails_remaining_per_offset: HashMap<u64, u64>,
    }

    impl<C> RaftNetwork<C> for BurstyNetwork
    where C: RaftTypeConfig<NodeId = u64>
    {
        async fn append_entries(
            &mut self,
            _rpc: AppendEntriesRequest<C>,
            _option: RPCOption,
        ) -> Result<AppendEntriesResponse<C::NodeId>, RPCError<C::NodeId, C::Node, RaftError<C::NodeId>>> {
            unimplemented!()
        }

        async fn vote(
            &mut self,
            _rpc: VoteRequest<C::NodeId>,
            _option: RPCOption,
        ) -> Result<VoteResponse<C::NodeId>, RPCError<C::NodeId, C::Node, RaftError<C::NodeId>>> {
            unimplemented!()
        }

        async fn full_snapshot(
            &mut self,
            _vote: Vote<C::NodeId>,
            _snapshot: Snapshot<C>,
            _cancel: impl Future<Output = ReplicationClosed> + OptionalSend + 'static,
            _option: RPCOption,
        ) -> Result<SnapshotResponse<C::NodeId>, StreamingError<C, Fatal<C::NodeId>>> {
            unimplemented!()
        }

        async fn install_snapshot(
            &mut self,
            rpc: InstallSnapshotRequest<C>,
            _option: RPCOption,
        ) -> Result<
            InstallSnapshotResponse<C::NodeId>,
            RPCError<C::NodeId, C::Node, RaftError<C::NodeId, InstallSnapshotError>>,
        > {
            self.received_offset.push(rpc.offset);

            let remaining = self.fails_remaining_per_offset.entry(rpc.offset).or_insert(0);
            if *remaining > 0 {
                *remaining -= 1;
                let any_err = AnyError::error("inject network error");
                return Err(RPCError::Network(NetworkError::new(&any_err)));
            }
            Ok(InstallSnapshotResponse { vote: rpc.vote })
        }
    }

    /// A successful chunk resets `consecutive_failures`, so several bursts of
    /// `MAX_RETRIES - 1` failures in a row succeed even though the total number of
    /// failures exceeds the budget.
    #[tokio::test]
    async fn test_chunked_success_resets_retry_budget() {
        let burst = SNAPSHOT_CHUNK_MAX_RETRIES - 1;
        let mut fails = HashMap::new();
        fails.insert(0, burst);
        fails.insert(1, burst);

        let mut net = BurstyNetwork {
            received_offset: vec![],
            fails_remaining_per_offset: fails,
        };

        let mut opt = RPCOption::new(Duration::from_millis(100));
        opt.snapshot_chunk_size = Some(1);
        let cancel = futures::future::pending();

        // 2-byte snapshot = 2 chunks. Total failures injected = 2 * burst > MAX_RETRIES.
        Chunked::send_snapshot(
            &mut net,
            Vote::new(1, 0),
            Snapshot::<UTConfig>::new(
                SnapshotMeta {
                    last_log_id: None,
                    last_membership: StoredMembership::default(),
                    snapshot_id: "1-1-1-8".to_string(),
                },
                Box::new(Cursor::new(vec![1, 2])),
            ),
            cancel,
            opt,
        )
        .await
        .unwrap();

        let attempts_per_offset = (burst + 1) as usize;
        let mut expected = Vec::new();
        expected.extend(std::iter::repeat_n(0u64, attempts_per_offset));
        expected.extend(std::iter::repeat_n(1u64, attempts_per_offset));
        assert_eq!(net.received_offset, expected);
    }

    #[derive(Clone, Copy)]
    enum TransientKind {
        Timeout,
        Network,
        Unreachable,
    }

    struct ScriptedErrorNetwork {
        received_offset: Vec<u64>,
        /// Responses to emit when `install_snapshot` is called at `fail_offset`, in order.
        script: Vec<TransientKind>,
        fail_offset: u64,
    }

    impl<C> RaftNetwork<C> for ScriptedErrorNetwork
    where C: RaftTypeConfig<NodeId = u64>
    {
        async fn append_entries(
            &mut self,
            _rpc: AppendEntriesRequest<C>,
            _option: RPCOption,
        ) -> Result<AppendEntriesResponse<C::NodeId>, RPCError<C::NodeId, C::Node, RaftError<C::NodeId>>> {
            unimplemented!()
        }

        async fn vote(
            &mut self,
            _rpc: VoteRequest<C::NodeId>,
            _option: RPCOption,
        ) -> Result<VoteResponse<C::NodeId>, RPCError<C::NodeId, C::Node, RaftError<C::NodeId>>> {
            unimplemented!()
        }

        async fn full_snapshot(
            &mut self,
            _vote: Vote<C::NodeId>,
            _snapshot: Snapshot<C>,
            _cancel: impl Future<Output = ReplicationClosed> + OptionalSend + 'static,
            _option: RPCOption,
        ) -> Result<SnapshotResponse<C::NodeId>, StreamingError<C, Fatal<C::NodeId>>> {
            unimplemented!()
        }

        fn backoff(&self) -> Backoff {
            Backoff::new(std::iter::repeat(Duration::from_millis(1)))
        }

        async fn install_snapshot(
            &mut self,
            rpc: InstallSnapshotRequest<C>,
            _option: RPCOption,
        ) -> Result<
            InstallSnapshotResponse<C::NodeId>,
            RPCError<C::NodeId, C::Node, RaftError<C::NodeId, InstallSnapshotError>>,
        > {
            self.received_offset.push(rpc.offset);

            if rpc.offset != self.fail_offset || self.script.is_empty() {
                return Ok(InstallSnapshotResponse { vote: rpc.vote });
            }

            Err(match self.script.remove(0) {
                TransientKind::Timeout => RPCError::Timeout(Timeout {
                    action: RPCTypes::InstallSnapshot,
                    id: 0,
                    target: 1,
                    timeout: Duration::from_millis(50),
                }),
                TransientKind::Network => {
                    let any_err = AnyError::error("inject network");
                    RPCError::Network(NetworkError::new(&any_err))
                }
                TransientKind::Unreachable => {
                    let any_err = AnyError::error("inject unreachable");
                    RPCError::Unreachable(Unreachable::new(&any_err))
                }
            })
        }
    }

    /// The retry budget counts every transient variant uniformly. Mixing `Timeout`,
    /// `Network`, and `Unreachable` errors still bails out at exactly
    /// `SNAPSHOT_CHUNK_MAX_RETRIES` consecutive failures.
    #[tokio::test]
    async fn test_chunked_retry_budget_is_universal_across_transient_variants() {
        let script = vec![
            TransientKind::Timeout,
            TransientKind::Network,
            TransientKind::Unreachable,
            TransientKind::Timeout,
            TransientKind::Network,
        ];
        assert_eq!(script.len() as u64, SNAPSHOT_CHUNK_MAX_RETRIES);

        let mut net = ScriptedErrorNetwork {
            received_offset: vec![],
            script,
            fail_offset: 1,
        };

        let mut opt = RPCOption::new(Duration::from_millis(100));
        opt.snapshot_chunk_size = Some(1);
        let cancel = futures::future::pending();

        let err = Chunked::send_snapshot(
            &mut net,
            Vote::new(1, 0),
            Snapshot::<UTConfig>::new(
                SnapshotMeta {
                    last_log_id: None,
                    last_membership: StoredMembership::default(),
                    snapshot_id: "1-1-1-9".to_string(),
                },
                Box::new(Cursor::new(vec![1, 2, 3])),
            ),
            cancel,
            opt,
        )
        .await
        .unwrap_err();

        // The last scripted variant is Network, so the returned StreamingError variant matches.
        assert!(
            matches!(err, StreamingError::Network(_)),
            "last variant was Network, got {err:?}"
        );

        let mut expected = vec![0];
        expected.extend(std::iter::repeat_n(1u64, SNAPSHOT_CHUNK_MAX_RETRIES as usize));
        assert_eq!(net.received_offset, expected);
    }
}