openraft 0.9.22

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
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549
1550
1551
1552
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571
1572
1573
1574
1575
1576
1577
1578
1579
1580
1581
1582
1583
1584
1585
1586
1587
1588
1589
1590
1591
1592
1593
1594
1595
1596
1597
1598
1599
1600
1601
1602
1603
1604
1605
1606
1607
1608
1609
1610
1611
1612
1613
1614
1615
1616
1617
1618
1619
1620
1621
1622
1623
1624
1625
1626
1627
1628
1629
1630
1631
1632
1633
1634
1635
1636
1637
1638
1639
1640
1641
1642
1643
1644
1645
1646
1647
1648
1649
1650
1651
1652
1653
1654
1655
1656
1657
1658
1659
1660
1661
1662
1663
1664
1665
1666
1667
1668
1669
1670
1671
1672
1673
1674
1675
1676
1677
1678
1679
1680
1681
1682
1683
1684
1685
1686
1687
1688
1689
1690
1691
1692
1693
1694
1695
1696
1697
1698
1699
1700
1701
1702
1703
1704
1705
1706
1707
1708
1709
1710
1711
1712
1713
1714
1715
1716
1717
1718
1719
1720
1721
1722
1723
1724
1725
1726
1727
use std::borrow::Borrow;
use std::collections::BTreeMap;
use std::fmt::Debug;
use std::fmt::Display;
use std::fmt::Formatter;
use std::marker::PhantomData;
use std::sync::atomic::Ordering;
use std::sync::Arc;
use std::time::Duration;

use anyerror::AnyError;
use futures::stream::FuturesUnordered;
use futures::StreamExt;
use futures::TryFutureExt;
use maplit::btreeset;
use tokio::select;
use tokio::sync::mpsc;
use tokio::sync::watch;
use tracing::Instrument;
use tracing::Level;
use tracing::Span;

use crate::async_runtime::AsyncOneshotSendExt;
use crate::config::Config;
use crate::config::RuntimeConfig;
use crate::core::balancer::Balancer;
use crate::core::command_state::CommandState;
use crate::core::notify::Notify;
use crate::core::raft_msg::external_command::ExternalCommand;
use crate::core::raft_msg::AppendEntriesTx;
use crate::core::raft_msg::ClientReadTx;
use crate::core::raft_msg::RaftMsg;
use crate::core::raft_msg::ResultSender;
use crate::core::raft_msg::VoteTx;
use crate::core::sm;
use crate::core::sm::handle;
use crate::core::sm::CommandSeq;
use crate::core::ServerState;
use crate::display_ext::DisplayInstantExt;
use crate::display_ext::DisplayOption;
use crate::display_ext::DisplayOptionExt;
use crate::display_ext::DisplaySlice;
use crate::engine::Command;
use crate::engine::Condition;
use crate::engine::Engine;
use crate::engine::Respond;
use crate::entry::FromAppData;
use crate::entry::RaftEntry;
use crate::error::ClientWriteError;
use crate::error::Fatal;
use crate::error::ForwardToLeader;
use crate::error::Infallible;
use crate::error::InitializeError;
use crate::error::QuorumNotEnough;
use crate::error::RPCError;
use crate::error::Timeout;
use crate::log_id::LogIdOptionExt;
use crate::log_id::RaftLogId;
use crate::metrics::RaftDataMetrics;
use crate::metrics::RaftMetrics;
use crate::metrics::RaftServerMetrics;
use crate::metrics::ReplicationMetrics;
use crate::network::RPCOption;
use crate::network::RPCTypes;
use crate::network::RaftNetwork;
use crate::network::RaftNetworkFactory;
use crate::progress::entry::ProgressEntry;
use crate::progress::Inflight;
use crate::progress::Progress;
use crate::quorum::QuorumSet;
use crate::raft::responder::Responder;
use crate::raft::AppendEntriesRequest;
use crate::raft::AppendEntriesResponse;
use crate::raft::ClientWriteResponse;
use crate::raft::VoteRequest;
use crate::raft::VoteResponse;
use crate::raft_state::LogIOId;
use crate::raft_state::LogStateReader;
use crate::replication;
use crate::replication::request::Replicate;
use crate::replication::request_id::RequestId;
use crate::replication::response::ReplicationResult;
use crate::replication::ReplicationCore;
use crate::replication::ReplicationHandle;
use crate::replication::ReplicationSessionId;
use crate::runtime::RaftRuntime;
use crate::storage::LogFlushed;
use crate::storage::RaftLogReaderExt;
use crate::storage::RaftLogStorage;
use crate::storage::RaftStateMachine;
use crate::type_config::alias::InstantOf;
use crate::type_config::alias::ResponderOf;
use crate::type_config::TypeConfigExt;
use crate::AsyncRuntime;
use crate::ChangeMembers;
use crate::Instant;
use crate::LogId;
use crate::Membership;
use crate::MessageSummary;
use crate::Node;
use crate::NodeId;
use crate::OptionalSend;
use crate::RaftTypeConfig;
use crate::StorageError;
use crate::StorageIOError;
use crate::Vote;

/// A temp struct to hold the data for a node that is being applied.
#[derive(Debug)]
pub(crate) struct ApplyingEntry<NID: NodeId, N: Node> {
    log_id: LogId<NID>,
    membership: Option<Membership<NID, N>>,
}

impl<NID: NodeId, N: Node> ApplyingEntry<NID, N> {
    pub(crate) fn new(log_id: LogId<NID>, membership: Option<Membership<NID, N>>) -> Self {
        Self { log_id, membership }
    }
}

/// The result of applying log entries to state machine.
pub(crate) struct ApplyResult<C: RaftTypeConfig> {
    pub(crate) since: u64,
    pub(crate) end: u64,
    pub(crate) last_applied: LogId<C::NodeId>,
    pub(crate) applying_entries: Vec<ApplyingEntry<C::NodeId, C::Node>>,
    pub(crate) apply_results: Vec<C::R>,
}

impl<C: RaftTypeConfig> Debug for ApplyResult<C> {
    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("ApplyResult")
            .field("since", &self.since)
            .field("end", &self.end)
            .field("last_applied", &self.last_applied)
            .finish()
    }
}

/// Data for a Leader.
///
/// It is created when RaftCore enters leader state, and will be dropped when it quits leader state.
pub(crate) struct LeaderData<C: RaftTypeConfig> {
    /// The time to send next heartbeat.
    pub(crate) next_heartbeat: <C::AsyncRuntime as AsyncRuntime>::Instant,
}

impl<C: RaftTypeConfig> LeaderData<C> {
    pub(crate) fn new() -> Self {
        Self {
            next_heartbeat: C::now(),
        }
    }
}

