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
//! Replication stream.

mod replication_session_id;
mod response;

use std::fmt;
use std::io::SeekFrom;
use std::sync::Arc;

use anyerror::AnyError;
use futures::future::FutureExt;
pub(crate) use replication_session_id::ReplicationSessionId;
pub(crate) use response::Response;
use tokio::io::AsyncReadExt;
use tokio::io::AsyncSeekExt;
use tokio::select;
use tokio::sync::mpsc;
use tokio::sync::oneshot;
use tokio::task::JoinHandle;
use tokio::time::sleep;
use tokio::time::timeout;
use tokio::time::Duration;
use tokio::time::Instant;
use tracing_futures::Instrument;

use crate::config::Config;
use crate::core::notify::Notify;
use crate::display_ext::DisplayOption;
use crate::display_ext::DisplayOptionExt;
use crate::error::HigherVote;
use crate::error::RPCError;
use crate::error::ReplicationClosed;
use crate::error::ReplicationError;
use crate::error::Timeout;
use crate::log_id::LogIdOptionExt;
use crate::log_id_range::LogIdRange;
use crate::network::Backoff;
use crate::network::RPCOption;
use crate::network::RPCTypes;
use crate::network::RaftNetwork;
use crate::network::RaftNetworkFactory;
use crate::raft::AppendEntriesRequest;
use crate::raft::AppendEntriesResponse;
use crate::raft::InstallSnapshotRequest;
use crate::storage::RaftLogReader;
use crate::storage::RaftLogStorage;
use crate::storage::Snapshot;
use crate::utime::UTime;
use crate::ErrorSubject;
use crate::ErrorVerb;
use crate::LogId;
use crate::MessageSummary;
use crate::NodeId;
use crate::RaftTypeConfig;
use crate::StorageError;
use crate::StorageIOError;
use crate::ToStorageResult;

/// The handle to a spawned replication stream.
pub(crate) struct ReplicationHandle<C>
where C: RaftTypeConfig
{
    /// The spawn handle the `ReplicationCore` task.
    pub(crate) join_handle: JoinHandle<Result<(), ReplicationClosed>>,

    /// The channel used for communicating with the replication task.
    pub(crate) tx_repl: mpsc::UnboundedSender<Replicate<C>>,
}

/// A task responsible for sending replication events to a target follower in the Raft cluster.
///
/// NOTE: we do not stack replication requests to targets because this could result in
/// out-of-order delivery. We always buffer until we receive a success response, then send the
/// next payload from the buffer.
pub(crate) struct ReplicationCore<C, N, LS>
where
    C: RaftTypeConfig,
    N: RaftNetworkFactory<C>,
    LS: RaftLogStorage<C>,
{
    /// The ID of the target Raft node which replication events are to be sent to.
    target: C::NodeId,

    /// Identifies which session this replication belongs to.
    session_id: ReplicationSessionId<C::NodeId>,

    /// A channel for sending events to the RaftCore.
    #[allow(clippy::type_complexity)]
    tx_raft_core: mpsc::UnboundedSender<Notify<C>>,

    /// A channel for receiving events from the RaftCore.
    rx_repl: mpsc::UnboundedReceiver<Replicate<C>>,

    /// The `RaftNetwork` interface.
    network: N::Network,

    /// The backoff policy if an [`Unreachable`](`crate::error::Unreachable`) error is returned.
    /// It will be reset to `None` when an successful response is received.
    backoff: Option<Backoff>,

    /// The `RaftLogReader` of a `RaftStorage` interface.
    log_reader: LS::LogReader,

    /// The Raft's runtime config.
    config: Arc<Config>,

    /// The log id of the highest log entry which is known to be committed in the cluster.
    committed: Option<LogId<C::NodeId>>,

    /// Last matching log id on a follower/learner
    matching: Option<LogId<C::NodeId>>,

    /// Next replication action to run.
    next_action: Option<Data<C>>,
}

