openraft 0.10.0-alpha.18

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
1728
1729
1730
1731
1732
1733
1734
1735
1736
1737
1738
1739
1740
1741
1742
1743
1744
1745
1746
1747
1748
1749
1750
1751
1752
1753
1754
1755
1756
1757
1758
1759
1760
1761
1762
1763
1764
1765
1766
1767
1768
1769
1770
1771
1772
1773
1774
1775
1776
1777
1778
1779
1780
1781
1782
1783
1784
1785
1786
1787
1788
1789
1790
1791
1792
1793
1794
1795
1796
1797
1798
1799
1800
1801
1802
1803
1804
1805
1806
1807
1808
1809
1810
1811
1812
1813
1814
1815
1816
1817
1818
1819
1820
1821
1822
1823
1824
1825
1826
1827
1828
1829
1830
1831
1832
1833
1834
1835
1836
1837
//! Public Raft interface and data types.
//!
//! [`Raft`] serves as the primary interface to a Raft node,
//! facilitating all interactions with the underlying RaftCore.
//!
//! While `RaftCore` operates as a singleton within an application, [`Raft`] instances are designed
//! to be cheaply cloneable.
//! This allows multiple components within the application that require interaction with `RaftCore`
//! to efficiently share access.

pub(crate) mod api;
#[cfg(test)]
mod declare_raft_types_test;
mod impl_raft_blocking_write;
pub mod linearizable_read;
pub(crate) mod message;
mod raft_inner;
pub mod responder;
mod runtime_config_handle;
pub(crate) mod stream_append;
pub mod trigger;
mod watch_handle;

pub(crate) use api::app::AppApi;
pub(crate) use api::management::ManagementApi;
pub(crate) use api::protocol::ProtocolApi;

pub(in crate::raft) mod core_state;
mod leader;

use std::fmt::Debug;
use std::future::Future;
use std::sync::Arc;
use std::sync::Mutex;
use std::time::Duration;

use core_state::CoreState;
use derive_more::Display;
use futures_util::FutureExt;
use linearizable_read::Linearizer;
pub use message::AppendEntriesRequest;
pub use message::AppendEntriesResponse;
pub use message::ClientWriteResponse;
pub use message::ClientWriteResult;
pub use message::InstallSnapshotRequest;
pub use message::InstallSnapshotResponse;
pub use message::LogSegment;
pub use message::SnapshotResponse;
pub use message::StreamAppendError;
pub use message::TransferLeaderRequest;
pub use message::VoteRequest;
pub use message::VoteResponse;
pub use message::WriteRequest;
pub use message::WriteResponse;
pub use message::WriteResult;
use openraft_macros::since;
pub use stream_append::StreamAppendResult;
use tracing::Instrument;
use tracing::Level;
use tracing::trace_span;

pub use self::leader::Leader;
pub use self::watch_handle::WatchChangeHandle;
use crate::Extensions;
use crate::OptionalSend;
use crate::RaftNetworkFactory;
use crate::RaftState;
pub use crate::RaftTypeConfig;
use crate::StorageError;
use crate::StorageHelper;
use crate::async_runtime::MpscWeakSender;
use crate::async_runtime::OneshotSender;
use crate::async_runtime::mpsc::MpscSender;
use crate::async_runtime::watch::WatchReceiver;
use crate::base::BoxFuture;
use crate::base::BoxOnce;
use crate::base::BoxStream;
use crate::config::Config;
use crate::config::RuntimeConfig;
use crate::core::ClientResponderQueue;
use crate::core::RaftCore;
use crate::core::SharedReplicateBatch;
use crate::core::Tick;
use crate::core::heartbeat::handle::HeartbeatWorkersHandle;
use crate::core::io_flush_tracking::AppliedProgress;
use crate::core::io_flush_tracking::CommitProgress;
pub use crate::core::io_flush_tracking::FlushPoint;
use crate::core::io_flush_tracking::IoProgressWatcher;
use crate::core::io_flush_tracking::LogProgress;
use crate::core::io_flush_tracking::SnapshotProgress;
use crate::core::io_flush_tracking::VoteProgress;
use crate::core::merged_raft_msg_receiver::BatchRaftMsgReceiver;
use crate::core::notification::Notification;
use crate::core::raft_msg::RaftMsg;
use crate::core::raft_msg::external_command::ExternalCommand;
use crate::core::runtime_stats::RuntimeStats;
use crate::core::sm;
use crate::core::sm::worker;
use crate::engine::Engine;
use crate::engine::EngineConfig;
use crate::entry::EntryPayload;
use crate::errors::ClientWriteError;
use crate::errors::Fatal;
use crate::errors::ForwardToLeader;
use crate::errors::InitializeError;
use crate::errors::LinearizableReadError;
use crate::errors::RaftError;
use crate::errors::into_raft_result::IntoRaftResult;
use crate::membership::IntoNodes;
use crate::metrics::MetricsRecorder;
use crate::metrics::RaftDataMetrics;
use crate::metrics::RaftMetrics;
use crate::metrics::RaftServerMetrics;
use crate::metrics::Wait;
use crate::raft::raft_inner::RaftInner;
pub use crate::raft::runtime_config_handle::RuntimeConfigHandle;
use crate::raft::trigger::Trigger;
use crate::raft_state::IOId;
use crate::storage::RaftLogStorage;
use crate::storage::RaftStateMachine;
use crate::type_config::TypeConfigExt;
use crate::type_config::alias::JoinErrorOf;
use crate::type_config::alias::LogIdOf;
use crate::type_config::alias::MpscWeakSenderOf;
use crate::type_config::alias::NodeIdOf;
use crate::type_config::alias::SnapshotDataOf;
use crate::type_config::alias::SnapshotOf;
use crate::type_config::alias::VoteOf;
use crate::type_config::alias::WatchReceiverOf;
use crate::type_config::alias::WriteResponderOf;
use crate::vote::Vote;
use crate::vote::leader_id::raft_leader_id::RaftLeaderId;
use crate::vote::leader_id::raft_leader_id::RaftLeaderIdExt;
use crate::vote::non_committed::UncommittedVote;
use crate::vote::raft_vote::RaftVote;
use crate::vote::raft_vote::RaftVoteExt;