// TODO: remove SM
/// The core type implementing the Raft protocol.
pub struct RaftCore<C, N, LS, SM>
where
    C: RaftTypeConfig,
    N: RaftNetworkFactory<C>,
    LS: RaftLogStorage<C>,
    SM: RaftStateMachine<C>,
{
    /// This node's ID.
    pub(crate) id: C::NodeId,

    /// This node's runtime config.
    pub(crate) config: Arc<Config>,

    pub(crate) runtime_config: Arc<RuntimeConfig>,

    /// The `RaftNetworkFactory` implementation.
    pub(crate) network: N,

    /// The [`RaftLogStorage`] implementation.
    pub(crate) log_store: LS,

    /// A controlling handle to the [`RaftStateMachine`] worker.
    pub(crate) sm_handle: handle::Handle<C>,

    pub(crate) engine: Engine<C>,

    /// Channels to send result back to client when logs are applied.
    pub(crate) client_resp_channels: BTreeMap<u64, ResponderOf<C>>,

    /// A mapping of node IDs the replication state of the target node.
    pub(crate) replications: BTreeMap<C::NodeId, ReplicationHandle<C>>,

    pub(crate) leader_data: Option<LeaderData<C>>,

    #[allow(dead_code)]
    pub(crate) tx_api: mpsc::UnboundedSender<RaftMsg<C>>,
    pub(crate) rx_api: mpsc::UnboundedReceiver<RaftMsg<C>>,

    /// A Sender to send callback by other components to [`RaftCore`], when an action is finished,
    /// such as flushing log to disk, or applying log entries to state machine.
    pub(crate) tx_notify: mpsc::UnboundedSender<Notify<C>>,

    /// A Receiver to receive callback from other components.
    pub(crate) rx_notify: mpsc::UnboundedReceiver<Notify<C>>,

    pub(crate) tx_metrics: watch::Sender<RaftMetrics<C::NodeId, C::Node>>,
    pub(crate) tx_data_metrics: watch::Sender<RaftDataMetrics<C::NodeId>>,
    pub(crate) tx_server_metrics: watch::Sender<RaftServerMetrics<C::NodeId, C::Node>>,

    pub(crate) command_state: CommandState,

    pub(crate) span: Span,

    pub(crate) _p: PhantomData<SM>,
}