impl<C, N, LS> ReplicationCore<C, N, LS>
where
    C: RaftTypeConfig,
    N: RaftNetworkFactory<C>,
    LS: RaftLogStorage<C>,
{
    /// Spawn a new replication task for the target node.
    #[tracing::instrument(level = "trace", skip_all,fields(target=display(target), session_id=display(session_id)))]
    #[allow(clippy::type_complexity)]
    #[allow(clippy::too_many_arguments)]
    pub(crate) fn spawn(
        target: C::NodeId,
        session_id: ReplicationSessionId<C::NodeId>,
        config: Arc<Config>,
        committed: Option<LogId<C::NodeId>>,
        matching: Option<LogId<C::NodeId>>,
        network: N::Network,
        log_reader: LS::LogReader,
        tx_raft_core: mpsc::UnboundedSender<Notify<C>>,
        span: tracing::Span,
    ) -> ReplicationHandle<C> {
        tracing::debug!(
            session_id = display(&session_id),
            target = display(&target),
            committed = display(committed.summary()),
            matching = debug(&matching),
            "spawn replication"
        );

        // other component to ReplicationStream
        let (tx_repl, rx_repl) = mpsc::unbounded_channel();

        let this = Self {
            target,
            session_id,
            network,
            backoff: None,
            log_reader,
            config,
            committed,
            matching,
            tx_raft_core,
            rx_repl,
            next_action: None,
        };

        let join_handle = tokio::spawn(this.main().instrument(span));

        ReplicationHandle { join_handle, tx_repl }
    }

    #[tracing::instrument(level="debug", skip(self), fields(session=%self.session_id, target=display(self.target), cluster=%self.config.cluster_name))]
    async fn main(mut self) -> Result<(), ReplicationClosed> {
        loop {
            let action = self.next_action.take();

            let mut repl_id = None;

            let res = match action {
                None => Ok(None),
                Some(d) => {
                    tracing::debug!(replication_data = display(&d), "{} send replication RPC", func_name!());

                    repl_id = d.request_id;

                    match d.payload {
                        Payload::Logs(log_id_range) => self.send_log_entries(d.request_id, log_id_range).await,
                        Payload::Snapshot(snapshot_rx) => self.stream_snapshot(d.request_id, snapshot_rx).await,
                    }
                }
            };

            tracing::debug!(res = debug(&res), "replication action done");

            match res {
                Ok(next) => {
                    // reset backoff
                    self.backoff = None;

                    // If the RPC was successful but not finished, continue.
                    if let Some(next) = next {
                        self.next_action = Some(next);
                    }
                }
                Err(err) => {
                    tracing::warn!(error=%err, "error replication to target={}", self.target);

                    match err {
                        ReplicationError::Closed(closed) => {
                            return Err(closed);
                        }
                        ReplicationError::HigherVote(h) => {
                            let _ = self.tx_raft_core.send(Notify::Network {
                                response: Response::HigherVote {
                                    target: self.target,
                                    higher: h.higher,
                                    vote: self.session_id.vote,
                                },
                            });
                            return Ok(());
                        }
                        ReplicationError::StorageError(error) => {
                            tracing::error!(error=%error, "error replication to target={}", self.target);

                            // TODO: report this error
                            let _ = self.tx_raft_core.send(Notify::Network {
                                response: Response::StorageError { error },
                            });
                            return Ok(());
                        }
                        ReplicationError::RPCError(err) => {
                            tracing::error!(err = display(&err), "RPCError");

                            if let Some(request_id) = repl_id {
                                let _ = self.tx_raft_core.send(Notify::Network {
                                    response: Response::Progress {
                                        target: self.target,
                                        request_id,
                                        result: Err(err.to_string()),
                                        session_id: self.session_id,
                                    },
                                });
                            } else {
                                tracing::warn!(
                                    err = display(&err),
                                    "encountered RPCError but request_id is None, no response is sent"
                                );
                            }

                            // If there is an [`Unreachable`] error, we will backoff for a period of time
                            // Backoff will be reset if there is a successful RPC is sent.
                            if let RPCError::Unreachable(_unreachable) = err {
                                if self.backoff.is_none() {
                                    self.backoff = Some(self.network.backoff());
                                }
                            }
                        }
                    };
                }
            };

            if let Some(b) = &mut self.backoff {
                let duration = b.next().unwrap_or_else(|| {
                    tracing::warn!("backoff exhausted, using default");
                    Duration::from_millis(500)
                });

                self.backoff_drain_events(Instant::now() + duration).await?;
            }

            self.drain_events().await?;
        }
    }

    /// Send an AppendEntries RPC to the target.
    ///
    /// This request will timeout if no response is received within the
    /// configured heartbeat interval.
    ///
    /// If an RPC is made but not completely finished, it returns the next action expected to do.
    #[tracing::instrument(level = "debug", skip_all)]
    async fn send_log_entries(
        &mut self,
        request_id: Option<u64>,
        log_id_range: LogIdRange<C::NodeId>,
    ) -> Result<Option<Data<C>>, ReplicationError<C::NodeId, C::Node>> {
        tracing::debug!(
            request_id = display(request_id.display()),
            log_id_range = display(&log_id_range),
            "send_log_entries",
        );

        let start = log_id_range.prev_log_id.next_index();
        let end = log_id_range.last_log_id.next_index();

        let logs = if start == end {
            vec![]
        } else {
            let logs = self.log_reader.try_get_log_entries(start..end).await?;
            debug_assert_eq!(
                logs.len(),
                (end - start) as usize,
                "expect logs {}..{} but got only {} entries",
                start,
                end,
                logs.len()
            );
            logs
        };

        let leader_time = Instant::now();

        // Build the heartbeat frame to be sent to the follower.
        let payload = AppendEntriesRequest {
            vote: self.session_id.vote,
            prev_log_id: log_id_range.prev_log_id,
            leader_commit: self.committed,
            entries: logs,
        };

        // Send the payload.
        tracing::debug!(
            payload=%payload.summary(),
            now = debug(leader_time),
            "start sending append_entries, timeout: {:?}",
            self.config.heartbeat_interval
        );

        let the_timeout = Duration::from_millis(self.config.heartbeat_interval);
        let option = RPCOption::new(the_timeout);
        let res = timeout(the_timeout, self.network.append_entries(payload, option)).await;

        tracing::debug!("append_entries res: {:?}", res);

        let append_res = res.map_err(|_e| {
            let to = Timeout {
                action: RPCTypes::AppendEntries,
                id: self.session_id.vote.leader_id().voted_for().unwrap(),
                target: self.target,
                timeout: the_timeout,
            };
            RPCError::Timeout(to)
        })?;
        let append_resp = append_res?;

        tracing::debug!(
            req = display(&log_id_range),
            resp = display(&append_resp),
            "append_entries resp"
        );

        match append_resp {
            AppendEntriesResponse::Success => {
                self.update_matching(request_id, leader_time, log_id_range.last_log_id);
                Ok(None)
            }
            AppendEntriesResponse::PartialSuccess(matching) => {
                debug_assert!(
                    matching <= log_id_range.last_log_id,
                    "matching ({}) should be <= last_log_id ({})",
                    matching.display(),
                    log_id_range.last_log_id.display()
                );
                debug_assert!(
                    matching.index() <= log_id_range.last_log_id.index(),
                    "matching.index ({}) should be <= last_log_id.index ({})",
                    matching.index().display(),
                    log_id_range.last_log_id.index().display()
                );
                debug_assert!(
                    matching >= log_id_range.prev_log_id,
                    "matching ({}) should be >= prev_log_id ({})",
                    matching.display(),
                    log_id_range.prev_log_id.display()
                );
                debug_assert!(
                    matching.index() >= log_id_range.prev_log_id.index(),
                    "matching.index ({}) should be >= prev_log_id.index ({})",
                    matching.index().display(),
                    log_id_range.prev_log_id.index().display()
                );

                self.update_matching(request_id, leader_time, matching);
                if matching < log_id_range.last_log_id {
                    // TODO(9): an RPC has already been made, it should use a newer time
                    Ok(Some(Data::new_logs(
                        request_id,
                        LogIdRange::new(matching, log_id_range.last_log_id),
                    )))
                } else {
                    Ok(None)
                }
            }
            AppendEntriesResponse::HigherVote(vote) => {
                debug_assert!(
                    vote > self.session_id.vote,
                    "higher vote({}) should be greater than leader's vote({})",
                    vote,
                    self.session_id.vote,
                );
                tracing::debug!(%vote, "append entries failed. converting to follower");

                Err(ReplicationError::HigherVote(HigherVote {
                    higher: vote,
                    mine: self.session_id.vote,
                }))
            }
            AppendEntriesResponse::Conflict => {
                let conflict = log_id_range.prev_log_id;
                debug_assert!(conflict.is_some(), "prev_log_id=None never conflict");

                let conflict = conflict.unwrap();
                self.update_conflicting(request_id, leader_time, conflict);

                Ok(None)
            }
        }
    }

    fn update_conflicting(&mut self, request_id: Option<u64>, leader_time: Instant, conflict: LogId<C::NodeId>) {
        tracing::debug!(
            target = display(self.target),
            request_id = display(request_id.display()),
            conflict = display(&conflict),
            "update_conflicting"
        );

        if let Some(request_id) = request_id {
            let _ = self.tx_raft_core.send({
                Notify::Network {
                    response: Response::Progress {
                        session_id: self.session_id,
                        request_id,
                        target: self.target,
                        result: Ok(UTime::new(leader_time, ReplicationResult::Conflict(conflict))),
                    },
                }
            });
        } else {
            tracing::info!(
                target = display(self.target),
                request_id = display(request_id.display()),
                conflict = display(&conflict),
                "replication conflict, but request_id is None, no response is sent to RaftCore"
            )
        }
    }

    /// Update the `matching` log id, which is for tracking follower replication, and report the
    /// matched log id to `RaftCore` to commit an entry.
    #[tracing::instrument(level = "trace", skip(self))]
    fn update_matching(
        &mut self,
        request_id: Option<u64>,
        leader_time: Instant,
        new_matching: Option<LogId<C::NodeId>>,
    ) {
        tracing::debug!(
            request_id = display(request_id.display()),
            target = display(self.target),
            curr_matching = display(DisplayOption(&self.matching)),
            new_matching = display(DisplayOption(&new_matching)),
            "{}",
            func_name!()
        );

        debug_assert!(self.matching <= new_matching);

        self.matching = new_matching;

        if let Some(request_id) = request_id {
            let _ = self.tx_raft_core.send({
                Notify::Network {
                    response: Response::Progress {
                        session_id: self.session_id,
                        request_id,
                        target: self.target,
                        result: Ok(UTime::new(leader_time, ReplicationResult::Matching(new_matching))),
                    },
                }
            });
        }
    }

    /// Drain all events in the channel in backoff mode, i.e., there was an un-retry-able error and
    /// should not send out anything before backoff interval expired.
    ///
    /// In the backoff period, we should not send out any RPCs, but we should still receive events,
    /// in case the channel is closed, it should quit at once.
    #[tracing::instrument(level = "debug", skip(self))]
    pub async fn backoff_drain_events(&mut self, until: Instant) -> Result<(), ReplicationClosed> {
        let d = until - Instant::now();
        tracing::warn!(
            interval = debug(d),
            "{} backoff mode: drain events without processing them",
            func_name!()
        );

        loop {
            let sleep_duration = until - Instant::now();
            let sleep = sleep(sleep_duration);

            let recv = self.rx_repl.recv();

            tracing::debug!("backoff timeout: {:?}", sleep_duration);

            select! {
                _ = sleep => {
                    tracing::debug!("backoff timeout");
                    return Ok(());
                }
                recv_res = recv => {
                    let event = recv_res.ok_or(ReplicationClosed{})?;
                    self.process_event(event);
                }
            }
        }
    }

    /// Receive and process events from RaftCore, until `next_action` is filled.
    ///
    /// It blocks until at least one event is received.
    #[tracing::instrument(level = "trace", skip_all)]
    pub async fn drain_events(&mut self) -> Result<(), ReplicationClosed> {
        tracing::debug!("drain_events");

        // If there is next action to run, do not block waiting for events,
        // instead, just try the best to drain all events.
        if self.next_action.is_none() {
            let event = self.rx_repl.recv().await.ok_or(ReplicationClosed {})?;
            self.process_event(event);
        }

        self.try_drain_events().await?;

        // No action filled after all events are processed, fill in an action to send committed
        // index.
        if self.next_action.is_none() {
            let m = &self.matching;

            // empty message, just for syncing the committed index
            self.next_action = Some(Data {
                // request_id==None will be ignored by RaftCore.
                request_id: None,
                payload: Payload::Logs(LogIdRange {
                    prev_log_id: *m,
                    last_log_id: *m,
                }),
            });
        }

        Ok(())
    }

    #[tracing::instrument(level = "trace", skip(self))]
    pub async fn try_drain_events(&mut self) -> Result<(), ReplicationClosed> {
        tracing::debug!("{}", func_name!());

        // Just drain all events in the channel.
        // There should NOT be more than one `Replicate::Data` event in the channel.
        // Looping it just collect all commit events and heartbeat events.
        loop {
            let maybe_res = self.rx_repl.recv().now_or_never();

            let recv_res = match maybe_res {
                None => {
                    // No more events in self.repl_rx
                    return Ok(());
                }
                Some(x) => x,
            };

            let event = recv_res.ok_or(ReplicationClosed {})?;

            self.process_event(event);
        }
    }

    #[tracing::instrument(level = "trace", skip_all)]
    pub fn process_event(&mut self, event: Replicate<C>) {
        tracing::debug!(event=%event.summary(), "process_event");

        match event {
            Replicate::Committed(c) => {
                // RaftCore may send a committed equals to the initial value.
                debug_assert!(
                    c >= self.committed,
                    "expect new committed {} > self.committed {}",
                    c.summary(),
                    self.committed.summary()
                );

                self.committed = c;
            }
            Replicate::Heartbeat => {
                // Nothing to do. Heartbeat message is just for waking up replication to send
                // something: When all messages are drained,
                // - if self.next_action is None, it sends an empty AppendEntries request as
                //   heartbeat.
                //-  If self.next_action is not None, next_action will serve as a heartbeat.
            }
            Replicate::Data(d) => {
                // TODO: Currently there is at most 1 in flight data. But in future RaftCore may send next data
                //       actions without waiting for the previous to finish.
                debug_assert!(self.next_action.is_none(), "there can not be two data action in flight");
                self.next_action = Some(d);
            }
        }
    }

    #[tracing::instrument(level = "info", skip_all)]
    async fn stream_snapshot(
        &mut self,
        request_id: Option<u64>,
        rx: oneshot::Receiver<Option<Snapshot<C>>>,
    ) -> Result<Option<Data<C>>, ReplicationError<C::NodeId, C::Node>> {
        tracing::info!(request_id = display(request_id.display()), "{}", func_name!());

        let snapshot = rx.await.map_err(|e| {
            let io_err = StorageIOError::read_snapshot(None, AnyError::error(e));
            StorageError::IO { source: io_err }
        })?;

        tracing::info!(
            "received snapshot: request_id={}; meta:{}",
            request_id.display(),
            snapshot.as_ref().map(|x| &x.meta).summary()
        );

        let mut snapshot = match snapshot {
            None => {
                let io_err = StorageIOError::read_snapshot(None, AnyError::error("snapshot not found"));
                let sto_err = StorageError::IO { source: io_err };
                return Err(ReplicationError::StorageError(sto_err));
            }
            Some(x) => x,
        };

        let err_x = || (ErrorSubject::Snapshot(Some(snapshot.meta.signature())), ErrorVerb::Read);

        let mut offset = 0;
        let end = snapshot.snapshot.seek(SeekFrom::End(0)).await.sto_res(err_x)?;
        let mut buf = Vec::with_capacity(self.config.snapshot_max_chunk_size as usize);

        loop {
            // Build the RPC.
            snapshot.snapshot.seek(SeekFrom::Start(offset)).await.sto_res(err_x)?;
            let n_read = snapshot.snapshot.read_buf(&mut buf).await.sto_res(err_x)?;

            let leader_time = Instant::now();

            let done = (offset + n_read as u64) == end;
            let req = InstallSnapshotRequest {
                vote: self.session_id.vote,
                meta: snapshot.meta.clone(),
                offset,
                data: Vec::from(&buf[..n_read]),
                done,
            };
            buf.clear();

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

            let snap_timeout = if done {
                self.config.install_snapshot_timeout()
            } else {
                self.config.send_snapshot_timeout()
            };

            let option = RPCOption::new(snap_timeout);

            let res = timeout(snap_timeout, self.network.install_snapshot(req, option)).await;

            let res = match res {
                Ok(outer_res) => match outer_res {
                    Ok(res) => res,
                    Err(err) => {
                        tracing::warn!(error=%err, "error sending InstallSnapshot RPC to target");

                        // If sender is closed, return at once
                        self.try_drain_events().await?;

                        // Sleep a short time otherwise in test environment it is a dead-loop that
                        // never yields. Because network implementation does
                        // not yield.
                        sleep(Duration::from_millis(10)).await;
                        continue;
                    }
                },
                Err(err) => {
                    // TODO(2): add backoff when Unreachable is returned
                    tracing::warn!(error=%err, "timeout while sending InstallSnapshot RPC to target");

                    // If sender is closed, return at once
                    self.try_drain_events().await?;

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

            // Handle response conditions.
            if res.vote > self.session_id.vote {
                return Err(ReplicationError::HigherVote(HigherVote {
                    higher: res.vote,
                    mine: self.session_id.vote,
                }));
            }

            // If we just sent the final chunk of the snapshot, then transition to lagging state.
            if done {
                tracing::debug!(
                    "done install snapshot: snapshot last_log_id: {:?}, matching: {}",
                    snapshot.meta.last_log_id,
                    self.matching.summary(),
                );

                // TODO: update leader lease for every successfully sent chunk.
                self.update_matching(request_id, leader_time, snapshot.meta.last_log_id);

                return Ok(None);
            }

            // Everything is good, so update offset for sending the next chunk.
            offset += n_read as u64;

            // Check raft channel to ensure we are staying up-to-date, then loop.
            self.try_drain_events().await?;
        }
    }
}

/// Request to replicate a chunk of data, logs or snapshot.
///
/// It defines what data to send to a follower/learner and an id to identify who is sending this
/// data.
#[derive(Debug)]
pub(crate) struct Data<C>
where C: RaftTypeConfig
{
    /// The id of this replication request.
    ///
    /// A replication request without an id does not need to send a reply to the caller.
    request_id: Option<u64>,

    payload: Payload<C>,
}

impl<C: RaftTypeConfig> fmt::Display for Data<C> {
    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
        write!(f, "{{id: {}, payload: {}}}", self.request_id.display(), self.payload)
    }
}

impl<C> MessageSummary<Data<C>> for Data<C>
where C: RaftTypeConfig
{
    fn summary(&self) -> String {
        match &self.payload {
            Payload::Logs(log_id_range) => {
                format!("Logs{{request_id={}, {}}}", self.request_id.display(), log_id_range)
            }
            Payload::Snapshot(_) => {
                format!("Snapshot{{request_id={}}}", self.request_id.display())
            }
        }
    }
}

impl<C> Data<C>
where C: RaftTypeConfig
{
    fn new_logs(request_id: Option<u64>, log_id_range: LogIdRange<C::NodeId>) -> Self {
        Self {
            request_id,
            payload: Payload::Logs(log_id_range),
        }
    }

    fn new_snapshot(request_id: Option<u64>, snapshot_rx: oneshot::Receiver<Option<Snapshot<C>>>) -> Self {
        Self {
            request_id,
            payload: Payload::Snapshot(snapshot_rx),
        }
    }
}

/// The data to replication.
///
/// Either a series of logs or a snapshot.
pub(crate) enum Payload<C>
where C: RaftTypeConfig
{
    Logs(LogIdRange<C::NodeId>),
    Snapshot(oneshot::Receiver<Option<Snapshot<C>>>),
}

impl<C> fmt::Display for Payload<C>
where C: RaftTypeConfig
{
    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
        match self {
            Self::Logs(log_id_range) => {
                write!(f, "Logs({})", log_id_range)
            }
            Self::Snapshot(_) => {
                write!(f, "Snapshot()")
            }
        }
    }
}