/// Define types for a Raft type configuration.
///
/// Since Rust has some limitations when deriving traits for types with generic arguments
/// and most types are parameterized by [`RaftTypeConfig`], we need to add supertraits to
/// a type implementing [`RaftTypeConfig`].
///
/// This macro does exactly that.
///
/// Example:
/// ```ignore
/// openraft::declare_raft_types!(
///    pub TypeConfig:
///        D            = ClientRequest,
///        R            = ClientResponse,
///        NodeId       = u64,
///        Node         = openraft::BasicNode,
///        Term         = u64,
///        LeaderId     = openraft::impls::leader_id_adv::LeaderId<Self::Term, Self::NodeId>,
///        Vote           = openraft::impls::Vote<Self::LeaderId>,
///        Entry          = openraft::Entry<Self>,
///        SnapshotData   = Cursor<Vec<u8>>,
///        Responder<T>   = openraft::impls::OneshotResponder<Self, T>,
///        AsyncRuntime   = openraft::TokioRuntime,
/// );
/// ```
///
/// Types can be omitted, and the following default type will be used:
/// - `D`:            `String`
/// - `R`:            `String`
/// - `NodeId`:       `u64`
/// - `Node`:         `::openraft::impls::BasicNode`
/// - `Term`:         `u64`
/// - `LeaderId`:     `::openraft::impls::leader_id_adv::LeaderId<Self::Term, Self::NodeId>`
/// - `Vote`:           `::openraft::impls::Vote<Self::LeaderId>`
/// - `Entry`:          `::openraft::impls::Entry<Self>`
/// - `SnapshotData`:   `Cursor<Vec<u8>>`
/// - `Responder<T>`:   `::openraft::impls::OneshotResponder<Self, T>`
/// - `AsyncRuntime`:   `::openraft::impls::TokioRuntime`
/// - `ErrorSource`:    `::anyerror::AnyError`
///
/// For example, to declare with only `D` and `R` types:
/// ```ignore
/// openraft::declare_raft_types!(
///    pub TypeConfig:
///        D = ClientRequest,
///        R = ClientResponse,
/// );
/// ```
///
/// Or just use the default type config:
/// ```ignore
/// openraft::declare_raft_types!(pub TypeConfig);
/// ```
#[macro_export]
macro_rules! declare_raft_types {
    // Add a trailing colon to    `declare_raft_types(MyType)`,
    // Make it the standard form: `declare_raft_types(MyType:)`.
    ($(#[$outer:meta])* $visibility:vis $id:ident) => {
        $crate::declare_raft_types!($(#[$outer])* $visibility $id:);
    };

    // The main entry of this macro
    ($(#[$outer:meta])* $visibility:vis $id:ident: $($(#[$inner:meta])* $type_id:ident = $type:ty),* $(,)? ) => {
        $(#[$outer])*
        #[derive(Debug, Clone, Copy, Default, Eq, PartialEq, Ord, PartialOrd)]
        $visibility struct $id {}

        impl $crate::RaftTypeConfig for $id {
            // `expand!(KEYED, ...)` ignores the duplicates.
            // Thus by appending default types after user defined types,
            // the absent user defined types are filled with default types.
            $crate::macros::expand!(
                KEYED,
                (T, ATTR, V) => {ATTR type T = V;},
                $(($type_id, $(#[$inner])*, $type),)*

                // Default types:
                (D            , , String                                       ),
                (R            , , String                                       ),
                (NodeId       , , u64                                          ),
                (Node         , , $crate::impls::BasicNode                     ),
                (Term         , , u64                                          ),
                (LeaderId     , , $crate::impls::leader_id_adv::LeaderId<Self::Term, Self::NodeId> ),
                (Vote           , , $crate::impls::Vote<Self::LeaderId>            ),
                (Entry          , , $crate::Entry<<Self::LeaderId as $crate::vote::RaftLeaderId>::Committed, Self::D, Self::NodeId, Self::Node> ),
                (SnapshotData   , , std::io::Cursor<Vec<u8>>                     ),
                (Responder<T>   , , $crate::impls::ProgressResponder<Self, T> where T: $crate::OptionalSend + 'static     ),
                (Batch<T>       , , $crate::impls::InlineBatch<T> where T: $crate::OptionalSend + 'static     ),
                (AsyncRuntime   , , $crate::impls::TokioRuntime                  ),
                (ErrorSource    , , $crate::impls::BoxedErrorSource               ),
            );

        }
    };
}

/// Policy that determines how to handle read operations in a Raft cluster.
///
/// This enum defines strategies for ensuring linearizable reads in distributed systems
/// while balancing between consistency guarantees and performance.
#[derive(Clone, Debug, Display, PartialEq, Eq)]
pub enum ReadPolicy {
    /// Uses leader lease to avoid network round-trips for read operations.
    ///
    /// With `LeaseRead`, the leader can serve reads locally without contacting followers
    /// as long as it believes its leadership lease is still valid. This provides better
    /// performance compared to `ReadIndex` but assumes clock drift between nodes is negligible.
    ///
    /// Note: This offers slightly weaker consistency guarantees than `ReadIndex` in exchange
    /// for lower latency.
    LeaseRead,

    /// Implements the ReadIndex protocol to ensure linearizable reads.
    ///
    /// With `ReadIndex`, the leader confirms its leadership status by contacting a quorum
    /// of followers before serving read requests. This ensures strong consistency but incurs
    /// the cost of network communication for each read operation.
    ///
    /// This is the safer option that provides the strongest consistency guarantees.
    ReadIndex,
}

/// Primary interface to a Raft node.
///
/// `Raft` provides the complete implementation of the Raft consensus protocol and serves as the
/// main interface for interacting with a Raft node in the cluster. Applications built on Raft use
/// this type to spawn a Raft task and communicate with it.
///
/// # Architecture
///
/// The `Raft` handle is a lightweight wrapper around an `Arc<RaftInner>`, making it cheap to clone.
/// The actual work is performed by an internal core task, which runs separately processing
/// requests through message channels.
///
/// # Lifecycle
///
/// 1. **Creation**: Use [`Raft::new`] to create and spawn a new Raft node
/// 2. **Initialization**: Call [`initialize`](Raft::initialize) on pristine nodes to form a cluster
/// 3. **Operation**: Use various methods to interact with the node:
///    - Protocol RPCs: [`append_entries`](Raft::append_entries), [`vote`](Raft::vote)
///    - Client operations: [`client_write`](Raft::client_write),
///      [`ensure_linearizable`](Raft::ensure_linearizable)
///    - Management: [`trigger`](Raft::trigger), [`metrics`](Raft::metrics)
/// 4. **Shutdown**: Call [`shutdown`](Raft::shutdown) to gracefully stop the node
///
/// # Cloning
///
/// `Raft` implements [`Clone`] with very low cost, allowing multiple components in your application
/// to hold handles to the same Raft node. All clones reference the same underlying Raft instance.
///
/// # Error Handling
///
/// Methods return [`RaftError::Fatal`] when the Raft node encounters unrecoverable errors or is
/// shutting down. Applications should monitor for fatal errors and initiate shutdown if needed.
///
/// # Examples
///
/// ```ignore
/// // Create a new Raft node
/// let raft = Raft::new(node_id, config, network, log_store, state_machine).await?;
///
/// // Initialize a new cluster
/// raft.initialize(btreeset![1, 2, 3]).await?;
///
/// // Write to the cluster
/// let response = raft.client_write(my_request).await?;
///
/// // Read linearizably
/// raft.ensure_linearizable(ReadPolicy::ReadIndex).await?;
/// let data = raft.with_state_machine(|sm| { sm.read("key") }).await?;
///
/// // Monitor metrics
/// let metrics = raft.metrics().borrow_watched();
/// println!("Current leader: {:?}", metrics.current_leader);
/// ```
///
/// # See Also
///
/// - [Raft specification](https://raft.github.io/raft.pdf) for protocol details
/// - [`Config`] for configuration options
/// - [`RaftMetrics`] for monitoring cluster state
pub struct Raft<C, SM = ()>
where C: RaftTypeConfig
{
    inner: Arc<RaftInner<C>>,
    sm_cmd_tx: MpscWeakSenderOf<C, sm::Command<C, SM>>,
}

impl<C, SM> Clone for Raft<C, SM>
where C: RaftTypeConfig
{
    fn clone(&self) -> Self {
        Self {
            inner: self.inner.clone(),
            sm_cmd_tx: self.sm_cmd_tx.clone(),
        }
    }
}

impl<C, SM> Debug for Raft<C, SM>
where C: RaftTypeConfig
{
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("Raft").field("id", &self.inner.id).finish()
    }
}

/// Forwarder task that bridges IO completion Watch channel to notification channel.
///
/// This task reads IO completion results from a Watch channel and forwards them
/// to the RaftCore notification channel, translating IOId and storage errors to notifications.
///
/// To reduce wakeup overhead, notifications are batched: at most one notification
/// is forwarded per `BATCH_INTERVAL`. When a change arrives, the forwarder waits
/// until the interval expires before reading and forwarding the latest value.
async fn io_completion_forwarder<C>(
    mut rx_io: WatchReceiverOf<C, Result<IOId<C>, StorageError<C>>>,
    weak_tx_notify: MpscWeakSenderOf<C, Notification<C>>,
) where
    C: RaftTypeConfig,
{
    const BATCH_INTERVAL: Duration = Duration::from_micros(1);

    loop {
        let deadline = C::now() + BATCH_INTERVAL;

        // Wait for IO completion notification
        if rx_io.changed().await.is_err() {
            // Watch sender dropped, exit forwarder
            tracing::debug!("IO completion watch channel closed, forwarder exiting");
            break;
        }

        let now = C::now();
        if now < deadline {
            C::sleep_until(deadline).await;

            // Drain all the changed events.
            let _ = rx_io.changed().now_or_never();
        }

        // Read the latest value after batching interval
        let result = {
            let borrowed = rx_io.borrow_watched();
            borrowed.clone()
        };

        // Try to upgrade weak sender
        let Some(tx) = weak_tx_notify.upgrade() else {
            tracing::debug!("Notification channel closed, forwarder exiting");
            break;
        };

        // Forward the result to notification channel
        let notification = match result {
            Ok(io_id) => Notification::LocalIO { io_id },
            Err(storage_error) => Notification::StorageError { error: storage_error },
        };

        if let Err(e) = tx.send(notification).await {
            tracing::warn!("failed to forward IO completion: {}", e.0);
            break;
        }
    }
}

impl<C, SM> Raft<C, SM>
where
    C: RaftTypeConfig,
    SM: RaftStateMachine<C>,
{
    /// Create and spawn a new Raft task.
    ///
    /// ### `id`
    /// The ID which the spawned Raft task will use to identify itself within the cluster.
    /// Applications must guarantee that the ID provided to this function is stable, and should be
    /// persisted in a well known location, probably alongside the Raft log and the application's
    /// state machine. This ensures that restarts of the node will yield the same ID every time.
    ///
    /// ### `config`
    /// Raft's runtime config. See the docs on the `Config` object for more details.
    ///
    /// ### `network`
    /// An implementation of the [`RaftNetworkFactory`] trait which will be used by Raft for
    /// sending RPCs to peer nodes within the cluster.
    ///
    /// ### `storage`
    /// An implementation of the [`RaftLogStorage`] and [`RaftStateMachine`] trait which will be
    /// used by Raft for data storage.
    #[tracing::instrument(level="debug", skip_all, fields(cluster=%config.cluster_name))]
    pub async fn new<LS, N>(
        id: C::NodeId,
        config: Arc<Config>,
        network: N,
        mut log_store: LS,
        mut state_machine: SM,
    ) -> Result<Self, Fatal<C>>
    where
        N: RaftNetworkFactory<C>,
        LS: RaftLogStorage<C>,
    {
        let api_channel_size = config.api_channel_size();
        let notification_channel_size = config.notification_channel_size();

        let (tx_api, rx_api) = C::mpsc(api_channel_size);
        let (tx_notify, rx_notify) = C::mpsc(notification_channel_size);
        let (tx_metrics, rx_metrics) = C::watch_channel(RaftMetrics::new_initial(id.clone()));
        let (tx_data_metrics, rx_data_metrics) = C::watch_channel(RaftDataMetrics::default());
        let (tx_server_metrics, rx_server_metrics) = C::watch_channel(RaftServerMetrics::new_initial(id.clone()));

        // Watch channel for IO completion notifications from storage callbacks.
        // Initial value is a dummy IOId with this node's ID.
        let leader_id = C::LeaderId::new_with_default_term(id.clone());
        let dummy_io_id = IOId::Vote(UncommittedVote::new(leader_id));
        let (tx_io_completed, rx_io_completed) = C::watch_channel(Ok(dummy_io_id));

        // Create weak sender for forwarder before moving tx_notify into RaftCore
        let weak_tx_notify = tx_notify.downgrade();

        let (tx_progress, progress_watcher) = IoProgressWatcher::new();
        let (tx_shutdown, rx_shutdown) = C::oneshot();

        let tick_handle = Tick::spawn(
            Duration::from_millis(config.heartbeat_interval * 3 / 2),
            tx_notify.clone(),
            config.enable_tick,
        );

        let runtime_config = Arc::new(RuntimeConfig::new(&config));

        let core_span = tracing::span!(
            parent: tracing::Span::current(),
            Level::DEBUG,
            "RaftCore",
            id = display(&id),
            cluster = display(&config.cluster_name)
        );

        let eng_config = EngineConfig::new(id.clone(), config.as_ref());

        let state = {
            let mut helper = StorageHelper::new(&mut log_store, &mut state_machine).with_id(id.clone());
            helper.get_initial_state().await?
        };

        let engine = Engine::new(state, eng_config);

        let sm_span = tracing::span!(parent: &core_span, Level::DEBUG, "sm_worker");

        let sm_handle = worker::Worker::spawn(
            state_machine,
            log_store.get_log_reader().await,
            tx_notify.clone(),
            config.state_machine_channel_size(),
            sm_span,
        );

        let sm_cmd_tx = sm_handle.downgrade_sender();

        let default_io_id = IOId::new_vote_io(UncommittedVote::new_with_default_term(id.clone()));
        let (io_accepted_tx, _io_accepted_rx) = C::watch_channel(default_io_id.clone());
        let (io_submitted_tx, _io_submitted_rx) = C::watch_channel(default_io_id);
        let (committed_tx, _committed_rx) = C::watch_channel(None);

        let shared_replicate_batch = SharedReplicateBatch::new();

        let core: RaftCore<C, N, LS, SM> = RaftCore {
            id: id.clone(),
            config: config.clone(),
            runtime_config: runtime_config.clone(),
            core_state: Default::default(),
            network_factory: network,
            log_store,
            sm_handle,

            engine,

            // initially, allocate for 8 kilo outstanding requests.
            client_responders: ClientResponderQueue::with_capacity(1024 * 8),

            replications: Default::default(),

            heartbeat_handle: HeartbeatWorkersHandle::new(id.clone(), config.clone()),
            tx_api: tx_api.clone(),
            rx_api: BatchRaftMsgReceiver::new(rx_api),

            tx_notification: tx_notify,
            rx_notification: rx_notify,

            tx_io_completed,

            io_accepted_tx,

            io_submitted_tx,

            committed_tx,
            tx_metrics,
            tx_data_metrics,
            tx_server_metrics,
            tx_progress,

            runtime_stats: RuntimeStats::new(&config),
            shared_replicate_batch,

            metrics_recorder: None,

            span: core_span,
        };

        // Spawn forwarder task to bridge Watch channel to notification channel
        let _forwarder_handle = C::spawn(io_completion_forwarder::<C>(rx_io_completed, weak_tx_notify));

        let core_handle = C::spawn(core.main(rx_shutdown).instrument(trace_span!("spawn").or_current()));

        let inner = RaftInner {
            id,
            config,
            runtime_config,
            tick_handle,
            tx_api,
            rx_metrics,
            rx_data_metrics,
            rx_server_metrics,
            progress_watcher,
            tx_shutdown: Mutex::new(Some(tx_shutdown)),
            core_state: Mutex::new(CoreState::Running(core_handle)),
            extensions: Extensions::default(),
        };

        Ok(Self {
            inner: Arc::new(inner),
            sm_cmd_tx,
        })
    }
}

impl<C, SM> Raft<C, SM>
where C: RaftTypeConfig
{
    /// Return a handle to update runtime config.
    ///
    /// Such enabling/disabling heartbeat, election, etc.
    ///
    /// Example:
    /// ```ignore
    /// let raft = Raft::new(...).await?;
    /// raft.runtime_config().heartbeat(true);
    /// raft.runtime_config().tick(true);
    /// raft.runtime_config().elect(true);
    /// ```
    pub fn runtime_config(&self) -> RuntimeConfigHandle<'_, C> {
        RuntimeConfigHandle::new(self.inner.as_ref())
    }

    /// Return the config of this Raft node.
    pub fn config(&self) -> &Arc<Config> {
        &self.inner.config
    }

    /// Access the underlying extensions map.
    ///
    /// For most use cases, prefer [`extension()`](Self::extension) which provides
    /// a simpler API for getting values.
    ///
    /// This method is useful when you need direct access to the [`Extensions`] type,
    /// such as checking if a value exists with [`contains()`](Extensions::contains)
    /// or removing a value with [`remove()`](Extensions::remove).
    #[since(version = "0.10.0")]
    pub fn extensions(&self) -> &Extensions {
        &self.inner.extensions
    }

    /// Get a clone of a user-defined extension value.
    ///
    /// If no value exists, `T::default()` is inserted and a clone is returned.
    /// Values must implement `Clone` and `Default`. Use `Arc` for shared mutable state.
    ///
    /// # Example
    ///
    /// ```ignore
    /// use std::sync::atomic::{AtomicU64, Ordering};
    /// use std::sync::Arc;
    ///
    /// #[derive(Clone, Default)]
    /// pub struct MyCounter(Arc<AtomicU64>);
    ///
    /// // Get a clone (auto-inserts default if not present)
    /// let counter = raft.extension::<MyCounter>();
    /// counter.0.fetch_add(1, Ordering::Relaxed);
    ///
    /// // Multiple calls share the same underlying data via Arc
    /// let counter2 = raft.extension::<MyCounter>();
    /// assert_eq!(counter2.0.load(Ordering::Relaxed), 1);
    /// ```
    #[since(version = "0.10.0")]
    pub fn extension<T>(&self) -> T
    where T: OptionalSend + Clone + Default + 'static {
        self.inner.extensions.get::<T>()
    }

    /// Return a copy of the runtime statistics.
    ///
    /// Sends a message to RaftCore to retrieve the current runtime statistics.
    /// This returns a snapshot of the stats at the time of the call.
    #[cfg(feature = "runtime-stats")]
    pub async fn runtime_stats(&self) -> Result<RuntimeStats<C>, Fatal<C>> {
        let (tx, rx) = C::oneshot();
        self.inner.call_core(RaftMsg::GetRuntimeStats { tx }, rx).await
    }

    /// Check if this node is currently the leader.
    ///
    /// Returns `true` if the node's current state is [`ServerState::Leader`].
    ///
    /// # Example
    ///
    /// ```ignore
    /// if raft.is_leader() {
    ///     // Perform leader-only operations
    /// }
    /// ```
    ///
    /// [`ServerState::Leader`]: crate::core::ServerState::Leader
    #[since(version = "0.10.0")]
    pub fn is_leader(&self) -> bool {
        self.inner.rx_metrics.borrow_watched().state.is_leader()
    }

    /// Get leader information if this node is currently a leader.
    ///
    /// Returns [`Leader`] containing the leader ID and health metadata if this node is the leader
    /// (i.e., its vote has been accepted by a quorum), otherwise returns
    /// [`ForwardToLeader`] error containing the current known leader information.
    ///
    /// # Example
    ///
    /// ```ignore
    /// match raft.as_leader() {
    ///     Ok(leader) => {
    ///         println!("This node is leader: {:?}", leader.leader_id());
    ///     }
    ///     Err(forward) => {
    ///         println!("Forward to leader: {:?}", forward.leader_id);
    ///     }
    /// }
    /// ```
    ///
    /// [`ForwardToLeader`]: crate::errors::ForwardToLeader
    #[since(version = "0.10.0")]
    pub fn as_leader(&self) -> Result<Leader<C, SM>, ForwardToLeader<C>> {
        // Do not use `is_leader()`, which depends on other state to determine, which may result in
        // inconsistent state. And `is_leader()` just do another reading from the metrics, which also may be
        // inconsistent.

        let metrics = self.inner.rx_metrics.borrow_watched();

        let Some(committed_vote) = metrics.vote.try_to_committed() else {
            return Err(ForwardToLeader::empty());
        };

        let leader_id = committed_vote.leader_id();
        let node_id = leader_id.node_id();

        if node_id == &self.inner.id {
            Ok(Leader {
                raft: self.clone(),
                leader_id: leader_id.clone(),
                last_quorum_acked: metrics.last_quorum_acked.map(|s| s.into_inner()),
            })
        } else {
            let node = metrics.membership_config.membership().get_node(node_id).cloned();

            Err(ForwardToLeader {
                leader_id: Some(node_id.clone()),
                leader_node: node,
            })
        }
    }

    /// Get the ID of this Raft node.
    ///
    /// # Example
    ///
    /// ```ignore
    /// let id = raft.node_id();
    /// println!("Node ID: {:?}", id);
    /// ```
    #[since(version = "0.10.0")]
    pub fn node_id(&self) -> &C::NodeId {
        &self.inner.id
    }

    /// Get an iterator over the current voter node IDs.
    ///
    /// Returns node IDs that are voters in the effective membership. Learners are not included.
    ///
    /// # Example
    ///
    /// ```ignore
    /// for voter_id in raft.voter_ids() {
    ///     println!("Voter: {:?}", voter_id);
    /// }
    /// ```
    #[since(version = "0.10.0")]
    pub fn voter_ids(&self) -> impl Iterator<Item = C::NodeId> {
        // borrow_watched() holds a lock that blocks RaftCore.
        // Clone and collect immediately to release the lock quickly.
        let membership = self.inner.rx_metrics.borrow_watched().membership_config.clone();
        membership.voter_ids().collect::<Vec<_>>().into_iter()
    }

    /// Get an iterator over the current learner node IDs.
    ///
    /// Returns node IDs that are learners in the effective membership. Voters are not included.
    ///
    /// # Example
    ///
    /// ```ignore
    /// for learner_id in raft.learner_ids() {
    ///     println!("Learner: {:?}", learner_id);
    /// }
    /// ```
    #[since(version = "0.10.0")]
    pub fn learner_ids(&self) -> impl Iterator<Item = C::NodeId> {
        // borrow_watched() holds a lock that blocks RaftCore.
        // Clone and collect immediately to release the lock quickly.
        let membership = self.inner.rx_metrics.borrow_watched().membership_config.clone();
        membership.membership().learner_ids().collect::<Vec<_>>().into_iter()
    }

    /// Create a new [`ProtocolApi`] to handle Raft protocol RPCs received by this Raft node.
    ///
    /// [`ProtocolApi`] provides the following protocol APIs:
    /// - [`ProtocolApi::append_entries`]
    /// - [`ProtocolApi::vote`]
    /// - [`ProtocolApi::get_snapshot`]
    /// - [`ProtocolApi::begin_receiving_snapshot`]
    /// - [`ProtocolApi::install_full_snapshot`]
    /// - [`ProtocolApi::handle_transfer_leader`]
    pub(crate) fn protocol_api(&self) -> ProtocolApi<C> {
        ProtocolApi::new(self.inner.clone())
    }

    pub(crate) fn app_api(&self) -> AppApi<'_, C> {
        AppApi::new(&self.inner)
    }

    pub(crate) fn management_api(&self) -> ManagementApi<'_, C> {
        ManagementApi::new(self.inner.as_ref())
    }

    /// Return a [`Trigger`] handle to manually trigger raft actions, such as elect or build
    /// snapshot.
    ///
    /// Example:
    /// ```ignore
    /// let raft = Raft::new(...).await?;
    /// raft.trigger().elect().await?;
    /// ```
    pub fn trigger(&self) -> Trigger<'_, C> {
        Trigger::new(self.inner.as_ref())
    }

    /// Set or unset a custom metrics recorder for exporting Raft metrics.
    ///
    /// This allows applications to plug in their own metrics collection backends
    /// (e.g., OpenTelemetry, Prometheus, StatsD) at runtime. The recorder will
    /// receive metrics events from RaftCore as they occur.
    ///
    /// Pass `Some(recorder)` to enable metrics recording, or `None` to disable it.
    ///
    /// # Example
    ///
    /// ```ignore
    /// use std::sync::Arc;
    /// use openraft::metrics::MetricsRecorder;
    ///
    /// struct MyRecorder;
    /// impl MetricsRecorder for MyRecorder {
    ///     fn record_apply_batch(&self, entry_count: u64) { /* ... */ }
    ///     fn record_append_batch(&self, entry_count: u64) { /* ... */ }
    /// }
    ///
    /// // Enable metrics recording
    /// raft.set_metrics_recorder(Some(Arc::new(MyRecorder))).await?;
    ///
    /// // Disable metrics recording
    /// raft.set_metrics_recorder(None).await?;
    /// ```
    ///
    /// # Errors
    ///
    /// Returns [`Fatal`] error if RaftCore is shut down or has a storage error.
    pub async fn set_metrics_recorder(&self, recorder: Option<Arc<dyn MetricsRecorder>>) -> Result<(), Fatal<C>> {
        self.inner.send_external_command(ExternalCommand::SetMetricsRecorder { recorder }).await
    }

    /// Submit an AppendEntries RPC to this Raft node.
    ///
    /// These RPCs are sent by the cluster leader to replicate log entries (§5.3), and are also
    /// used as heartbeats (§5.2).
    #[tracing::instrument(level = "debug", skip_all)]
    pub async fn append_entries(&self, rpc: AppendEntriesRequest<C>) -> Result<AppendEntriesResponse<C>, RaftError<C>> {
        self.protocol_api().append_entries(rpc).await.into_raft_result()
    }

    /// Submit a stream of AppendEntries RPCs to this Raft node.
    ///
    /// This is a stream-oriented version of [`Self::append_entries`] with pipelining support.
    /// It spawns a background task that reads from the input stream, sends requests to RaftCore,
    /// and forwards response receivers to the output stream. Responses are yielded in order.
    ///
    /// ## Pipelining Behavior
    ///
    /// - A background task reads from the input stream and sends to RaftCore
    /// - Uses a bounded channel (64 slots) for backpressure
    /// - Responses are yielded in order (FIFO) as they complete
    ///
    /// ## Output
    ///
    /// The output stream emits:
    /// - `Ok(log_id)` when logs are successfully flushed
    /// - `Err(e)` when an error occurs, which terminates the stream
    ///
    /// ## Pinning
    ///
    /// The returned stream is `!Unpin` because it uses async closures internally.
    /// You must pin the stream before calling `.next()`:
    ///
    /// ```ignore
    /// use std::pin::pin;
    ///
    /// let mut output = pin!(raft.stream_append(input));
    /// while let Some(result) = output.next().await { /* ... */ }
    /// ```
    ///
    /// Alternatively, use `Box::pin` for heap pinning if the stream needs to be stored or returned:
    ///
    /// ```ignore
    /// let mut output = Box::pin(raft.stream_append(input));
    /// ```
    ///
    /// # Example
    ///
    /// ```ignore
    /// use std::pin::pin;
    /// use futures_util::StreamExt;
    ///
    /// let input_stream = futures_util::stream::iter(vec![request1, request2, request3]);
    /// let mut output_stream = pin!(raft.stream_append(input_stream));
    ///
    /// while let Some(result) = output_stream.next().await {
    ///     match result {
    ///         Ok(Ok(log_id)) => println!("Flushed: {:?}", log_id),
    ///         Ok(Err(err)) => {
    ///             println!("Append error: {}", err);
    ///             break;
    ///         }
    ///         Err(fatal) => {
    ///             println!("Fatal: {}", fatal);
    ///             break;
    ///         }
    ///     }
    /// }
    /// ```
    #[since(version = "0.10.0", change = "stream item contains Fatal")]
    #[since(version = "0.10.0")]
    pub fn stream_append<S>(
        &self,
        stream: S,
    ) -> impl futures_util::Stream<Item = Result<StreamAppendResult<C>, Fatal<C>>> + OptionalSend + 'static
    where
        S: futures_util::Stream<Item = AppendEntriesRequest<C>> + OptionalSend + 'static,
    {
        self.protocol_api().stream_append(stream)
    }

    /// Submit a VoteRequest (RequestVote in the spec) RPC to this Raft node.
    ///
    /// These RPCs are sent by cluster peers which are in candidate state attempting to gather votes
    /// (§5.2).
    #[tracing::instrument(level = "debug", skip_all)]
    pub async fn vote(&self, rpc: VoteRequest<C>) -> Result<VoteResponse<C>, RaftError<C>> {
        self.protocol_api().vote(rpc).await.into_raft_result()
    }

    /// Get the latest snapshot from the state machine.
    ///
    /// It returns error only when `RaftCore` fails to serve the request, e.g., Encountering a
    /// storage error or shutting down.
    #[tracing::instrument(level = "debug", skip_all)]
    pub async fn get_snapshot(&self) -> Result<Option<SnapshotOf<C>>, RaftError<C>> {
        self.protocol_api().get_snapshot().await.into_raft_result()
    }

    /// Get a snapshot data for receiving snapshot from the leader.
    #[since(version = "0.10.0", change = "SnapshotData without Box")]
    #[tracing::instrument(level = "debug", skip_all)]
    pub async fn begin_receiving_snapshot(&self) -> Result<SnapshotDataOf<C>, RaftError<C>> {
        self.protocol_api().begin_receiving_snapshot().await.into_raft_result()
    }

    /// Install a completely received snapshot to the state machine.
    ///
    /// This method is used to implement an application defined snapshot transmission.
    /// The application receives a snapshot from the leader, in chunks or a stream, and
    /// then rebuild a snapshot, then pass the snapshot to Raft to install.
    #[since(version = "0.9.0")]
    #[tracing::instrument(level = "debug", skip_all)]
    pub async fn install_full_snapshot(
        &self,
        vote: VoteOf<C>,
        snapshot: SnapshotOf<C>,
    ) -> Result<SnapshotResponse<C>, Fatal<C>> {
        self.protocol_api().install_full_snapshot(vote, snapshot).await
    }

    /// Get the ID of the current leader from this Raft node.
    ///
    /// This method is based on the Raft metrics system which does a good job at staying
    /// up-to-date; however, the `is_leader` method must still be used to guard against stale
    /// reads. This method is perfect for making decisions on where to route client requests.
    #[tracing::instrument(level = "debug", skip(self))]
    pub async fn current_leader(&self) -> Option<C::NodeId> {
        self.metrics().borrow_watched().current_leader.clone()
    }

    /// Ensures reads performed after this method are linearizable across the cluster
    /// using an explicitly provided policy. This method is just a shorthand for calling
    /// [`get_read_log_id()`](Raft::get_read_log_id) and then calling [Raft::wait].
    ///
    /// This method is just a shorthand for combining calling
    /// [`Raft::get_read_linearizer()`](Self::get_read_linearizer) and
    /// [`Linearizer::try_await_ready()`](Linearizer::try_await_ready), i.e., it is
    /// equivalent to:
    /// ```ignore
    /// my_raft.get_read_linearizer(read_policy).await?.try_await_ready(&my_raft, None).await?;
    /// ```
    ///
    /// To support follower read, i.e., get `read_log_id` on a remote leader then read on local
    /// state machine, see [`Raft::get_read_linearizer`].
    ///
    /// The `read_policy` defines the policy to ensure leadership. See: [`ReadPolicy`].
    ///
    /// Returns:
    /// - `Ok(read_log_id)` on successful confirmation that the node is the leader. `read_log_id`
    ///   represents the log id up to which the state machine has applied to ensure a linearizable
    ///   read.
    /// - `Err(RaftError<LinearizableReadError>)` if fails to assert leadership.
    ///
    /// # Examples
    /// ```ignore
    /// // Use a strict policy for this specific critical read
    /// my_raft.ensure_linearizable(ReadPolicy::ReadIndex).await?;
    ///
    /// // Or use a more performant policy when consistency requirements are less strict
    /// my_raft.ensure_linearizable(ReadPolicy::LeaseRead).await?;
    ///
    /// // Then proceed with the state machine read
    /// ```
    /// Read more about how it works: [Read Operation](crate::docs::protocol::read)
    #[since(version = "0.9.0")]
    #[tracing::instrument(level = "debug", skip(self))]
    pub async fn ensure_linearizable(
        &self,
        read_policy: ReadPolicy,
    ) -> Result<Option<LogIdOf<C>>, RaftError<C, LinearizableReadError<C>>> {
        let linearizer = self.app_api().get_read_linearizer(read_policy).await.into_raft_result()?;

        // Safe unwrap: it never times out.
        let state = linearizer.await_ready(self).await?;
        Ok(Some(state.read_log_id().clone()))
    }

    /// Legacy method that returns log IDs directly. Use
    /// [`Raft::get_read_linearizer`] instead.
    ///
    /// This method extracts log IDs from a [`Linearizer`] and returns them as a tuple.
    /// **For new code, use [`Raft::get_read_linearizer`]** which provides a better API.
    ///
    /// See [`Raft::get_read_linearizer`] for full documentation.
    #[since(version = "0.9.0")]
    #[tracing::instrument(level = "debug", skip(self))]
    pub async fn get_read_log_id(
        &self,
        read_policy: ReadPolicy,
    ) -> Result<(Option<LogIdOf<C>>, Option<LogIdOf<C>>), RaftError<C, LinearizableReadError<C>>> {
        let linearizer = self.app_api().get_read_linearizer(read_policy).await.into_raft_result()?;

        let read_log_id = linearizer.read_log_id();
        let applied = linearizer.applied();

        Ok((Some(read_log_id.clone()), applied.cloned()))
    }

    /// Ensures this node is leader and returns a [`Linearizer`] to linearize reads.
    ///
    /// This method confirms leadership and provides the necessary information to linearize reads
    /// across the cluster. The leadership is ensured by sending heartbeats or by lease according
    /// to the specified policy. See: [`ReadPolicy`].
    ///
    /// Returns:
    /// - `Ok(Linearizer<C>)` on successful confirmation that the node is the leader. The
    ///   [`Linearizer`] contains the `read_log_id` up to which the state machine should apply to
    ///   linearize reads, and the last `applied` log id.
    /// - `Err(RaftError<LinearizableReadError>)` if this node fails to ensure its leadership, for
    ///   example, it detects a higher term, or fails to communicate with a quorum.
    ///
    /// Once returned, the caller should block until the state machine to apply up to `read_log_id`
    /// using [`Linearizer::try_await_ready`].
    ///
    /// # Examples
    /// ```ignore
    /// let linearizer = my_raft.get_read_linearizer(ReadPolicy::ReadIndex).await?;
    /// let _ = linearizer.try_await_ready(&my_raft, None).await?.unwrap();
    ///
    /// // Following read from state machine is linearized across the cluster
    /// let val = my_raft.with_state_machine(|sm| { sm.read("foo") }).await?;
    /// ```
    ///
    /// # Follower Read
    ///
    /// For follower reads, obtain the `read_log_id` from the leader via application-defined RPC,
    /// then use [`Linearizer::try_await_ready`] to wait for local state machine to catch
    /// up.
    ///
    /// ```ignore
    /// // Application defined RPC to get the `read_log_id` from the remote leader
    /// let leader_id = my_raft.current_leader().await?.unwrap();
    /// let linearizer = my_app_rpc.get_read_linearizer(leader_id, ReadPolicy::ReadIndex).await?;
    ///
    /// // Block waiting local state machine to apply up to to the `read_log_id`
    /// let _ = linearizer.try_await_ready(&my_raft, None).await?.unwrap();
    ///
    /// // Following read from state machine is linearized across the cluster
    /// let val = my_raft.with_state_machine(|sm| { sm.read("foo") }).await?;
    /// ```
    ///
    /// See: [Read Operation](crate::docs::protocol::read)
    #[since(version = "0.10.0")]
    #[tracing::instrument(level = "debug", skip_all)]
    pub async fn get_read_linearizer(
        &self,
        read_policy: ReadPolicy,
    ) -> Result<Linearizer<C>, RaftError<C, LinearizableReadError<C>>> {
        self.app_api().get_read_linearizer(read_policy).await.into_raft_result()
    }

    /// Submit a mutating client request to Raft to update the state of the system (§5.1).
    ///
    /// It will be appended to the log, committed to the cluster, and then applied to the
    /// application state machine. The result of applying the request to the state machine will
    /// be returned as the response from this method.
    ///
    /// Our goal for Raft is to implement linearizable semantics. If the leader crashes after
    /// committing a log entry but before responding to the client, the client may retry the
    /// command with a new leader, causing it to be executed a second time. As such, clients
    /// should assign unique serial numbers to every command. Then, the state machine should
    /// track the latest serial number processed for each client, along with the associated
    /// response. If it receives a command whose serial number has already been executed, it
    /// responds immediately without re-executing the request (§8). The
    /// [`RaftStateMachine::apply`] method is the perfect place to implement
    /// this.
    ///
    /// These are application specific requirements, and must be implemented by the application
    /// which is being built on top of Raft.
    ///
    /// # Examples
    ///
    /// ```ignore
    /// // Submit a write request
    /// let request = MyAppData { key: "foo".to_string(), value: "bar".to_string() };
    /// let response = raft.client_write(request).await?;
    /// println!("Applied at log index: {:?}", response.log_id);
    /// ```
    #[tracing::instrument(level = "debug", skip(self, app_data))]
    pub async fn client_write(
        &self,
        app_data: C::D,
    ) -> Result<ClientWriteResponse<C>, RaftError<C, ClientWriteError<C>>> {
        self.app_api().client_write(EntryPayload::Normal(app_data)).await.into_raft_result()
    }

    /// Write a blank log entry to the Raft log.
    ///
    /// A blank entry contains no application data and is typically used to:
    /// - Commit entries from previous terms when a new leader is elected
    /// - Advance the commit index without any state machine changes
    /// - Act as a barrier to ensure all previous entries are committed
    ///
    /// Returns when the blank entry has been applied to the state machine.
    #[since(version = "0.10.0")]
    #[tracing::instrument(level = "debug", skip(self))]
    pub async fn write_blank(&self) -> Result<ClientWriteResponse<C>, RaftError<C, ClientWriteError<C>>> {
        self.app_api().client_write(EntryPayload::Blank).await.into_raft_result()
    }

    /// Submit a mutating client request to Raft to update the state machine, returns an application
    /// defined response receiver [`Responder::Receiver`].
    ///
    /// `_ff` means fire and forget.
    ///
    /// It is same as [`Self::client_write`] but does not wait for the response.
    #[since(version = "0.10.0", date = "2025-10-27", change = "add responder arg")]
    #[since(version = "0.10.0")]
    pub async fn client_write_ff(
        &self,
        app_data: C::D,
        responder: Option<WriteResponderOf<C>>,
    ) -> Result<(), Fatal<C>> {
        self.app_api().client_write_ff(EntryPayload::Normal(app_data), responder).await
    }

    /// Write multiple application data payloads in a single batch.
    ///
    /// Returns a stream that yields each result in submission order.
    /// This is more efficient than calling [`client_write()`](Self::client_write) multiple times
    /// as it sends all payloads in a single message to the Raft core.
    ///
    /// # Examples
    ///
    /// ```ignore
    /// use futures_util::TryStreamExt;
    ///
    /// let mut stream = raft.client_write_many([data1, data2, data3]).await?;
    ///
    /// // try_next() extracts Fatal error, result is WriteResult
    /// while let Some(result) = stream.try_next().await? {
    ///     match result {
    ///         Ok(response) => println!("Applied at log index: {:?}", response.log_id),
    ///         Err(forward_err) => eprintln!("Forward to leader: {:?}", forward_err),
    ///     }
    /// }
    /// ```
    #[since(version = "0.10.0")]
    #[tracing::instrument(level = "debug", skip_all)]
    pub async fn client_write_many(
        &self,
        app_data: impl IntoIterator<Item = C::D>,
    ) -> Result<BoxStream<'static, Result<WriteResult<C>, Fatal<C>>>, Fatal<C>> {
        self.app_api().client_write_many(app_data.into_iter().map(EntryPayload::Normal)).await
    }

    /// Submit a write request to Raft.
    ///
    /// Returns a [`WriteRequest`] builder. Fire-and-forget by default;
    /// use [`.responder()`] for results, [`.with_leader()`] for conditional writes.
    ///
    /// # Examples
    ///
    /// ```ignore
    /// use openraft::impls::ProgressResponder;
    ///
    /// // Fire-and-forget
    /// raft.write(my_data).await?;
    ///
    /// // With responder
    /// let (responder, _, rx) = ProgressResponder::new();
    /// raft.write(my_data).responder(responder).await?;
    /// let result = rx.await??;
    ///
    /// // Conditional write (fails if leader changed)
    /// let leader_id = raft.as_leader()?.to_committed_leader_id();
    /// raft.write(my_data)
    ///     .with_leader(leader_id)
    ///     .responder(responder)
    ///     .await?;
    /// ```
    ///
    /// [`.responder()`]: WriteRequest::responder
    /// [`.with_leader()`]: WriteRequest::with_leader
    #[since(version = "0.10.0")]
    pub fn write(&self, app_data: C::D) -> WriteRequest<'_, C> {
        WriteRequest {
            inner: &self.inner,
            app_data,
            responder: None,
            expected_leader: None,
        }
    }

    /// Handle the LeaderTransfer request from a Leader node.
    ///
    /// If this node is the `to` node, it resets the Leader lease and triggers an election when the
    /// expected log entries are flushed.
    /// Otherwise, it just resets the Leader lease to allow the `to` node to become the Leader.
    ///
    /// The application calls
    /// [`Raft::trigger().transfer_leader()`](crate::raft::trigger::Trigger::transfer_leader) to
    /// submit Transfer Leader command. Then, the current Leader will broadcast it to every node in
    /// the cluster via [`RaftNetworkV2::transfer_leader`] and the implementation on the remote node
    /// responds to transfer leader request by calling this method.
    ///
    /// [`RaftNetworkV2::transfer_leader`]: crate::network::RaftNetworkV2::transfer_leader
    #[since(version = "0.10.0")]
    #[tracing::instrument(level = "debug", skip_all)]
    pub async fn handle_transfer_leader(&self, req: TransferLeaderRequest<C>) -> Result<(), Fatal<C>> {
        self.protocol_api().handle_transfer_leader(req).await
    }

    /// Return `true` if this node is already initialized and cannot be initialized again with
    /// [`Raft::initialize`]
    #[since(version = "0.10.0")]
    pub async fn is_initialized(&self) -> Result<bool, Fatal<C>> {
        let initialized = self.with_raft_state(|st| st.is_initialized()).await?;

        Ok(initialized)
    }

    /// Initialize a pristine Raft node with the given config.
    ///
    /// This command should be called on pristine nodes — where the log index is 0 and the node is
    /// in Learner state — as if either of those constraints are false, it indicates that the
    /// cluster is already formed and in motion. If `InitializeError::NotAllowed` is returned
    /// from this function, it is safe to ignore, as it simply indicates that the cluster is
    /// already up and running, which is ultimately the goal of this function. You can check
    /// if the cluster is initialized with [`Raft::is_initialized()`] and then avoid re-initialize
    /// it in case you want to get rid of this error.
    ///
    /// ## Recommended Usage
    ///
    /// The simplest and most appropriate way to initialize a cluster is to call `initialize()`
    /// on **exactly one node**. The other nodes should remain empty and wait for the initialized
    /// node to replicate logs to them.
    ///
    /// Calling `initialize()` on multiple nodes with **identical configuration** is also
    /// acceptable and will not cause any consistency issues — the Raft voting protocol ensures
    /// that only one leader will be elected.
    ///
    /// However, calling `initialize()` with **different configurations** on different nodes
    /// may lead to a split-brain condition and must be avoided.
    ///
    /// ## Behavior
    ///
    /// Once a node is successfully initialized, it will commit a new membership config
    /// log entry to store, then enter Candidate state and attempt to elect itself as the
    /// leader.
    ///
    /// # Examples
    ///
    /// ```ignore
    /// use std::collections::BTreeMap;
    /// use openraft::BasicNode;
    ///
    /// // Initialize a single-node cluster
    /// let mut nodes = BTreeMap::new();
    /// nodes.insert(1, BasicNode { addr: "127.0.0.1:8080".to_string() });
    /// raft.initialize(nodes).await?;
    ///
    /// // Initialize a three-node cluster
    /// let mut nodes = BTreeMap::new();
    /// nodes.insert(1, BasicNode { addr: "127.0.0.1:8080".to_string() });
    /// nodes.insert(2, BasicNode { addr: "127.0.0.1:8081".to_string() });
    /// nodes.insert(3, BasicNode { addr: "127.0.0.1:8082".to_string() });
    /// raft.initialize(nodes).await?;
    /// ```
    #[tracing::instrument(level = "debug", skip(self))]
    pub async fn initialize<T>(&self, members: T) -> Result<(), RaftError<C, InitializeError<C>>>
    where T: IntoNodes<C::NodeId, C::Node> + Debug {
        self.management_api().initialize(members).await.into_raft_result()
    }

    /// Provides read-only access to [`RaftState`] through a user-provided function.
    ///
    /// The function `func` is applied to the current [`RaftState`]. The result of this function,
    /// of type `V`, is returned wrapped in `Result<V, Fatal<C>>`. `Fatal` error will be
    /// returned if failed to receive a reply from `RaftCore`.
    ///
    /// A `Fatal` error is returned if:
    /// - Raft core task is stopped normally.
    /// - Raft core task is panicked due to programming error.
    /// - Raft core task is encountered a storage error.
    ///
    /// Example for getting the current committed log id:
    /// ```ignore
    /// let committed = my_raft.with_raft_state(|st| st.committed).await?;
    /// ```
    pub async fn with_raft_state<F, V>(&self, func: F) -> Result<V, Fatal<C>>
    where
        F: FnOnce(&RaftState<C>) -> V + OptionalSend + 'static,
        V: OptionalSend + 'static,
    {
        let (tx, rx) = C::oneshot();

        self.external_request(|st| {
            let result = func(st);
            if let Err(_err) = tx.send(result) {
                tracing::error!("{}: to-Raft tx send error", func_name!());
            }
        })
        .await?;

        match rx.await {
            Ok(res) => Ok(res),
            Err(err) => {
                tracing::error!("{}: rx recv error: {}", func_name!(), err);
                let fatal = self.inner.get_core_stop_error().await;
                Err(fatal)
            }
        }
    }

    /// Send a request to the Raft core loop in a fire-and-forget manner.
    ///
    /// This method returns immediately after sending the message to the Raft core loop,
    /// without waiting for the request to be executed. The returned `Result` indicates
    /// whether the message was successfully sent, not whether the request was executed.
    ///
    /// The request functor will be called with an immutable reference to the [`RaftState`]
    /// and serialized with other Raft core loop processing (e.g., client requests
    /// or general state changes).
    ///
    /// If a response is required, then the caller can store the sender of a one-shot channel
    /// in the closure of the request functor, which can then be used to send the response
    /// asynchronously.
    ///
    /// Returns a `Fatal` error if:
    /// - Raft core task is stopped normally.
    /// - Raft core task is panicked due to programming error.
    /// - Raft core task is encountered a storage error.
    pub async fn external_request<F>(&self, req: F) -> Result<(), Fatal<C>>
    where F: FnOnce(&RaftState<C>) + OptionalSend + 'static {
        let req: BoxOnce<'static, RaftState<C>> = Box::new(req);
        self.inner.send_msg(RaftMsg::WithRaftState { req }).await
    }

    /// Get a handle to the metrics channel.
    ///
    /// # Examples
    ///
    /// ```ignore
    /// // Get current metrics
    /// let metrics = raft.metrics().borrow_watched().clone();
    /// println!("Current leader: {:?}", metrics.current_leader);
    /// println!("Current term: {}", metrics.current_term);
    /// ```
    pub fn metrics(&self) -> WatchReceiverOf<C, RaftMetrics<C>> {
        self.inner.rx_metrics.clone()
    }

    /// Get a handle to the data metrics channel.
    pub fn data_metrics(&self) -> WatchReceiverOf<C, RaftDataMetrics<C>> {
        self.inner.rx_data_metrics.clone()
    }

    /// Get a handle to the server metrics channel.
    pub fn server_metrics(&self) -> WatchReceiverOf<C, RaftServerMetrics<C>> {
        self.inner.rx_server_metrics.clone()
    }

    /// Get a handle to watch log I/O flush progress.
    ///
    /// Tracks when log entries and votes are durably written to storage.
    /// Updated on every I/O completion (vote saves and log appends).
    ///
    /// # Example
    ///
    /// ```ignore
    /// let mut log_progress = raft.watch_log_progress();
    ///
    /// // Wait for a specific log entry to be flushed
    /// let target = Some(FlushPoint::new(
    ///     Vote::new_committed(2, node_id),
    ///     Some(LogId::new(LeaderId::new(2, node_id), 100))
    /// ));
    /// log_progress.wait_until_ge(&target).await?;
    /// ```
    #[since(version = "0.10.0")]
    #[must_use = "progress handle should be stored to track I/O progress"]
    pub fn watch_log_progress(&self) -> LogProgress<C> {
        self.inner.progress_watcher.log_progress()
    }

    /// Get a handle to watch vote I/O flush progress.
    ///
    /// Tracks when votes (leadership changes) are durably written to storage.
    /// Updated only when the vote changes (new term or leader), not on every log append.
    ///
    /// Use this when you only care about leadership changes, not specific log entries.
    ///
    /// # Example
    ///
    /// ```ignore
    /// let mut vote_progress = raft.watch_vote_progress();
    ///
    /// // Wait for term 2 to be persisted
    /// let target = Some(Vote::new_committed(2, 0));
    /// vote_progress.wait_until_ge(&target).await?;
    /// ```
    #[since(version = "0.10.0")]
    #[must_use = "progress handle should be stored to track vote progress"]
    pub fn watch_vote_progress(&self) -> VoteProgress<C> {
        self.inner.progress_watcher.vote_progress()
    }

    /// Get a handle to watch commit log progress.
    ///
    /// Tracks when committed logs advance(persisted on a quorum and the last-log is proposed by the
    /// leader). Updated whenever the committed cursor moves forward.
    ///
    /// # Example
    ///
    /// ```ignore
    /// let mut commit_progress = raft.watch_commit_progress();
    ///
    /// // Wait until log index 42 is committed
    /// let target = Some(LogId::new(LeaderId::new(2, node_id), 42));
    /// commit_progress.wait_until_ge(&target).await?;
    /// ```
    #[since(version = "0.10.0")]
    #[must_use = "progress handle should be stored to track commit progress"]
    pub fn watch_commit_progress(&self) -> CommitProgress<C> {
        self.inner.progress_watcher.commit_progress()
    }

    /// Get a handle to watch snapshot persistence progress.
    ///
    /// Tracks when snapshots are persisted to storage.
    /// Updated whenever a snapshot is built or installed and persisted.
    ///
    /// # Example
    ///
    /// ```ignore
    /// let mut snapshot_progress = raft.watch_snapshot_progress();
    ///
    /// // Wait until snapshot covering log index 100 is persisted
    /// let target = Some(LogId::new(LeaderId::new(2, node_id), 100));
    /// snapshot_progress.wait_until_ge(&target).await?;
    /// ```
    #[since(version = "0.10.0")]
    #[must_use = "progress handle should be stored to track snapshot progress"]
    pub fn watch_snapshot_progress(&self) -> SnapshotProgress<C> {
        self.inner.progress_watcher.snapshot_progress()
    }

    /// Get a handle to watch applied log progress.
    ///
    /// Tracks when logs are applied to the state machine.
    /// Updated whenever the last applied log id advances.
    ///
    /// # Note
    ///
    /// If the state machine does not persist the applied state immediately, the watcher
    /// may observe duplicate events when the server restarts and re-applies log entries.
    ///
    /// # Example
    ///
    /// ```ignore
    /// let mut apply_progress = raft.watch_apply_progress();
    ///
    /// // Wait until log index 42 is applied
    /// let target = Some(LogId::new(LeaderId::new(2, node_id), 42));
    /// apply_progress.wait_until_ge(&target).await?;
    /// ```
    #[since(version = "0.10.0")]
    #[must_use = "progress handle should be stored to track applied progress"]
    pub fn watch_apply_progress(&self) -> AppliedProgress<C> {
        self.inner.progress_watcher.apply_progress()
    }

    /// Watch for any leader changes in the cluster and invoke callback on each change.
    ///
    /// Returns a [`WatchChangeHandle`] that must be held to keep watching.
    /// If the handle is dropped or [`WatchChangeHandle::close()`] is called,
    /// the background task will be terminated and the callback will no longer be invoked.
    ///
    /// The callback receives:
    /// - `old`: The previous leader state `(leader_id, committed)`, or `None` on the first callback
    /// - `new`: The current leader state `(leader_id, committed)`
    ///
    /// This fires on ANY leader change in the cluster, not just when this node's leadership
    /// changes. For a simpler API that only fires when THIS node becomes or loses leadership,
    /// see [`on_leader_change()`].
    ///
    /// # Note on Start/Stop Service Pattern
    ///
    /// If you use this API to start/stop services based on leadership, be aware that
    /// consecutive callbacks may show the same node as leader with different Terms
    /// (e.g., Term 1 → Term 2). The simple `if is_leader { start } else { stop }` pattern
    /// could call `start` twice without an intervening `stop`.
    ///
    /// For the start/stop service pattern, prefer [`on_leader_change()`] which guarantees
    /// alternating `start`/`stop` callbacks.
    ///
    /// # Example
    ///
    /// ```ignore
    /// let my_node_id = 1;
    ///
    /// let mut handle = raft.on_cluster_leader_change(move |_old, (leader_id, committed)| {
    ///     let is_leader = leader_id.node_id == my_node_id && committed;
    ///
    ///     async move {
    ///         if is_leader {
    ///             // This node just became the committed leader
    ///             // Start leader-only services (e.g., cron jobs, cache warming)
    ///             start_leader_services().await;
    ///         } else {
    ///             // This node is no longer the leader
    ///             // Stop leader-only services to avoid duplicate work
    ///             stop_leader_services().await;
    ///         }
    ///     }
    /// });
    ///
    /// // Later, stop watching
    /// handle.close().await;
    /// ```
    ///
    /// [`on_leader_change()`]: Self::on_leader_change
    #[since(version = "0.10.0")]
    #[must_use = "handle must be held to keep the watch task running"]
    pub fn on_cluster_leader_change<F, Fut>(&self, mut callback: F) -> WatchChangeHandle<C>
    where
        F: FnMut(Option<(C::LeaderId, bool)>, (C::LeaderId, bool)) -> Fut + OptionalSend + 'static,
        Fut: Future<Output = ()> + OptionalSend + 'static,
    {
        let mut prev_vote: Option<Vote<C::LeaderId>> = None;

        self.watch_vote_change(move |new_vote, _my_node_id| {
            let old_leader = prev_vote.as_ref().map(|v| v.leader_id().clone());
            let new_leader = new_vote.leader_id().clone();

            // Only call callback if leader_id actually changed
            let fut = if old_leader.as_ref() != Some(&new_leader) {
                let old_state = prev_vote.as_ref().map(|v| (v.leader_id().clone(), v.is_committed()));
                let new_state = (new_vote.leader_id().clone(), new_vote.is_committed());
                Some(callback(old_state, new_state))
            } else {
                None
            };
            prev_vote = Some(new_vote);

            async move {
                if let Some(f) = fut {
                    f.await;
                }
            }
        })
    }

    /// Register callbacks for when this node becomes or stops being the committed leader.
    ///
    /// Returns a [`WatchChangeHandle`] that must be held to keep watching.
    /// If the handle is dropped or [`WatchChangeHandle::close()`] is called,
    /// the background task will be terminated and the callbacks will no longer be invoked.
    ///
    /// Unlike [`on_cluster_leader_change()`] which fires on any leader change in the cluster,
    /// this method only fires when THIS node becomes or stops being the leader.
    ///
    /// - `start`: Called when this node becomes the leader (committed, quorum-acknowledged)
    /// - `stop`: Called when this node is no longer the leader (another node becomes leader)
    ///
    /// # Callback Guarantees
    ///
    /// The `start` and `stop` callbacks are guaranteed to be called in alternating order:
    /// `start` → `stop` → `start` → `stop` → ...
    ///
    /// Even if a node transitions directly from leader in Term 1 to leader in Term 2,
    /// `stop` will be called with the old `leader_id` before `start` is called with the
    /// new `leader_id`. This ensures proper resource cleanup between leadership terms.
    ///
    /// # Example
    ///
    /// ```ignore
    /// let handle = raft.on_leader_change(
    ///     |leader_id| async move {
    ///         println!("Became leader: {:?}", leader_id);
    ///         start_leader_services().await;
    ///     },
    ///     |old_leader| async move {
    ///         println!("Stopped leading: {:?}", old_leader);
    ///         stop_leader_services().await;
    ///     },
    /// );
    ///
    /// // Later, stop watching
    /// handle.close().await;
    /// ```
    ///
    /// [`on_cluster_leader_change()`]: Self::on_cluster_leader_change
    #[since(version = "0.10.0")]
    #[must_use = "handle must be held to keep the watch task running"]
    pub fn on_leader_change<F1, F2, Fut1, Fut2>(&self, start: F1, stop: F2) -> WatchChangeHandle<C>
    where
        F1: Fn(C::LeaderId) -> Fut1 + OptionalSend + 'static,
        F2: Fn(C::LeaderId) -> Fut2 + OptionalSend + 'static,
        Fut1: Future<Output = ()> + OptionalSend + 'static,
        Fut2: Future<Output = ()> + OptionalSend + 'static,
    {
        let mut prev_leader_id = None;

        self.watch_vote_change(move |vote, my_node_id| {
            let leader_id = vote.leader_id().clone();

            // Fire `start` when THIS node becomes committed leader
            // and it's a new leadership (different from current)
            #[allow(clippy::collapsible_else_if)]
            let (stop_fut, start_fut) = if leader_id.node_id() == my_node_id {
                if vote.is_committed() && prev_leader_id.as_ref() != Some(&leader_id) {
                    // Call stop first if transitioning from one leadership to another
                    // (e.g., Term 1 leader -> Term 2 leader)
                    // This guarantees alternating start/stop calls.
                    let stop_fut = prev_leader_id.take().map(&stop);
                    let start_fut = Some(start(leader_id.clone()));
                    prev_leader_id = Some(leader_id);
                    (stop_fut, start_fut)
                } else {
                    (None, None)
                }
            } else {
                let stop_fut = prev_leader_id.take().map(&stop);
                (stop_fut, None)
            };

            async move {
                if let Some(f) = stop_fut {
                    f.await;
                }
                if let Some(f) = start_fut {
                    f.await;
                }
            }
        })
    }

    /// Spawn a task that watches vote changes and invokes async callback on each change.
    ///
    /// This is an internal helper used by [`Self::on_leader_change()`] and
    /// [`Self::on_cluster_leader_change()`].
    ///
    /// The callback returns a future that will be awaited before processing
    /// the next vote change.
    fn watch_vote_change<F, Fut>(&self, mut callback: F) -> WatchChangeHandle<C>
    where
        F: FnMut(Vote<C::LeaderId>, &NodeIdOf<C>) -> Fut + OptionalSend + 'static,
        Fut: Future<Output = ()> + OptionalSend + 'static,
    {
        use futures_util::FutureExt;

        let my_node_id = self.inner.id().clone();
        let mut vote_progress = self.watch_vote_progress();
        let (cancel_tx, cancel_rx) = C::oneshot::<()>();

        let handle = C::spawn(async move {
            let mut cancel_rx = cancel_rx.fuse();

            loop {
                futures_util::select! {
                    _ = cancel_rx => break,
                    res = vote_progress.changed().fuse() => {
                        if res.is_err() {
                            break;
                        }
                        let Some(vote) = vote_progress.get() else {
                            continue;
                        };

                        callback(vote, &my_node_id).await;
                    }
                }
            }
        });

        WatchChangeHandle {
            cancel_tx: Some(cancel_tx),
            join_handle: Some(handle),
        }
    }

    /// Get a handle to wait for the metrics to satisfy some condition.
    ///
    ///
    /// If `timeout` is `None`, then it will wait forever(10 years).
    /// If `timeout` is `Some`, then it will wait for the specified duration.
    ///
    /// ```ignore
    /// # use std::time::Duration;
    /// # use openraft::{State, Raft};
    ///
    /// let timeout = Duration::from_millis(200);
    ///
    /// // wait for raft log-3 to be received and applied:
    /// r.wait(Some(timeout)).log(Some(3), "log").await?;
    ///
    /// // wait for ever for raft node's current leader to become 3:
    /// r.wait(None).current_leader(2, "wait for leader").await?;
    ///
    /// // wait for raft state to become a follower
    /// r.wait(None).state(State::Follower, "state").await?;
    /// ```
    pub fn wait(&self, timeout: Option<Duration>) -> Wait<C> {
        self.inner.wait(timeout)
    }

    /// Shutdown this Raft node.
    ///
    /// It sends a shutdown signal and waits until `RaftCore` returns.
    ///
    /// # Examples
    ///
    /// ```ignore
    /// // Gracefully shutdown the Raft node
    /// raft.shutdown().await?;
    /// ```
    pub async fn shutdown(&self) -> Result<(), JoinErrorOf<C>> {
        if let Some(tx) = self.inner.tx_shutdown.lock().unwrap().take() {
            // A failure to send means the RaftCore is already shutdown. Continue to check the task
            // return value.
            let send_res = tx.send(());
            tracing::info!("sending shutdown signal to RaftCore, sending res: {:?}", send_res);
        }
        self.inner.join_core_task().await;
        if let Some(join_handle) = self.inner.tick_handle.shutdown() {
            join_handle.await.ok();
        }

        // TODO(xp): API change: replace `JoinError` with `Fatal`,
        //           to let the caller know the return value of RaftCore task.
        Ok(())
    }

    /// Provides mutable access to [`RaftStateMachine`] through a user-provided function.
    ///
    /// The function `func` is applied to the current [`RaftStateMachine`]. The result of this
    /// function, of type `V`, is returned wrapped in `Result<V, Fatal<C>>`.
    ///
    /// A `Fatal` error is returned if:
    /// - Raft core task is stopped normally.
    /// - Raft core task is panicked due to programming error.
    /// - Raft core task is encountered a storage error.
    ///
    /// Example for getting the last applied log id from SM(assume there is `last_applied()` method
    /// provided):
    ///
    /// ```rust,ignore
    /// let last_applied_log_id = my_raft.with_state_machine(|sm| {
    ///     async move { sm.last_applied().await }
    /// }).await?;
    /// ```
    #[since(version = "0.10.0")]
    pub async fn with_state_machine<F, V>(&self, func: F) -> Result<V, Fatal<C>>
    where
        SM: OptionalSend + 'static,
        F: FnOnce(&mut SM) -> BoxFuture<V> + OptionalSend + 'static,
        V: OptionalSend + 'static,
    {
        let (tx, rx) = C::oneshot();

        self.external_state_machine_request(|sm| {
            Box::pin(async move {
                let resp = func(sm).await;
                if let Err(_err) = tx.send(resp) {
                    tracing::error!("{}: failed to send response to user tx", func_name!());
                }
            })
        })
        .await?;

        let recv_res = rx.await;
        tracing::debug!("{}: receives result is error: {:?}", func_name!(), recv_res.is_err());

        let Ok(v) = recv_res else {
            let fatal = self.inner.get_core_stop_error().await;
            tracing::error!("{}: error: {}", func_name!(), fatal);
            return Err(fatal);
        };

        Ok(v)
    }

    /// Send a request to the [`RaftStateMachine`] worker in a fire-and-forget manner.
    ///
    /// This method returns immediately after sending the message to the state machine worker,
    /// without waiting for the request to be executed. The returned `Result` indicates
    /// whether the message was successfully sent, not whether the request was executed.
    ///
    /// The request functor will be called with a mutable reference to the state machine.
    /// The functor returns a [`Future`] because state machine methods are `async`.
    ///
    /// Returns a `Fatal` error if:
    /// - Raft core task is stopped normally.
    /// - Raft core task is panicked due to programming error.
    /// - Raft core task is encountered a storage error.
    #[since(version = "0.10.0")]
    pub async fn external_state_machine_request<F>(&self, req: F) -> Result<(), Fatal<C>>
    where
        SM: OptionalSend + 'static,
        F: FnOnce(&mut SM) -> BoxFuture<()> + OptionalSend + 'static,
    {
        let Some(tx) = self.sm_cmd_tx.upgrade() else {
            return Err(Fatal::Stopped);
        };

        let sm_cmd = sm::Command::ExternalFunc {
            func: Box::new(move |sm| req(sm)),
        };
        tx.send(sm_cmd).await.map_err(|_e| Fatal::Stopped)
    }
}