impl<C, N, LS, SM> RaftCore<C, N, LS, SM>
where
    C: RaftTypeConfig,
    N: RaftNetworkFactory<C>,
    LS: RaftLogStorage<C>,
    SM: RaftStateMachine<C>,
{
    /// The main loop of the Raft protocol.
    pub(crate) async fn main(
        mut self,
        rx_shutdown: <C::AsyncRuntime as AsyncRuntime>::OneshotReceiver<()>,
    ) -> Result<Infallible, Fatal<C::NodeId>> {
        let span = tracing::span!(parent: &self.span, Level::DEBUG, "main");
        let res = self.do_main(rx_shutdown).instrument(span).await;

        // Flush buffered metrics
        self.report_metrics(None);

        // Safe unwrap: res is Result<Infallible, _>
        let err = res.unwrap_err();
        match err {
            Fatal::Stopped => { /* Normal quit */ }
            _ => {
                tracing::error!(error = display(&err), "quit RaftCore::main on error");
            }
        }

        tracing::debug!("update the metrics for shutdown");
        {
            let mut curr = self.tx_metrics.borrow().clone();
            curr.state = ServerState::Shutdown;
            curr.running_state = Err(err.clone());

            let _ = self.tx_metrics.send(curr);
        }

        tracing::info!("RaftCore shutdown complete");

        Err(err)
    }

    #[tracing::instrument(level="trace", skip_all, fields(id=display(&self.id), cluster=%self.config.cluster_name))]
    async fn do_main(
        &mut self,
        rx_shutdown: <C::AsyncRuntime as AsyncRuntime>::OneshotReceiver<()>,
    ) -> Result<Infallible, Fatal<C::NodeId>> {
        tracing::debug!("raft node is initializing");

        self.engine.startup();
        // It may not finish running all of the commands, if there is a command waiting for a callback.
        self.run_engine_commands().await?;

        // Initialize metrics.
        self.report_metrics(None);

        self.runtime_loop(rx_shutdown).await
    }

    /// Handle `is_leader` requests.
    ///
    /// Send heartbeat to all voters. We respond once we have
    /// a quorum of agreement.
    ///
    /// Why:
    /// To ensure linearizability, a read request proposed at time `T1` confirms this node's
    /// leadership to guarantee that all the committed entries proposed before `T1` are present in
    /// this node.
    // TODO: the second condition is such a read request can only read from state machine only when the last log it sees
    //       at `T1` is committed.
    #[tracing::instrument(level = "trace", skip(self, tx))]
    pub(super) async fn handle_check_is_leader_request(&mut self, tx: ClientReadTx<C>) {
        // Setup sentinel values to track when we've received majority confirmation of leadership.

        let resp = {
            let l = self.engine.leader_handler();
            let lh = match l {
                Ok(leading_handler) => leading_handler,
                Err(forward) => {
                    let _ = tx.send(Err(forward.into()));
                    return;
                }
            };

            let read_log_id = lh.get_read_log_id();

            // TODO: this applied is a little stale when being returned to client.
            //       Fix this when the following heartbeats are replaced with calling RaftNetwork.
            let applied = self.engine.state.io_applied().cloned();

            (read_log_id, applied)
        };

        let my_id = self.id.clone();
        let my_vote = self.engine.state.vote_ref().clone();
        let ttl = Duration::from_millis(self.config.heartbeat_interval);
        let eff_mem = self.engine.state.membership_state.effective().clone();
        let core_tx = self.tx_notify.clone();

        let mut granted = btreeset! {my_id.clone()};

        if eff_mem.is_quorum(granted.iter()) {
            let _ = tx.send(Ok(resp));
            return;
        }

        // Spawn parallel requests, all with the standard timeout for heartbeats.
        let mut pending = FuturesUnordered::new();

        let voter_progresses = {
            let l = &self.engine.leader.as_ref().unwrap();
            l.progress.iter().filter(|(id, _v)| l.progress.is_voter(id) == Some(true))
        };

        for (target, progress) in voter_progresses {
            let target = target.clone();

            if target == my_id {
                continue;
            }

            let rpc = AppendEntriesRequest {
                vote: my_vote.clone(),
                prev_log_id: progress.matching.clone(),
                entries: vec![],
                leader_commit: self.engine.state.committed().cloned(),
            };

            // Safe unwrap(): target is in membership
            let target_node = eff_mem.get_node(&target).unwrap().clone();
            let mut client = self.network.new_client(target.clone(), &target_node).await;

            let option = RPCOption::new(ttl);

            let fu = {
                let my_id = my_id.clone();
                let target = target.clone();
                async move {
                    let outer_res = C::AsyncRuntime::timeout(ttl, client.append_entries(rpc, option)).await;
                    match outer_res {
                        Ok(append_res) => match append_res {
                            Ok(x) => Ok((target.clone(), x)),
                            Err(err) => Err((target.clone(), err)),
                        },
                        Err(_timeout) => {
                            let timeout_err = Timeout {
                                action: RPCTypes::AppendEntries,
                                id: my_id,
                                target: target.clone(),
                                timeout: ttl,
                            };

                            Err((target, RPCError::Timeout(timeout_err)))
                        }
                    }
                }
            };

            let fu = fu.instrument(tracing::debug_span!("spawn_is_leader", target = target.to_string()));
            let task = C::AsyncRuntime::spawn(fu).map_err(move |err| (target, err));

            pending.push(task);
        }

        let waiting_fu = async move {
            // Handle responses as they return.
            while let Some(res) = pending.next().await {
                let (target, append_res) = match res {
                    Ok(Ok(res)) => res,
                    Ok(Err((target, err))) => {
                        tracing::error!(target=display(&target), error=%err, "timeout while confirming leadership for read request");
                        continue;
                    }
                    Err((target, err)) => {
                        tracing::error!(target = display(&target), "fail to join task: {}", err);
                        continue;
                    }
                };

                // If we receive a response with a greater vote, then revert to follower and abort this
                // request.
                if let AppendEntriesResponse::HigherVote(vote) = append_res {
                    debug_assert!(
                        vote > my_vote,
                        "committed vote({}) has total order relation with other votes({})",
                        my_vote,
                        vote
                    );

                    let send_res = core_tx.send(Notify::HigherVote {
                        target,
                        higher: vote,
                        sender_vote: my_vote,
                    });

                    if let Err(_e) = send_res {
                        tracing::error!("fail to send HigherVote to RaftCore");
                    }

                    // we are no longer leader so error out early
                    let err = ForwardToLeader::empty();
                    let _ = tx.send(Err(err.into()));
                    return;
                }

                granted.insert(target);

                if eff_mem.is_quorum(granted.iter()) {
                    let _ = tx.send(Ok(resp));
                    return;
                }
            }

            // If we've hit this location, then we've failed to gather needed confirmations due to
            // request failures.

            let _ = tx.send(Err(QuorumNotEnough {
                cluster: eff_mem.membership().summary(),
                got: granted,
            }
            .into()));
        };

        // TODO: do not spawn, manage read requests with a queue by RaftCore

        // False positive lint warning(`non-binding `let` on a future`): https://github.com/rust-lang/rust-clippy/issues/9932
        #[allow(clippy::let_underscore_future)]
        let _ = C::AsyncRuntime::spawn(waiting_fu.instrument(tracing::debug_span!("spawn_is_leader_waiting")));
    }

    /// Submit change-membership by writing a Membership log entry.
    ///
    /// If `retain` is `true`, removed `voter` will becomes `learner`. Otherwise they will
    /// be just removed.
    ///
    /// Changing membership includes changing voters config or adding/removing learners:
    ///
    /// - To change voters config, it will build a new **joint** config. If it already a joint
    ///   config, it returns the final uniform config.
    /// - Adding a learner does not affect election, thus it does not need to enter joint consensus.
    ///   But it still has to wait for the previous membership to commit. Otherwise a second
    ///   proposed membership implies the previous one is committed.
    // ---
    // TODO: This limit can be removed if membership_state is replaced by a list of membership logs.
    //       Because allowing this requires the engine to be able to store more than 2
    //       membership logs. And it does not need to wait for the previous membership log to commit
    //       to propose the new membership log.
    #[tracing::instrument(level = "debug", skip(self, tx))]
    pub(super) fn change_membership(
        &mut self,
        changes: ChangeMembers<C::NodeId, C::Node>,
        retain: bool,
        tx: ResponderOf<C>,
    ) {
        let res = self.engine.state.membership_state.change_handler().apply(changes, retain);
        let new_membership = match res {
            Ok(x) => x,
            Err(e) => {
                tx.send(Err(ClientWriteError::ChangeMembershipError(e)));
                return;
            }
        };

        let ent = C::Entry::new_membership(LogId::default(), new_membership);
        self.write_entry(ent, Some(tx));
    }

    /// Write a log entry to the cluster through raft protocol.
    ///
    /// I.e.: append the log entry to local store, forward it to a quorum(including the leader),
    /// waiting for it to be committed and applied.
    ///
    /// The result of applying it to state machine is sent to `resp_tx`, if it is not `None`.
    /// The calling side may not receive a result from `resp_tx`, if raft is shut down.
    #[tracing::instrument(level = "debug", skip_all, fields(id = display(&self.id)))]
    pub fn write_entry(&mut self, entry: C::Entry, resp_tx: Option<ResponderOf<C>>) -> bool {
        tracing::debug!(payload = display(&entry), "write_entry");

        let (mut lh, tx) = if let Some((lh, tx)) = self.engine.get_leader_handler_or_reject(resp_tx) {
            (lh, tx)
        } else {
            return false;
        };

        let entries = vec![entry];
        // TODO: it should returns membership config error etc. currently this is done by the
        //       caller.
        lh.leader_append_entries(entries);
        let index = lh.state.last_log_id().unwrap().index;

        // Install callback channels.
        if let Some(tx) = tx {
            self.client_resp_channels.insert(index, tx);
        }

        true
    }

    /// Send a heartbeat message to every followers/learners.
    ///
    /// Currently heartbeat is a blank log
    #[tracing::instrument(level = "debug", skip_all, fields(id = display(&self.id)))]
    pub fn send_heartbeat(&mut self, emitter: impl Display) -> bool {
        tracing::debug!(now = debug(C::now()), "send_heartbeat");

        let mut lh = if let Some((lh, _)) = self.engine.get_leader_handler_or_reject(None) {
            lh
        } else {
            tracing::debug!(now = debug(C::now()), "{} failed to send heartbeat", emitter);
            return false;
        };

        lh.send_heartbeat();

        tracing::debug!("{} triggered sending heartbeat", emitter);
        true
    }

    #[tracing::instrument(level = "debug", skip_all)]
    pub fn flush_metrics(&mut self) {
        let leader_metrics = if let Some(leader) = self.engine.leader.as_ref() {
            let prog = &leader.progress;
            Some(
                prog.iter()
                    .map(|(id, p)| {
                        (
                            id.clone(),
                            <ProgressEntry<<C as RaftTypeConfig>::NodeId> as Borrow<Option<LogId<C::NodeId>>>>::borrow(
                                p,
                            )
                            .clone(),
                        )
                    })
                    .collect(),
            )
        } else {
            None
        };
        self.report_metrics(leader_metrics);
    }

    /// Report a metrics payload on the current state of the Raft node.
    #[tracing::instrument(level = "debug", skip_all)]
    pub(crate) fn report_metrics(&mut self, replication: Option<ReplicationMetrics<C::NodeId>>) {
        let last_quorum_acked = self.last_quorum_acked_time();
        let millis_since_quorum_ack = last_quorum_acked.map(|t| t.elapsed().as_millis() as u64);

        let st = &self.engine.state;

        let membership_config = st.membership_state.effective().stored_membership().clone();
        let current_leader = self.current_leader();

        let m = RaftMetrics {
            running_state: Ok(()),
            id: self.id.clone(),

            // --- data ---
            current_term: st.vote_ref().leader_id().get_term(),
            vote: st.io_state().vote().clone(),
            last_log_index: st.last_log_id().index(),
            last_applied: st.io_applied().cloned(),
            snapshot: st.io_snapshot_last_log_id().cloned(),
            purged: st.io_purged().cloned(),

            // --- cluster ---
            state: st.server_state,
            current_leader: current_leader.clone(),
            millis_since_quorum_ack,
            membership_config: membership_config.clone(),

            // --- replication ---
            replication: replication.clone(),
        };

        let data_metrics = RaftDataMetrics {
            last_log: st.last_log_id().cloned(),
            last_applied: st.io_applied().cloned(),
            snapshot: st.io_snapshot_last_log_id().cloned(),
            purged: st.io_purged().cloned(),
            millis_since_quorum_ack,
            replication,
        };

        let server_metrics = RaftServerMetrics {
            id: self.id.clone(),
            vote: st.io_state().vote().clone(),
            state: st.server_state,
            current_leader,
            membership_config,
        };

        // Start to send metrics
        // `RaftMetrics` is sent last, because `Wait` only examines `RaftMetrics`
        // but not `RaftDataMetrics` and `RaftServerMetrics`.
        // Thus if `RaftMetrics` change is perceived, the other two should have been updated.

        self.tx_data_metrics.send_if_modified(|metrix| {
            if data_metrics.ne(metrix) {
                *metrix = data_metrics.clone();
                return true;
            }
            false
        });

        self.tx_server_metrics.send_if_modified(|metrix| {
            if server_metrics.ne(metrix) {
                *metrix = server_metrics.clone();
                return true;
            }
            false
        });

        tracing::debug!("report_metrics: {}", m.summary());
        let res = self.tx_metrics.send(m);

        if let Err(err) = res {
            tracing::error!(error=%err, id=display(&self.id), "error reporting metrics");
        }
    }

    /// Handle the admin command `initialize`.
    ///
    /// It is allowed to initialize only when `last_log_id.is_none()` and `vote==(0,0)`.
    /// See: [Conditions for initialization][precondition]
    ///
    /// [precondition]: crate::docs::cluster_control::cluster_formation#preconditions-for-initialization
    #[tracing::instrument(level = "debug", skip(self, tx))]
    pub(crate) fn handle_initialize(
        &mut self,
        member_nodes: BTreeMap<C::NodeId, C::Node>,
        tx: ResultSender<C, (), InitializeError<C::NodeId, C::Node>>,
    ) {
        tracing::debug!(member_nodes = debug(&member_nodes), "{}", func_name!());

        let membership = Membership::from(member_nodes);

        let entry = C::Entry::new_membership(LogId::default(), membership);
        let res = self.engine.initialize(entry);
        self.engine.output.push_command(Command::Respond {
            when: None,
            resp: Respond::new(res, tx),
        });
    }

    /// Trigger a snapshot building(log compaction) job if there is no pending building job.
    #[tracing::instrument(level = "debug", skip(self))]
    pub(crate) fn trigger_snapshot(&mut self) {
        tracing::debug!("{}", func_name!());
        self.engine.snapshot_handler().trigger_snapshot();
    }

    #[tracing::instrument(level = "debug", skip(self))]
    pub(crate) fn current_leader(&self) -> Option<C::NodeId> {
        tracing::debug!(
            self_id = display(&self.id),
            vote = display(self.engine.state.vote_ref().summary()),
            "get current_leader"
        );

        let vote = self.engine.state.vote_ref();

        if !vote.is_committed() {
            return None;
        }

        // Safe unwrap(): vote that is committed has to already have voted for some node.
        let id = vote.leader_id().voted_for().unwrap();

        // TODO: `is_voter()` is slow, maybe cache `current_leader`,
        //       e.g., only update it when membership or vote changes
        if self.engine.state.membership_state.effective().is_voter(&id) {
            Some(id)
        } else {
            tracing::debug!("id={} is not a voter", id);
            None
        }
    }

    /// Retrieves the most recent timestamp that is acknowledged by a quorum.
    ///
    /// This function returns the latest known time at which the leader received acknowledgment
    /// from a quorum of followers, indicating its leadership is current and recognized.
    /// If the node is not a leader or no acknowledgment has been received, `None` is returned.
    fn last_quorum_acked_time(&mut self) -> Option<InstantOf<C>> {
        let leading = self.engine.leader.as_mut();
        leading.and_then(|l| l.last_quorum_acked_time())
    }

    pub(crate) fn get_leader_node(&self, leader_id: Option<C::NodeId>) -> Option<C::Node> {
        let leader_id = leader_id?;

        self.engine.state.membership_state.effective().get_node(&leader_id).cloned()
    }

    /// A temp wrapper to make non-blocking `append_to_log` a blocking.
    #[tracing::instrument(level = "debug", skip_all)]
    pub(crate) async fn append_to_log<I>(
        &mut self,
        entries: I,
        vote: Vote<C::NodeId>,
        last_log_id: LogId<C::NodeId>,
    ) -> Result<(), StorageError<C::NodeId>>
    where
        I: IntoIterator<Item = C::Entry> + OptionalSend,
        I::IntoIter: OptionalSend,
    {
        tracing::debug!("append_to_log");

        let (tx, rx) = C::AsyncRuntime::oneshot();
        let log_io_id = LogIOId::new(vote, Some(last_log_id));

        let callback = LogFlushed::new(log_io_id, tx);

        self.log_store.append(entries, callback).await?;
        rx.await
            .map_err(|e| StorageIOError::write_logs(AnyError::error(e)))?
            .map_err(|e| StorageIOError::write_logs(AnyError::error(e)))?;
        Ok(())
    }

    #[tracing::instrument(level = "debug", skip_all)]
    pub(crate) async fn apply_to_state_machine(
        &mut self,
        seq: CommandSeq,
        since: u64,
        upto_index: u64,
    ) -> Result<(), StorageError<C::NodeId>> {
        tracing::debug!(upto_index = display(upto_index), "{}", func_name!());

        let end = upto_index + 1;

        debug_assert!(
            since <= end,
            "last_applied index {} should <= committed index {}",
            since,
            end
        );

        if since == end {
            return Ok(());
        }

        let entries = self.log_store.get_log_entries(since..end).await?;
        tracing::debug!(
            entries = display(DisplaySlice::<_>(entries.as_slice())),
            "about to apply"
        );

        let last_applied = entries[entries.len() - 1].get_log_id().clone();

        let cmd = sm::Command::apply(entries).with_seq(seq);
        self.sm_handle.send(cmd).map_err(|e| StorageIOError::apply(last_applied, AnyError::error(e)))?;

        Ok(())
    }

    /// When received results of applying log entries to the state machine, send back responses to
    /// the callers that proposed the entries.
    #[tracing::instrument(level = "debug", skip_all)]
    pub(crate) fn handle_apply_result(&mut self, res: ApplyResult<C>) {
        tracing::debug!(last_applied = display(res.last_applied), "{}", func_name!());

        let mut results = res.apply_results.into_iter();
        let mut applying_entries = res.applying_entries.into_iter();

        for log_index in res.since..res.end {
            let ent = applying_entries.next().unwrap();
            let apply_res = results.next().unwrap();
            let tx = self.client_resp_channels.remove(&log_index);

            Self::send_response(ent, apply_res, tx);
        }
    }

    /// Send result of applying a log entry to its client.
    #[tracing::instrument(level = "debug", skip_all)]
    pub(super) fn send_response(entry: ApplyingEntry<C::NodeId, C::Node>, resp: C::R, tx: Option<ResponderOf<C>>) {
        tracing::debug!(entry = debug(&entry), "send_response");

        let tx = match tx {
            None => return,
            Some(x) => x,
        };

        let membership = entry.membership;

        let res = Ok(ClientWriteResponse {
            log_id: entry.log_id,
            data: resp,
            membership,
        });

        tx.send(res);
    }

    /// Spawn a new replication stream returning its replication state handle.
    #[tracing::instrument(level = "debug", skip(self))]
    #[allow(clippy::type_complexity)]
    pub(crate) async fn spawn_replication_stream(
        &mut self,
        target: C::NodeId,
        progress_entry: ProgressEntry<C::NodeId>,
    ) -> ReplicationHandle<C> {
        // Safe unwrap(): target must be in membership
        let target_node = self.engine.state.membership_state.effective().get_node(&target).unwrap();

        let membership_log_id = self.engine.state.membership_state.effective().log_id();
        let network = self.network.new_client(target.clone(), target_node).await;
        let snapshot_network = self.network.new_client(target.clone(), target_node).await;

        let leader = self.engine.leader.as_ref().unwrap();

        let session_id = ReplicationSessionId::new(leader.vote.clone(), membership_log_id.clone());

        ReplicationCore::<C, N, LS>::spawn(
            target.clone(),
            session_id,
            self.config.clone(),
            self.engine.state.committed().cloned(),
            progress_entry.matching,
            network,
            snapshot_network,
            self.log_store.get_log_reader().await,
            self.sm_handle.new_snapshot_reader(),
            self.tx_notify.clone(),
            tracing::span!(parent: &self.span, Level::DEBUG, "replication", id=display(&self.id), target=display(&target)),
        )
    }

    /// Remove all replication.
    #[tracing::instrument(level = "debug", skip_all)]
    pub async fn remove_all_replication(&mut self) {
        tracing::info!("remove all replication");

        let nodes = std::mem::take(&mut self.replications);

        tracing::debug!(
            targets = debug(nodes.iter().map(|x| x.0.clone()).collect::<Vec<_>>()),
            "remove all targets from replication_metrics"
        );

        for (target, s) in nodes {
            let handle = s.join_handle;

            // Drop sender to notify the task to shutdown
            drop(s.tx_repl);

            tracing::debug!("joining removed replication: {}", target);
            let _x = handle.await;
            tracing::info!("Done joining removed replication : {}", target);
        }
    }

    /// Run as many commands as possible.
    ///
    /// If there is a command that waits for a callback, just return and wait for
    /// next RaftMsg.
    #[tracing::instrument(level = "debug", skip_all)]
    pub(crate) async fn run_engine_commands(&mut self) -> Result<(), StorageError<C::NodeId>> {
        if tracing::enabled!(Level::DEBUG) {
            tracing::debug!("queued commands: start...");
            for c in self.engine.output.iter_commands() {
                tracing::debug!("queued commands: {:?}", c);
            }
            tracing::debug!("queued commands: end...");
        }

        while let Some(cmd) = self.engine.output.pop_command() {
            tracing::debug!("run command: {:?}", cmd);

            let res = self.run_command(cmd).await?;

            if let Some(cmd) = res {
                tracing::debug!("early return: postpone command: {:?}", cmd);
                self.engine.output.postpone_command(cmd);

                if tracing::enabled!(Level::DEBUG) {
                    for c in self.engine.output.iter_commands().take(8) {
                        tracing::debug!("postponed, first 8 queued commands: {:?}", c);
                    }
                }

                return Ok(());
            }
        }

        Ok(())
    }

    /// Run an event handling loop
    ///
    /// It always returns a [`Fatal`] error upon returning.
    #[tracing::instrument(level="debug", skip_all, fields(id=display(&self.id)))]
    async fn runtime_loop(
        &mut self,
        mut rx_shutdown: <C::AsyncRuntime as AsyncRuntime>::OneshotReceiver<()>,
    ) -> Result<Infallible, Fatal<C::NodeId>> {
        // Ratio control the ratio of number of RaftMsg to process to number of Notify to process.
        let mut balancer = Balancer::new(10_000);

        loop {
            self.flush_metrics();

            // In each loop, it does not have to check rx_shutdown and flush metrics for every RaftMsg
            // processed.
            // In each loop, the first step is blocking waiting for any message from any channel.
            // Then if there is any message, process as many as possible to maximize throughput.

            select! {
                // Check shutdown in each loop first so that a message flood in `tx_api` won't block shutting down.
                // `select!` without `biased` provides a random fairness.
                // We want to check shutdown prior to other channels.
                // See: https://docs.rs/tokio/latest/tokio/macro.select.html#fairness
                biased;

                _ = &mut rx_shutdown => {
                    tracing::info!("recv from rx_shutdown");
                    return Err(Fatal::Stopped);
                }

                notify_res = self.rx_notify.recv() => {
                    match notify_res {
                        Some(notify) => self.handle_notify(notify)?,
                        None => {
                            tracing::error!("all rx_notify senders are dropped");
                            return Err(Fatal::Stopped);
                        }
                    };
                }

                msg_res = self.rx_api.recv() => {
                    match msg_res {
                        Some(msg) => self.handle_api_msg(msg).await,
                        None => {
                            tracing::info!("all rx_api senders are dropped");
                            return Err(Fatal::Stopped);
                        }
                    };
                }
            }

            self.run_engine_commands().await?;

            // There is a message waking up the loop, process channels one by one.

            let raft_msg_processed = self.process_raft_msg(balancer.raft_msg()).await?;
            let notify_processed = self.process_notify(balancer.notify()).await?;

            // If one of the channel consumed all its budget, re-balance the budget ratio.

            #[allow(clippy::collapsible_else_if)]
            if notify_processed == balancer.notify() {
                tracing::info!("there may be more Notify to process, increase Notify ratio");
                balancer.increase_notify();
            } else {
                if raft_msg_processed == balancer.raft_msg() {
                    tracing::info!("there may be more RaftMsg to process, increase RaftMsg ratio");
                    balancer.increase_raft_msg();
                }
            }
        }
    }

    /// Process RaftMsg as many as possible.
    ///
    /// It returns the number of processed message.
    /// If the input channel is closed, it returns `Fatal::Stopped`.
    async fn process_raft_msg(&mut self, at_most: u64) -> Result<u64, Fatal<C::NodeId>> {
        for i in 0..at_most {
            let res = self.rx_api.try_recv();
            let msg = match res {
                Ok(msg) => msg,
                Err(e) => match e {
                    mpsc::error::TryRecvError::Empty => {
                        tracing::debug!("all RaftMsg are processed, wait for more");
                        return Ok(i + 1);
                    }
                    mpsc::error::TryRecvError::Disconnected => {
                        tracing::debug!("rx_api is disconnected, quit");
                        return Err(Fatal::Stopped);
                    }
                },
            };

            self.handle_api_msg(msg).await;

            // TODO: does run_engine_commands() run too frequently?
            //       to run many commands in one shot, it is possible to batch more commands to gain
            //       better performance.

            self.run_engine_commands().await?;
        }

        tracing::debug!("at_most({}) reached, there are more queued RaftMsg to process", at_most);

        Ok(at_most)
    }

    /// Process Notify as many as possible.
    ///
    /// It returns the number of processed notifications.
    /// If the input channel is closed, it returns `Fatal::Stopped`.
    async fn process_notify(&mut self, at_most: u64) -> Result<u64, Fatal<C::NodeId>> {
        for i in 0..at_most {
            let res = self.rx_notify.try_recv();
            let notify = match res {
                Ok(msg) => msg,
                Err(e) => match e {
                    mpsc::error::TryRecvError::Empty => {
                        tracing::debug!("all Notify are processed, wait for more");
                        return Ok(i + 1);
                    }
                    mpsc::error::TryRecvError::Disconnected => {
                        tracing::error!("rx_notify is disconnected, quit");
                        return Err(Fatal::Stopped);
                    }
                },
            };

            self.handle_notify(notify)?;

            // TODO: does run_engine_commands() run too frequently?
            //       to run many commands in one shot, it is possible to batch more commands to gain
            //       better performance.

            self.run_engine_commands().await?;
        }

        tracing::debug!("at_most({}) reached, there are more queued Notify to process", at_most);

        Ok(at_most)
    }

    /// Spawn parallel vote requests to all cluster members.
    #[tracing::instrument(level = "trace", skip_all, fields(vote=vote_req.summary()))]
    async fn spawn_parallel_vote_requests(&mut self, vote_req: &VoteRequest<C::NodeId>) {
        let members = self.engine.state.membership_state.effective().voter_ids();

        let vote = vote_req.vote.clone();

        for target in members {
            if target == self.id {
                continue;
            }

            let req = vote_req.clone();

            // Safe unwrap(): target must be in membership
            let target_node = self.engine.state.membership_state.effective().get_node(&target).unwrap().clone();
            let mut client = self.network.new_client(target.clone(), &target_node).await;

            let tx = self.tx_notify.clone();

            let ttl = Duration::from_millis(self.config.election_timeout_min);
            let id = self.id.clone();
            let option = RPCOption::new(ttl);

            // False positive lint warning(`non-binding `let` on a future`): https://github.com/rust-lang/rust-clippy/issues/9932
            #[allow(clippy::let_underscore_future)]
            let _ = C::AsyncRuntime::spawn(
                {
                    let target = target.clone();
                    let vote = vote.clone();

                    async move {
                        let tm_res = C::AsyncRuntime::timeout(ttl, client.vote(req, option)).await;
                        let res = match tm_res {
                            Ok(res) => res,

                            Err(_timeout) => {
                                let timeout_err = Timeout {
                                    action: RPCTypes::Vote,
                                    id,
                                    target: target.clone(),
                                    timeout: ttl,
                                };
                                tracing::error!({error = %timeout_err, target = display(&target)}, "timeout");
                                return;
                            }
                        };

                        match res {
                            Ok(resp) => {
                                let _ = tx.send(Notify::VoteResponse {
                                    target,
                                    resp,
                                    sender_vote: vote,
                                });
                            }
                            Err(err) => tracing::error!({error=%err, target=display(&target)}, "while requesting vote"),
                        }
                    }
                }
                .instrument(tracing::debug_span!(
                    parent: &Span::current(),
                    "send_vote_req",
                    target = display(&target)
                )),
            );
        }
    }

    #[tracing::instrument(level = "debug", skip_all)]
    pub(super) fn handle_vote_request(&mut self, req: VoteRequest<C::NodeId>, tx: VoteTx<C>) {
        tracing::info!(req = display(req.summary()), func = func_name!());

        let resp = self.engine.handle_vote_req(req);
        self.engine.output.push_command(Command::Respond {
            when: None,
            resp: Respond::new(Ok(resp), tx),
        });
    }

    #[tracing::instrument(level = "debug", skip_all)]
    pub(super) fn handle_append_entries_request(&mut self, req: AppendEntriesRequest<C>, tx: AppendEntriesTx<C>) {
        tracing::debug!(req = display(req.summary()), func = func_name!());

        let is_ok = self.engine.handle_append_entries(&req.vote, req.prev_log_id, req.entries, Some(tx));

        if is_ok {
            self.engine.handle_commit_entries(req.leader_commit);
        }
    }

    // TODO: Make this method non-async. It does not need to run any async command in it.
    #[tracing::instrument(level = "debug", skip(self, msg), fields(state = debug(self.engine.state.server_state), id=display(&self.id)))]
    pub(crate) async fn handle_api_msg(&mut self, msg: RaftMsg<C>) {
        tracing::debug!("recv from rx_api: {}", msg.summary());

        match msg {
            RaftMsg::AppendEntries { rpc, tx } => {
                self.handle_append_entries_request(rpc, tx);
            }
            RaftMsg::RequestVote { rpc, tx } => {
                let now = C::now();
                tracing::info!(
                    now = display(now.display()),
                    vote_request = display(&rpc),
                    "received RaftMsg::RequestVote: {}",
                    func_name!()
                );

                self.handle_vote_request(rpc, tx);
            }
            RaftMsg::BeginReceivingSnapshot { tx } => {
                self.engine.handle_begin_receiving_snapshot(tx);
            }
            RaftMsg::InstallFullSnapshot { vote, snapshot, tx } => {
                self.engine.handle_install_full_snapshot(vote, snapshot, tx);
            }
            RaftMsg::CheckIsLeaderRequest { tx } => {
                self.handle_check_is_leader_request(tx).await;
            }
            RaftMsg::ClientWriteRequest { app_data, tx } => {
                self.write_entry(C::Entry::from_app_data(app_data), Some(tx));
            }
            RaftMsg::Initialize { members, tx } => {
                tracing::info!(
                    members = debug(&members),
                    "received RaftMsg::Initialize: {}",
                    func_name!()
                );

                self.handle_initialize(members, tx);
            }
            RaftMsg::ChangeMembership { changes, retain, tx } => {
                tracing::info!(
                    members = debug(&changes),
                    retain = debug(&retain),
                    "received RaftMsg::ChangeMembership: {}",
                    func_name!()
                );

                self.change_membership(changes, retain, tx);
            }
            RaftMsg::ExternalCoreRequest { req } => {
                req(&self.engine.state);
            }
            RaftMsg::ExternalCommand { cmd } => {
                tracing::info!(cmd = debug(&cmd), "received RaftMsg::ExternalCommand: {}", func_name!());

                match cmd {
                    ExternalCommand::Elect => {
                        if self.engine.state.membership_state.effective().is_voter(&self.id) {
                            // TODO: reject if it is already a leader?
                            self.engine.elect();
                            tracing::debug!("ExternalCommand: triggered election");
                        } else {
                            // Node is switched to learner.
                        }
                    }
                    ExternalCommand::Heartbeat => {
                        self.send_heartbeat("ExternalCommand");
                    }
                    ExternalCommand::Snapshot => self.trigger_snapshot(),
                    ExternalCommand::GetSnapshot { tx } => {
                        let cmd = sm::Command::get_snapshot(tx);
                        let res = self.sm_handle.send(cmd);
                        if let Err(e) = res {
                            tracing::error!(error = display(e), "error sending GetSnapshot to sm worker");
                        }
                    }
                    ExternalCommand::PurgeLog { upto } => {
                        self.engine.trigger_purge_log(upto);
                    }
                }
            }
        };
    }

    // TODO: Make this method non-async. It does not need to run any async command in it.
    #[tracing::instrument(level = "debug", skip_all, fields(state = debug(self.engine.state.server_state), id=display(&self.id)))]
    pub(crate) fn handle_notify(&mut self, notify: Notify<C>) -> Result<(), Fatal<C::NodeId>> {
        tracing::debug!("recv from rx_notify: {}", notify.summary());

        match notify {
            Notify::VoteResponse {
                target,
                resp,
                sender_vote,
            } => {
                let now = C::now();

                tracing::info!(
                    now = display(now.display()),
                    resp = display(&resp),
                    "received Notify::VoteResponse: {}",
                    func_name!()
                );

                if self.does_vote_match(&sender_vote, "VoteResponse") {
                    self.engine.handle_vote_resp(target, resp);
                }
            }

            Notify::HigherVote {
                target,
                higher,
                sender_vote,
            } => {
                tracing::info!(
                    target = display(&target),
                    higher_vote = display(&higher),
                    sending_vote = display(&sender_vote),
                    "received Notify::HigherVote: {}",
                    func_name!()
                );

                if self.does_vote_match(&sender_vote, "HigherVote") {
                    // Rejected vote change is ok.
                    let _ = self.engine.vote_handler().update_vote(&higher);
                }
            }

            Notify::Tick { i } => {
                // check every timer

                let now = C::now();
                tracing::debug!("received tick: {}, now: {:?}", i, now);

                self.handle_tick_election();

                // TODO: test: fixture: make isolated_nodes a single-way isolating.

                // TODO: check if it is Leader with Engine
                // Leader send heartbeat
                let heartbeat_at = self.leader_data.as_ref().map(|x| x.next_heartbeat);
                if let Some(t) = heartbeat_at {
                    if now >= t {
                        if self.runtime_config.enable_heartbeat.load(Ordering::Relaxed) {
                            self.send_heartbeat("tick");
                        }

                        // Install next heartbeat
                        if let Some(l) = &mut self.leader_data {
                            l.next_heartbeat = C::now() + Duration::from_millis(self.config.heartbeat_interval);
                        }
                    }
                }

                // When a membership that removes the leader is committed,
                // the leader continue to work for a short while before reverting to a learner.
                // This way, let the leader replicate the `membership-log-is-committed` message to
                // followers.
                // Otherwise, if the leader step down at once, the follower might have to
                // re-commit the membership log again, electing itself.
                //
                // ---
                //
                // Stepping down only when the response of the second change-membership is sent.
                // Otherwise the Sender to the caller will be dropped before sending back the
                // response.

                // TODO: temp solution: Manually wait until the second membership log being applied to state
                //       machine. Because the response is sent back to the caller after log is
                //       applied.
                //       ---
                //       A better way is to make leader step down a command that waits for the log to be applied.
                if self.engine.state.io_applied() >= self.engine.state.membership_state.effective().log_id().as_ref() {
                    self.engine.leader_step_down();
                }
            }

            Notify::Network { response } => {
                //
                match response {
                    replication::Response::Progress {
                        target,
                        request_id: id,
                        result,
                        session_id,
                    } => {
                        // If vote or membership changes, ignore the message.
                        // There is chance delayed message reports a wrong state.
                        if self.does_replication_session_match(&session_id, "UpdateReplicationMatched") {
                            self.handle_replication_progress(target, id, result);
                        }
                    }

                    replication::Response::StorageError { error } => {
                        tracing::error!(
                            error = display(&error),
                            "received Notify::ReplicationStorageError: {}",
                            func_name!()
                        );

                        return Err(Fatal::from(error));
                    }

                    replication::Response::HigherVote {
                        target,
                        higher,
                        sender_vote,
                    } => {
                        tracing::info!(
                            target = display(&target),
                            higher_vote = display(&higher),
                            sender_vote = display(&sender_vote),
                            "received Notify::HigherVote: {}",
                            func_name!()
                        );

                        if self.does_vote_match(&sender_vote, "HigherVote") {
                            // Rejected vote change is ok.
                            let _ = self.engine.vote_handler().update_vote(&higher);
                        }
                    }
                }
            }

            Notify::StateMachine { command_result } => {
                tracing::debug!("sm::StateMachine command result: {:?}", command_result);

                let seq = command_result.command_seq;
                let res = command_result.result?;

                match res {
                    // BuildSnapshot is a read operation that does not have to be serialized by
                    // sm::Worker. Thus it may finish out of order.
                    sm::Response::BuildSnapshot(_) => {}
                    _ => {
                        debug_assert!(
                            self.command_state.finished_sm_seq < seq,
                            "sm::StateMachine command result is out of order: expect {} < {}",
                            self.command_state.finished_sm_seq,
                            seq
                        );
                    }
                }
                self.command_state.finished_sm_seq = seq;

                match res {
                    sm::Response::BuildSnapshot(meta) => {
                        tracing::info!(
                            "sm::StateMachine command done: BuildSnapshot: {}: {}",
                            meta.summary(),
                            func_name!()
                        );

                        // Update in-memory state first, then the io state.
                        // In-memory state should always be ahead or equal to the io state.

                        let last_log_id = meta.last_log_id.clone();
                        self.engine.finish_building_snapshot(meta);

                        let st = self.engine.state.io_state_mut();
                        st.update_snapshot(last_log_id);
                    }
                    sm::Response::InstallSnapshot(meta) => {
                        tracing::info!(
                            "sm::StateMachine command done: InstallSnapshot: {}: {}",
                            meta.summary(),
                            func_name!()
                        );

                        if let Some(meta) = meta {
                            let st = self.engine.state.io_state_mut();
                            st.update_applied(meta.last_log_id.clone());
                            st.update_snapshot(meta.last_log_id);
                        }
                    }
                    sm::Response::Apply(res) => {
                        self.engine.state.io_state_mut().update_applied(Some(res.last_applied.clone()));

                        self.handle_apply_result(res);
                    }
                }
            }
        };
        Ok(())
    }

    #[tracing::instrument(level = "debug", skip_all)]
    fn handle_tick_election(&mut self) {
        let now = C::now();

        tracing::debug!("try to trigger election by tick, now: {:?}", now);

        // TODO: leader lease should be extended. Or it has to examine if it is leader
        //       before electing.
        if self.engine.state.server_state == ServerState::Leader {
            tracing::debug!("already a leader, do not elect again");
            return;
        }

        if !self.engine.state.membership_state.effective().is_voter(&self.id) {
            tracing::debug!("this node is not a voter");
            return;
        }

        if !self.runtime_config.enable_elect.load(Ordering::Relaxed) {
            tracing::debug!("election is disabled");
            return;
        }

        if self.engine.state.membership_state.effective().voter_ids().count() == 1 {
            if self.engine.candidate_ref().is_some() {
                tracing::debug!("skip election, single voter already has an active election in progress");
                return;
            }
            tracing::debug!("this is the only voter, do election at once");
        } else {
            tracing::debug!("there are multiple voter, check election timeout");

            let current_vote = self.engine.state.vote_ref();
            let utime = self.engine.state.vote_last_modified();
            let timer_config = &self.engine.config.timer_config;

            let mut election_timeout = if current_vote.is_committed() {
                timer_config.leader_lease + timer_config.election_timeout
            } else {
                timer_config.election_timeout
            };

            if self.engine.is_there_greater_log() {
                election_timeout += timer_config.smaller_log_timeout;
            }

            tracing::debug!(
                "vote utime: {:?}, current_vote: {}, now-utime:{:?}, election_timeout: {:?}",
                utime,
                current_vote,
                utime.map(|x| now - x),
                election_timeout,
            );

            // Follower/Candidate timer: next election
            if utime > Some(now - election_timeout) {
                tracing::debug!("election timeout has not yet passed",);
                return;
            }

            tracing::info!("election timeout passed, check if it is a voter for election");
        }

        // Every time elect, reset this flag.
        self.engine.reset_greater_log();

        tracing::info!("do trigger election");
        self.engine.elect();
    }

    #[tracing::instrument(level = "debug", skip_all)]
    fn handle_replication_progress(
        &mut self,
        target: C::NodeId,
        request_id: RequestId,
        result: Result<ReplicationResult<C>, String>,
    ) {
        tracing::debug!(
            target = display(&target),
            request_id = display(request_id),
            result = debug(&result),
            "handle_replication_progress"
        );

        #[allow(clippy::collapsible_if)]
        if tracing::enabled!(Level::DEBUG) {
            if !self.replications.contains_key(&target) {
                tracing::warn!("leader has removed target: {}", target);
            };
        }

        // A leader may have stepped down.
        if self.engine.leader.is_some() {
            self.engine.replication_handler().update_progress(target, request_id, result);
        }
    }

    /// If a message is sent by a previous server state but is received by current server state,
    /// it is a stale message and should be just ignored.
    fn does_vote_match(&self, sender_vote: &Vote<C::NodeId>, msg: impl Display) -> bool {
        // Get the current leading vote:
        // - If input `sender_vote` is committed, it is sent by a Leader. Therefore we check against current
        //   Leader's vote.
        // - Otherwise, it is sent by a Candidate, we check against the current in progress voting state.
        let my_vote = if sender_vote.is_committed() {
            let l = self.engine.leader.as_ref();
            l.map(|x| x.vote.clone())
        } else {
            // If it finished voting, Candidate's vote is None.
            let candidate = self.engine.candidate_ref();
            candidate.map(|x| x.vote_ref().clone())
        };

        if Some(sender_vote) != my_vote.as_ref() {
            tracing::warn!(
                "A message will be ignored because vote changed: msg sent by vote: {}; current my vote: {}; when ({})",
                sender_vote,
                my_vote.display(),
                msg
            );
            false
        } else {
            true
        }
    }
    /// If a message is sent by a previous replication session but is received by current server
    /// state, it is a stale message and should be just ignored.
    fn does_replication_session_match(
        &self,
        session_id: &ReplicationSessionId<C::NodeId>,
        msg: impl Display + Copy,
    ) -> bool {
        if !self.does_vote_match(session_id.vote_ref(), msg) {
            return false;
        }

        if &session_id.membership_log_id != self.engine.state.membership_state.effective().log_id() {
            tracing::warn!(
                "membership_log_id changed: msg sent by: {}; curr: {}; ignore when ({})",
                session_id.membership_log_id.summary(),
                self.engine.state.membership_state.effective().log_id().summary(),
                msg
            );
            return false;
        }
        true
    }
}