impl<C> fmt::Debug for Payload<C>
where C: RaftTypeConfig
{
    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
        match self {
            Self::Logs(log_id_range) => {
                write!(f, "Logs({})", log_id_range)
            }
            Self::Snapshot(_) => {
                write!(f, "Snapshot()")
            }
        }
    }
}

/// Result of an replication action.
#[derive(Clone, Debug)]
pub(crate) enum ReplicationResult<NID: NodeId> {
    Matching(Option<LogId<NID>>),
    Conflict(LogId<NID>),
}

/// A replication request sent by RaftCore leader state to replication stream.
pub(crate) enum Replicate<C>
where C: RaftTypeConfig
{
    /// Inform replication stream to forward the committed log id to followers/learners.
    Committed(Option<LogId<C::NodeId>>),

    /// Send an empty AppendEntries RPC as heartbeat.
    Heartbeat,

    /// Send a chunk of data, e.g., logs or snapshot.
    Data(Data<C>),
}

impl<C> Replicate<C>
where C: RaftTypeConfig
{
    pub(crate) fn logs(id: Option<u64>, log_id_range: LogIdRange<C::NodeId>) -> Self {
        Self::Data(Data::new_logs(id, log_id_range))
    }

    pub(crate) fn snapshot(id: Option<u64>, snapshot_rx: oneshot::Receiver<Option<Snapshot<C>>>) -> Self {
        Self::Data(Data::new_snapshot(id, snapshot_rx))
    }
}

impl<C> MessageSummary<Replicate<C>> for Replicate<C>
where C: RaftTypeConfig
{
    fn summary(&self) -> String {
        match self {
            Replicate::Committed(c) => {
                format!("Replicate::Committed: {:?}", c)
            }
            Replicate::Heartbeat => "Replicate::Heartbeat".to_string(),
            Replicate::Data(d) => {
                format!("Replicate::Data({})", d.summary())
            }
        }
    }
}