impl<C, N, LS, SM> RaftRuntime<C> for RaftCore<C, N, LS, SM>
where
    C: RaftTypeConfig,
    N: RaftNetworkFactory<C>,
    LS: RaftLogStorage<C>,
    SM: RaftStateMachine<C>,
{
    async fn run_command(&mut self, cmd: Command<C>) -> Result<Option<Command<C>>, StorageError<C::NodeId>> {
        let condition = cmd.condition();
        tracing::debug!("condition: {:?}", condition);

        if let Some(condition) = condition {
            match condition {
                Condition::LogFlushed { .. } => {
                    todo!()
                }
                Condition::Applied { log_id } => {
                    if self.engine.state.io_applied() < log_id.as_ref() {
                        tracing::debug!(
                            "log_id: {} has not yet applied, postpone cmd: {:?}",
                            DisplayOption(log_id),
                            cmd
                        );
                        return Ok(Some(cmd));
                    }
                }
                Condition::StateMachineCommand { command_seq } => {
                    if self.command_state.finished_sm_seq < *command_seq {
                        tracing::debug!(
                            "sm::Command({}) has not yet finished({}), postpone cmd: {:?}",
                            command_seq,
                            self.command_state.finished_sm_seq,
                            cmd
                        );
                        return Ok(Some(cmd));
                    }
                }
            }
        }

        match cmd {
            Command::BecomeLeader => {
                debug_assert!(self.leader_data.is_none(), "can not become leader twice");
                self.leader_data = Some(LeaderData::new());
            }
            Command::QuitLeader => {
                self.leader_data = None;
            }
            Command::AppendInputEntries { vote, entries } => {
                let last_log_id = entries.last().unwrap().get_log_id().clone();
                tracing::debug!("AppendInputEntries: {}", DisplaySlice::<_>(&entries),);

                self.append_to_log(entries, vote, last_log_id.clone()).await?;

                // The leader may have changed.
                // But reporting to a different leader is not a problem.
                if let Ok(mut lh) = self.engine.leader_handler() {
                    lh.replication_handler().update_local_progress(Some(last_log_id));
                }
            }
            Command::SaveVote { vote } => {
                self.log_store.save_vote(&vote).await?;
                self.engine.state.io_state_mut().update_vote(vote.clone());

                let _ = self.tx_notify.send(Notify::VoteResponse {
                    target: self.id.clone(),
                    // last_log_id is not used when sending VoteRequest to local node
                    resp: VoteResponse::new(vote.clone(), None, true),
                    sender_vote: vote,
                });
            }
            Command::PurgeLog { upto } => {
                self.log_store.purge(upto.clone()).await?;
                self.engine.state.io_state_mut().update_purged(Some(upto));
            }
            Command::DeleteConflictLog { since } => {
                self.log_store.truncate(since.clone()).await?;

                // Inform clients waiting for logs to be applied.
                let removed = self.client_resp_channels.split_off(&since.index);
                if !removed.is_empty() {
                    let leader_id = self.current_leader();
                    let leader_node = self.get_leader_node(leader_id.clone());

                    // False positive lint warning(`non-binding `let` on a future`): https://github.com/rust-lang/rust-clippy/issues/9932
                    #[allow(clippy::let_underscore_future)]
                    let _ = C::spawn(async move {
                        for (log_index, tx) in removed.into_iter() {
                            tx.send(Err(ClientWriteError::ForwardToLeader(ForwardToLeader {
                                leader_id: leader_id.clone(),
                                leader_node: leader_node.clone(),
                            })));

                            tracing::debug!("sent ForwardToLeader for log_index: {}", log_index,);
                        }
                    });
                }
            }
            Command::SendVote { vote_req } => {
                self.spawn_parallel_vote_requests(&vote_req).await;
            }
            Command::ReplicateCommitted { committed } => {
                for node in self.replications.values() {
                    let _ = node.tx_repl.send(Replicate::Committed(committed.clone()));
                }
            }
            Command::Commit {
                seq,
                ref already_committed,
                ref upto,
            } => {
                self.log_store.save_committed(Some(upto.clone())).await?;
                self.apply_to_state_machine(seq, already_committed.next_index(), upto.index).await?;
            }
            Command::Replicate { req, target } => {
                let node = self.replications.get(&target).expect("replication to target node exists");

                match req {
                    Inflight::None => {
                        let _ = node.tx_repl.send(Replicate::Heartbeat);
                    }
                    Inflight::Logs { id, log_id_range } => {
                        let _ = node.tx_repl.send(Replicate::logs(RequestId::new_append_entries(id), log_id_range));
                    }
                    Inflight::Snapshot { id, last_log_id } => {
                        // unwrap: The replication channel must not be dropped or it is a bug.
                        node.tx_repl.send(Replicate::snapshot(RequestId::new_snapshot(id), last_log_id)).map_err(
                            |_e| StorageIOError::read_snapshot(None, AnyError::error("replication channel closed")),
                        )?;
                    }
                }
            }
            Command::RebuildReplicationStreams { targets } => {
                self.remove_all_replication().await;

                for (target, matching) in targets.iter() {
                    let handle = self.spawn_replication_stream(target.clone(), matching.clone()).await;
                    self.replications.insert(target.clone(), handle);
                }
            }
            Command::StateMachine { command } => {
                // Just forward a state machine command to the worker.
                self.sm_handle.send(command).map_err(|_e| {
                    StorageIOError::write_state_machine(AnyError::error("can not send to sm::Worker".to_string()))
                })?;
            }
            Command::Respond { resp: send, .. } => {
                send.send();
            }
        }

        Ok(None)
    }
}