d-engine-core 0.2.3

Pure Raft consensus algorithm - for building custom Raft-based systems
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
use std::fmt::Debug;
use std::path::PathBuf;
use std::time::Duration;

use config::ConfigError;
use serde::Deserialize;
use serde::Serialize;
use tracing::warn;

use super::lease::LeaseConfig;
use super::validate_directory;
use crate::Error;
use crate::Result;

/// Configuration parameters for the Raft consensus algorithm implementation
#[derive(Serialize, Deserialize, Clone)]
pub struct RaftConfig {
    /// Configuration settings related to log replication
    /// Includes parameters like heartbeat interval and AppendEntries entry count limit
    #[serde(default)]
    pub replication: ReplicationConfig,

    /// Client request batching configuration
    ///
    /// Controls flush thresholds for the leader's propose and linearizable-read buffers.
    /// Separate from replication config — this governs the client ingestion layer, not
    /// the Leader→Follower replication path.
    #[serde(default)]
    pub batching: BatchingConfig,

    /// Configuration settings for leader election mechanism
    /// Controls timeouts and randomization factors for election timing
    #[serde(default)]
    pub election: ElectionConfig,

    /// Configuration settings for cluster membership changes
    /// Handles joint consensus transitions and cluster reconfiguration rules
    #[serde(default)]
    pub membership: MembershipConfig,

    /// Configuration settings for state machine behavior
    /// Controls state machine operations like lease management, compaction, etc.
    /// For backward compatibility, can also be configured via `storage` in TOML files.
    #[serde(default, alias = "storage")]
    pub state_machine: StateMachineConfig,

    /// Configuration settings for snapshot feature
    #[serde(default)]
    pub snapshot: SnapshotConfig,

    /// Configuration settings for log persistence behavior
    /// Controls how and when log entries are persisted to stable storage
    #[serde(default)]
    pub persistence: PersistenceConfig,

    /// Maximum allowed log entry gap between leader and learner nodes
    /// Learners with larger gaps than this value will trigger catch-up replication
    /// Default value is set via default_learner_catchup_threshold() function
    #[serde(default = "default_learner_catchup_threshold")]
    pub learner_catchup_threshold: u64,

    /// Throttle interval (milliseconds) for learner progress checks
    /// Prevents excessive checking of learner promotion eligibility
    /// Default value is set via default_learner_check_throttle_ms() function
    #[serde(default = "default_learner_check_throttle_ms")]
    pub learner_check_throttle_ms: u64,

    /// Base timeout duration (in milliseconds) for general Raft operations
    /// Used as fallback timeout when operation-specific timeouts are not set
    /// Default value is set via default_general_timeout() function
    #[serde(default = "default_general_timeout")]
    pub general_raft_timeout_duration_in_ms: u64,

    /// Timeout for snapshot RPC operations (milliseconds)
    #[serde(default = "default_snapshot_rpc_timeout_ms")]
    pub snapshot_rpc_timeout_ms: u64,

    /// Configuration settings for new node auto join feature
    #[serde(default)]
    pub auto_join: AutoJoinConfig,

    /// Configuration for read operation consistency behavior
    /// Controls the trade-off between read performance and consistency guarantees
    #[serde(default)]
    pub read_consistency: ReadConsistencyConfig,

    /// Backpressure configuration for client request flow control
    /// Prevents unbounded memory growth by rejecting excess requests
    #[serde(default)]
    pub backpressure: BackpressureConfig,

    /// RPC compression configuration for different service types
    ///
    /// Controls which RPC service types use response compression.
    /// Allows fine-tuning for performance optimization based on
    /// deployment environment and traffic patterns.
    #[serde(default)]
    pub rpc_compression: RpcCompressionConfig,

    /// Configuration for Watch mechanism that monitors key changes
    /// Controls event queue sizes and metrics behavior
    #[serde(default)]
    pub watch: WatchConfig,

    /// Performance metrics configuration
    /// Controls metrics emission and sampling for observability vs performance trade-off
    #[serde(default)]
    pub metrics: MetricsConfig,
}

impl Debug for RaftConfig {
    fn fmt(
        &self,
        f: &mut std::fmt::Formatter<'_>,
    ) -> std::fmt::Result {
        f.debug_struct("RaftConfig").finish()
    }
}
impl Default for RaftConfig {
    fn default() -> Self {
        Self {
            replication: ReplicationConfig::default(),
            batching: BatchingConfig::default(),
            election: ElectionConfig::default(),
            membership: MembershipConfig::default(),
            state_machine: StateMachineConfig::default(),
            snapshot: SnapshotConfig::default(),
            persistence: PersistenceConfig::default(),
            learner_catchup_threshold: default_learner_catchup_threshold(),
            learner_check_throttle_ms: default_learner_check_throttle_ms(),
            general_raft_timeout_duration_in_ms: default_general_timeout(),
            auto_join: AutoJoinConfig::default(),
            snapshot_rpc_timeout_ms: default_snapshot_rpc_timeout_ms(),
            read_consistency: ReadConsistencyConfig::default(),
            backpressure: BackpressureConfig::default(),
            rpc_compression: RpcCompressionConfig::default(),
            watch: WatchConfig::default(),
            metrics: MetricsConfig::default(),
        }
    }
}
impl RaftConfig {
    /// Validates all Raft subsystem configurations
    pub fn validate(&self) -> Result<()> {
        if self.learner_catchup_threshold == 0 {
            return Err(Error::Config(ConfigError::Message(
                "learner_catchup_threshold must be greater than 0".into(),
            )));
        }

        if self.general_raft_timeout_duration_in_ms < 1 {
            return Err(Error::Config(ConfigError::Message(
                "general_raft_timeout_duration_in_ms must be at least 1ms".into(),
            )));
        }

        self.replication.validate()?;
        self.batching.validate()?;
        self.election.validate()?;
        self.membership.validate()?;
        self.state_machine.validate()?;
        self.snapshot.validate()?;
        self.read_consistency.validate()?;
        self.watch.validate()?;

        // Warn if lease duration is too long compared to election timeout
        if self.read_consistency.lease_duration_ms > self.election.election_timeout_min / 2 {
            warn!(
                "read_consistency.lease_duration_ms ({}) is greater than half of election_timeout_min ({}ms). \
                     This may cause lease expiration during normal operation.",
                self.read_consistency.lease_duration_ms,
                self.election.election_timeout_min / 2
            );
        }

        Ok(())
    }
}

fn default_learner_catchup_threshold() -> u64 {
    1
}

fn default_learner_check_throttle_ms() -> u64 {
    1000 // 1 second
}

// in ms
fn default_general_timeout() -> u64 {
    50
}
fn default_snapshot_rpc_timeout_ms() -> u64 {
    // 1 hour - sufficient for large snapshots
    3_600_000
}
#[derive(Debug, Serialize, Deserialize, Clone)]
pub struct ReplicationConfig {
    /// Heartbeat interval (milliseconds): how often the leader sends AppendEntries RPCs.
    #[serde(default = "default_append_interval")]
    pub rpc_append_entries_clock_in_ms: u64,

    /// Maximum log entries per single AppendEntries RPC to a follower.
    #[serde(default = "default_entries_per_replication")]
    pub append_entries_max_entries_per_replication: u64,
}

impl Default for ReplicationConfig {
    fn default() -> Self {
        Self {
            rpc_append_entries_clock_in_ms: default_append_interval(),
            append_entries_max_entries_per_replication: default_entries_per_replication(),
        }
    }
}
impl ReplicationConfig {
    fn validate(&self) -> Result<()> {
        if self.rpc_append_entries_clock_in_ms == 0 {
            return Err(Error::Config(ConfigError::Message(
                "rpc_append_entries_clock_in_ms cannot be 0".into(),
            )));
        }

        if self.append_entries_max_entries_per_replication == 0 {
            return Err(Error::Config(ConfigError::Message(
                "append_entries_max_entries_per_replication must be > 0".into(),
            )));
        }

        Ok(())
    }
}

/// Batching configuration for leader-side drain loops and buffer allocation.
///
/// A single value intentionally covers all drain loops because they all operate
/// within the same heartbeat period and share the same order-of-magnitude concurrency:
/// - `raft.rs` cmd_rx drain (client propose ingestion)
/// - `raft.rs` role_rx drain (commit index event coalescing)
/// - `DefaultCommitHandler` new_commit_rx drain (state machine apply batching)
///
/// Also used as the initial Vec capacity for propose and linearizable-read buffers
/// (construction-time hint only; the propose buffer retains capacity across flushes
/// via `mem::swap`, while the read buffer resets each flush via `mem::take`).
///
/// # Tuning Guidelines
/// - Low latency priority: lower values → smaller batches, faster flush
/// - Throughput priority: higher values → larger batches, fewer RPCs
#[derive(Debug, Serialize, Deserialize, Clone)]
pub struct BatchingConfig {
    /// Maximum items to drain per heartbeat period across all drain loops.
    ///
    /// **Default**: 100
    #[serde(default = "default_max_batch_size")]
    pub max_batch_size: usize,
}

impl Default for BatchingConfig {
    fn default() -> Self {
        Self {
            max_batch_size: default_max_batch_size(),
        }
    }
}

impl BatchingConfig {
    fn validate(&self) -> Result<()> {
        if self.max_batch_size == 0 {
            return Err(Error::Config(ConfigError::Message(
                "batching.max_batch_size must be > 0".into(),
            )));
        }
        Ok(())
    }
}

fn default_append_interval() -> u64 {
    100
}
fn default_max_batch_size() -> usize {
    100
}
fn default_entries_per_replication() -> u64 {
    100
}
#[derive(Debug, Serialize, Deserialize, Clone)]
pub struct ElectionConfig {
    #[serde(default = "default_election_timeout_min")]
    pub election_timeout_min: u64,

    #[serde(default = "default_election_timeout_max")]
    pub election_timeout_max: u64,

    #[serde(default = "default_peer_monitor_interval")]
    pub rpc_peer_connectinon_monitor_interval_in_sec: u64,

    #[serde(default = "default_client_request_id")]
    pub internal_rpc_client_request_id: u32,
}

impl Default for ElectionConfig {
    fn default() -> Self {
        Self {
            election_timeout_min: default_election_timeout_min(),
            election_timeout_max: default_election_timeout_max(),
            rpc_peer_connectinon_monitor_interval_in_sec: default_peer_monitor_interval(),
            internal_rpc_client_request_id: default_client_request_id(),
        }
    }
}
impl ElectionConfig {
    fn validate(&self) -> Result<()> {
        if self.election_timeout_min >= self.election_timeout_max {
            return Err(Error::Config(ConfigError::Message(format!(
                "election_timeout_min {}ms must be less than election_timeout_max {}ms",
                self.election_timeout_min, self.election_timeout_max
            ))));
        }

        if self.rpc_peer_connectinon_monitor_interval_in_sec == 0 {
            return Err(Error::Config(ConfigError::Message(
                "rpc_peer_connectinon_monitor_interval_in_sec cannot be 0".into(),
            )));
        }

        Ok(())
    }
}
fn default_election_timeout_min() -> u64 {
    500
}
fn default_election_timeout_max() -> u64 {
    1000
}
fn default_peer_monitor_interval() -> u64 {
    30
}
fn default_client_request_id() -> u32 {
    0
}

#[derive(Debug, Serialize, Deserialize, Clone)]
pub struct MembershipConfig {
    #[serde(default = "default_probe_service")]
    pub cluster_healthcheck_probe_service_name: String,

    #[serde(default = "default_verify_leadership_persistent_timeout")]
    pub verify_leadership_persistent_timeout: Duration,

    #[serde(default = "default_membership_maintenance_interval")]
    pub membership_maintenance_interval: Duration,

    #[serde(default)]
    pub zombie: ZombieConfig,

    /// Configuration settings for ready learners promotion
    #[serde(default)]
    pub promotion: PromotionConfig,
}
impl Default for MembershipConfig {
    fn default() -> Self {
        Self {
            cluster_healthcheck_probe_service_name: default_probe_service(),
            verify_leadership_persistent_timeout: default_verify_leadership_persistent_timeout(),
            membership_maintenance_interval: default_membership_maintenance_interval(),
            zombie: ZombieConfig::default(),
            promotion: PromotionConfig::default(),
        }
    }
}
fn default_probe_service() -> String {
    "d_engine.server.cluster.ClusterManagementService".to_string()
}

// 30 seconds
fn default_membership_maintenance_interval() -> Duration {
    Duration::from_secs(30)
}

/// Default timeout for leader to keep verifying its leadership.
///
/// In Raft, the leader may retry sending no-op entries to confirm it still holds leadership.
/// This timeout defines how long the leader will keep retrying before stepping down.
///
/// Default: 1 hour.
fn default_verify_leadership_persistent_timeout() -> Duration {
    Duration::from_secs(3600)
}

impl MembershipConfig {
    fn validate(&self) -> Result<()> {
        if self.cluster_healthcheck_probe_service_name.is_empty() {
            return Err(Error::Config(ConfigError::Message(
                "cluster_healthcheck_probe_service_name cannot be empty".into(),
            )));
        }
        Ok(())
    }
}

/// State machine behavior configuration
///
/// Controls state machine operations including lease management, compaction policies,
/// and other data lifecycle features. This configuration affects how the state machine
/// processes applied log entries and manages data.
#[derive(Serialize, Deserialize, Clone, Debug)]
#[derive(Default)]
pub struct StateMachineConfig {
    /// Lease (time-based expiration) configuration
    ///
    /// For backward compatibility, can also be configured via `ttl` in TOML files.
    #[serde(alias = "ttl")]
    pub lease: LeaseConfig,
}

impl StateMachineConfig {
    pub fn validate(&self) -> Result<()> {
        self.lease.validate()?;
        Ok(())
    }
}

/// Submit processor-specific configuration
#[derive(Debug, Serialize, Deserialize, Clone)]
pub struct SnapshotConfig {
    /// If enable the snapshot or not
    #[serde(default = "default_snapshot_enabled")]
    pub enable: bool,

    /// Maximum number of log entries to accumulate before triggering snapshot creation
    /// This helps control memory usage by enforcing periodic state compaction
    #[serde(default = "default_max_log_entries_before_snapshot")]
    pub max_log_entries_before_snapshot: u64,

    /// Minimum duration to wait between consecutive snapshot checks.
    /// Acts as a cooldown period to avoid overly frequent snapshot evaluations.
    #[serde(default = "default_snapshot_cool_down_since_last_check")]
    pub snapshot_cool_down_since_last_check: Duration,

    /// Number of historical snapshot versions to retain during cleanup
    /// Ensures we maintain a safety buffer of previous states for recovery
    #[serde(default = "default_cleanup_retain_count")]
    pub cleanup_retain_count: u64,

    /// Snapshot storage directory
    ///
    /// Default: `default_snapshots_dir()` (/tmp/snapshots)
    #[serde(default = "default_snapshots_dir")]
    pub snapshots_dir: PathBuf,

    #[serde(default = "default_snapshots_dir_prefix")]
    pub snapshots_dir_prefix: String,

    /// Size (in bytes) of individual chunks when transferring snapshots
    ///
    /// Default: `default_chunk_size()` (typically 1MB)
    #[serde(default = "default_chunk_size")]
    pub chunk_size: usize,

    /// Number of log entries to retain (0 = disable retention)
    #[serde(default = "default_retained_log_entries")]
    pub retained_log_entries: u64,

    /// Number of chunks to process before yielding the task
    #[serde(default = "default_sender_yield_every_n_chunks")]
    pub sender_yield_every_n_chunks: usize,

    /// Number of chunks to process before yielding the task
    #[serde(default = "default_receiver_yield_every_n_chunks")]
    pub receiver_yield_every_n_chunks: usize,

    #[serde(default = "default_max_bandwidth_mbps")]
    pub max_bandwidth_mbps: u32,

    #[serde(default = "default_push_queue_size")]
    pub push_queue_size: usize,

    #[serde(default = "default_cache_size")]
    pub cache_size: usize,

    #[serde(default = "default_max_retries")]
    pub max_retries: u32,

    #[serde(default = "default_transfer_timeout_in_sec")]
    pub transfer_timeout_in_sec: u64,

    #[serde(default = "default_retry_interval_in_ms")]
    pub retry_interval_in_ms: u64,

    #[serde(default = "default_snapshot_push_backoff_in_ms")]
    pub snapshot_push_backoff_in_ms: u64,

    #[serde(default = "default_snapshot_push_max_retry")]
    pub snapshot_push_max_retry: u32,

    #[serde(default = "default_push_timeout_in_ms")]
    pub push_timeout_in_ms: u64,
}
impl Default for SnapshotConfig {
    fn default() -> Self {
        Self {
            max_log_entries_before_snapshot: default_max_log_entries_before_snapshot(),
            snapshot_cool_down_since_last_check: default_snapshot_cool_down_since_last_check(),
            cleanup_retain_count: default_cleanup_retain_count(),
            snapshots_dir: default_snapshots_dir(),
            snapshots_dir_prefix: default_snapshots_dir_prefix(),
            chunk_size: default_chunk_size(),
            retained_log_entries: default_retained_log_entries(),
            sender_yield_every_n_chunks: default_sender_yield_every_n_chunks(),
            receiver_yield_every_n_chunks: default_receiver_yield_every_n_chunks(),
            max_bandwidth_mbps: default_max_bandwidth_mbps(),
            push_queue_size: default_push_queue_size(),
            cache_size: default_cache_size(),
            max_retries: default_max_retries(),
            transfer_timeout_in_sec: default_transfer_timeout_in_sec(),
            retry_interval_in_ms: default_retry_interval_in_ms(),
            snapshot_push_backoff_in_ms: default_snapshot_push_backoff_in_ms(),
            snapshot_push_max_retry: default_snapshot_push_max_retry(),
            push_timeout_in_ms: default_push_timeout_in_ms(),
            enable: default_snapshot_enabled(),
        }
    }
}
impl SnapshotConfig {
    fn validate(&self) -> Result<()> {
        if self.max_log_entries_before_snapshot == 0 {
            return Err(Error::Config(ConfigError::Message(
                "max_log_entries_before_snapshot must be greater than 0".into(),
            )));
        }

        if self.cleanup_retain_count == 0 {
            return Err(Error::Config(ConfigError::Message(
                "cleanup_retain_count must be greater than 0".into(),
            )));
        }
        // Validate storage paths
        validate_directory(&self.snapshots_dir, "snapshots_dir")?;

        // chunk_size should be > 0
        if self.chunk_size == 0 {
            return Err(Error::Config(ConfigError::Message(format!(
                "chunk_size must be at least {} bytes (got {})",
                0, self.chunk_size
            ))));
        }

        if self.retained_log_entries < 1 {
            return Err(Error::Config(ConfigError::Message(format!(
                "retained_log_entries must be >= 1, (got {})",
                self.retained_log_entries
            ))));
        }

        if self.sender_yield_every_n_chunks < 1 {
            return Err(Error::Config(ConfigError::Message(format!(
                "sender_yield_every_n_chunks must be >= 1, (got {})",
                self.sender_yield_every_n_chunks
            ))));
        }

        if self.receiver_yield_every_n_chunks < 1 {
            return Err(Error::Config(ConfigError::Message(format!(
                "receiver_yield_every_n_chunks must be >= 1, (got {})",
                self.receiver_yield_every_n_chunks
            ))));
        }

        if self.push_queue_size < 1 {
            return Err(Error::Config(ConfigError::Message(format!(
                "push_queue_size must be >= 1, (got {})",
                self.push_queue_size
            ))));
        }

        if self.snapshot_push_max_retry < 1 {
            return Err(Error::Config(ConfigError::Message(format!(
                "snapshot_push_max_retry must be >= 1, (got {})",
                self.snapshot_push_max_retry
            ))));
        }

        Ok(())
    }
}

fn default_snapshot_enabled() -> bool {
    true
}

/// Default threshold for triggering snapshot creation
fn default_max_log_entries_before_snapshot() -> u64 {
    1000
}

/// Default cooldown duration between snapshot checks.
///
/// Prevents constant evaluation of snapshot conditions in tight loops.
/// Currently set to 1 hour (3600 seconds).
fn default_snapshot_cool_down_since_last_check() -> Duration {
    Duration::from_secs(3600)
}

/// Default number of historical snapshots to retain
fn default_cleanup_retain_count() -> u64 {
    2
}
/// Default snapshots storage path
fn default_snapshots_dir() -> PathBuf {
    PathBuf::from("/tmp/snapshots")
}
/// Default snapshots directory prefix
fn default_snapshots_dir_prefix() -> String {
    "snapshot-".to_string()
}

/// 1KB chunks by default
fn default_chunk_size() -> usize {
    1024
}

#[derive(Debug, Serialize, Deserialize, Clone)]
pub struct AutoJoinConfig {
    #[serde(default = "default_rpc_enable_compression")]
    pub rpc_enable_compression: bool,
}
impl Default for AutoJoinConfig {
    fn default() -> Self {
        Self {
            rpc_enable_compression: default_rpc_enable_compression(),
        }
    }
}
fn default_rpc_enable_compression() -> bool {
    true
}

fn default_retained_log_entries() -> u64 {
    1
}

fn default_sender_yield_every_n_chunks() -> usize {
    1
}

fn default_receiver_yield_every_n_chunks() -> usize {
    1
}

fn default_max_bandwidth_mbps() -> u32 {
    1
}

fn default_push_queue_size() -> usize {
    100
}

fn default_cache_size() -> usize {
    10000
}
fn default_max_retries() -> u32 {
    1
}
fn default_transfer_timeout_in_sec() -> u64 {
    600
}
fn default_retry_interval_in_ms() -> u64 {
    10
}
fn default_snapshot_push_backoff_in_ms() -> u64 {
    100
}
fn default_snapshot_push_max_retry() -> u32 {
    3
}
fn default_push_timeout_in_ms() -> u64 {
    300_000
}

#[derive(Debug, Serialize, Deserialize, Clone)]
pub struct ZombieConfig {
    /// zombie connection failed threshold
    #[serde(default = "default_zombie_threshold")]
    pub threshold: u32,

    #[serde(default = "default_zombie_purge_interval")]
    pub purge_interval: Duration,
}

impl Default for ZombieConfig {
    fn default() -> Self {
        Self {
            threshold: default_zombie_threshold(),
            purge_interval: default_zombie_purge_interval(),
        }
    }
}

fn default_zombie_threshold() -> u32 {
    3
}
// 30 seconds
fn default_zombie_purge_interval() -> Duration {
    Duration::from_secs(30)
}

#[derive(Debug, Serialize, Deserialize, Clone)]
pub struct PromotionConfig {
    #[serde(default = "default_stale_learner_threshold")]
    pub stale_learner_threshold: Duration,
    #[serde(default = "default_stale_check_interval")]
    pub stale_check_interval: Duration,
}

impl Default for PromotionConfig {
    fn default() -> Self {
        Self {
            stale_learner_threshold: default_stale_learner_threshold(),
            stale_check_interval: default_stale_check_interval(),
        }
    }
}

// 5 minutes
fn default_stale_learner_threshold() -> Duration {
    Duration::from_secs(300)
}
// 30 seconds
fn default_stale_check_interval() -> Duration {
    Duration::from_secs(30)
}
/// Defines how Raft log entries are persisted and accessed.
///
/// All strategies use a configurable [`FlushPolicy`] to control when memory contents
/// are flushed to disk, affecting write latency and durability guarantees.
///
/// **Note:** Both strategies now fully load all log entries from disk into memory at startup.
/// The in-memory `SkipMap` serves as the primary data structure for reads in all modes.
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq)]
pub enum PersistenceStrategy {
    /// Disk-first persistence strategy.
    ///
    /// - **Write path**: On append, the log entry is first written to disk. Only after a successful
    ///   disk write is it acknowledged and stored in the in-memory `SkipMap`.
    ///
    /// - **Read path**: Reads are always served from the in-memory `SkipMap`.
    ///
    /// - **Startup behavior**: All log entries are loaded from disk into memory at startup,
    ///   ensuring consistent access speed regardless of disk state.
    ///
    /// - Suitable for systems prioritizing strong durability while still providing in-memory
    ///   performance for reads.
    DiskFirst,

    /// Memory-first persistence strategy.
    ///
    /// - **Write path**: On append, the log entry is first written to the in-memory `SkipMap` and
    ///   acknowledged immediately. Disk persistence happens asynchronously in the background,
    ///   governed by [`FlushPolicy`].
    ///
    /// - **Read path**: Reads are always served from the in-memory `SkipMap`.
    ///
    /// - **Startup behavior**: All log entries are loaded from disk into memory at startup, the
    ///   same as `DiskFirst`.
    ///
    /// - Suitable for systems that favor lower write latency and faster failover, while still
    ///   retaining a disk-backed log for crash recovery.
    MemFirst,
}

/// Controls when in-memory logs should be flushed to disk.
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq)]
pub enum FlushPolicy {
    /// Flush each log write immediately to disk.
    ///
    /// - Guarantees the highest durability.
    /// - Each append operation causes a disk write.
    Immediate,

    /// Flush entries to disk when either of two conditions is met:
    /// - The number of unflushed entries reaches the given threshold.
    /// - The elapsed time since the last flush exceeds the configured interval.
    ///
    /// - Balances performance and durability.
    /// - Recent unflushed entries may be lost in the event of a crash or power failure.
    Batch { threshold: usize, interval_ms: u64 },
}

/// Configuration parameters for log persistence behavior
#[derive(Serialize, Deserialize, Clone, Debug)]
pub struct PersistenceConfig {
    /// Strategy for persisting Raft logs
    ///
    /// This controls the trade-off between durability guarantees and performance
    /// characteristics. The choice impacts both write throughput and recovery
    /// behavior after node failures.
    #[serde(default = "default_persistence_strategy")]
    pub strategy: PersistenceStrategy,

    /// Flush policy for asynchronous strategies
    ///
    /// This controls when log entries are flushed to disk. The choice impacts
    /// write performance and durability guarantees.
    #[serde(default = "default_flush_policy")]
    pub flush_policy: FlushPolicy,

    /// Maximum number of in-memory log entries to buffer when using async strategies
    ///
    /// This acts as a safety valve to prevent memory exhaustion during periods of
    /// high write throughput or when disk persistence is slow.
    #[serde(default = "default_max_buffered_entries")]
    pub max_buffered_entries: usize,

    /// Number of flush worker threads to use for log persistence.
    ///
    /// - If set to 0, the system falls back to spawning a new task per flush (legacy behavior,
    ///   lower latency but less stable under high load).
    /// - If set to a positive number, a worker pool of that size will be created to process flush
    ///   requests (more stable and efficient under high load).
    ///
    /// This parameter allows tuning between throughput and latency depending on
    /// workload characteristics.
    #[serde(default = "default_flush_workers")]
    pub flush_workers: usize,

    /// Capacity of the internal task channel for flush workers.
    ///
    /// - Provides **backpressure** during high write throughput.
    /// - Prevents unbounded task accumulation in memory when disk I/O is slow.
    /// - Larger values improve throughput at the cost of higher memory usage, while smaller values
    ///   apply stricter flow control but may reduce parallelism.
    #[serde(default = "default_channel_capacity")]
    pub channel_capacity: usize,
}

/// Default persistence strategy (optimized for balanced workloads)
fn default_persistence_strategy() -> PersistenceStrategy {
    PersistenceStrategy::DiskFirst
}

/// Default value for flush_workers
fn default_flush_workers() -> usize {
    2
}

/// Default value for channel_capacity
fn default_channel_capacity() -> usize {
    100
}

/// Default flush policy for asynchronous strategies
///
/// This controls when log entries are flushed to disk. The choice impacts
/// write performance and durability guarantees.
fn default_flush_policy() -> FlushPolicy {
    FlushPolicy::Batch {
        threshold: 1024,
        interval_ms: 100,
    }
}

/// Default maximum buffered log entries
fn default_max_buffered_entries() -> usize {
    10_000
}

impl Default for PersistenceConfig {
    fn default() -> Self {
        Self {
            strategy: default_persistence_strategy(),
            flush_policy: default_flush_policy(),
            max_buffered_entries: default_max_buffered_entries(),
            flush_workers: default_flush_workers(),
            channel_capacity: default_channel_capacity(),
        }
    }
}

/// Backpressure configuration for client request flow control
///
/// Prevents unbounded memory growth by limiting pending client requests.
/// When limits are reached, new requests are rejected with RESOURCE_EXHAUSTED
/// error until the system processes existing requests.
///
/// # Value Semantics
/// - `0` = unlimited (no backpressure)
/// - `> 0` = maximum pending requests allowed
///
/// # Tuning Guidelines(only for reference)
/// - Low memory (< 4GB): 1000-5000
/// - Medium memory (4-16GB): 5000-20000
/// - High memory (> 16GB): 20000-50000
///
/// # Example
/// ```toml
/// [raft.backpressure]
/// max_pending_writes = 10000
/// max_pending_reads = 50000
/// ```
#[derive(Debug, Clone, Serialize, Deserialize)]
pub struct BackpressureConfig {
    /// Maximum pending write (propose) requests
    ///
    /// Limits the number of client write requests waiting in the leader's
    /// propose buffer. Write requests typically consume more resources
    /// (replication, persistence) than reads.
    ///
    /// **Default**: 10000 (0 = unlimited)
    #[serde(default = "default_max_pending_writes")]
    pub max_pending_writes: usize,

    /// Maximum pending linearizable read requests
    ///
    /// Limits the number of client linearizable read requests waiting in
    /// the leader's read buffer. Read requests can tolerate higher limits
    /// as they don't require replication.
    ///
    /// **Default**: 50000 (0 = unlimited)
    #[serde(default = "default_max_pending_reads")]
    pub max_pending_reads: usize,
}

impl Default for BackpressureConfig {
    fn default() -> Self {
        Self {
            max_pending_writes: default_max_pending_writes(),
            max_pending_reads: default_max_pending_reads(),
        }
    }
}

fn default_max_pending_writes() -> usize {
    10_000
}

fn default_max_pending_reads() -> usize {
    50_000
}

impl BackpressureConfig {
    /// Check if write request should be rejected due to backpressure
    ///
    /// Returns true if the current pending count exceeds the limit.
    /// When `max_pending_writes == 0`, always returns false (unlimited).
    pub fn should_reject_write(
        &self,
        current_pending: usize,
    ) -> bool {
        self.max_pending_writes > 0 && current_pending >= self.max_pending_writes
    }

    /// Check if read request should be rejected due to backpressure
    ///
    /// Returns true if the current pending count exceeds the limit.
    /// When `max_pending_reads == 0`, always returns false (unlimited).
    pub fn should_reject_read(
        &self,
        current_pending: usize,
    ) -> bool {
        self.max_pending_reads > 0 && current_pending >= self.max_pending_reads
    }
}

/// Policy for read operation consistency guarantees
///
/// Determines the trade-off between read consistency and performance.
/// Clients can choose the appropriate level based on their requirements.
#[derive(Clone, Debug, Default, PartialEq, Eq, Serialize, Deserialize)]
pub enum ReadConsistencyPolicy {
    /// Lease-based reads for better performance with weaker consistency
    ///
    /// The leader serves reads locally without contacting followers
    /// during the valid lease period. Assumes bounded clock drift between nodes.
    /// Provides lower latency but slightly weaker consistency guarantees
    /// compared to LinearizableRead.
    LeaseRead,

    /// Fully linearizable reads for strongest consistency
    ///
    /// The leader verifies its leadership with a quorum before serving
    /// the read, ensuring strict linearizability. This guarantees that
    /// all reads reflect the most recent committed value in the cluster.
    #[default]
    LinearizableRead,

    /// Eventually consistent reads from any node
    ///
    /// Allows reading from any node (leader, follower, or candidate) without
    /// additional consistency checks. May return stale data but provides
    /// best read performance and availability. Suitable for scenarios where
    /// eventual consistency is acceptable.
    /// **Can be served by non-leader nodes.**
    EventualConsistency,
}

/// Configuration for read operation consistency behavior
#[derive(Clone, Debug, Serialize, Deserialize)]
pub struct ReadConsistencyConfig {
    /// Default read consistency policy for the cluster
    ///
    /// This sets the cluster-wide default behavior. Individual read requests
    /// can still override this setting when needed for specific use cases.
    #[serde(default)]
    pub default_policy: ReadConsistencyPolicy,

    /// Lease duration in milliseconds for LeaseRead policy
    ///
    /// Only applicable when using the LeaseRead policy. The leader considers
    /// itself valid for this duration after successfully heartbeating to a quorum.
    #[serde(default = "default_lease_duration_ms")]
    pub lease_duration_ms: u64,

    /// Whether to allow clients to override the default policy per request
    ///
    /// When true, clients can specify consistency requirements per read request.
    /// When false, all reads use the cluster's default_policy setting.
    #[serde(default = "default_allow_client_override")]
    pub allow_client_override: bool,

    /// Timeout in milliseconds to wait for state machine to catch up with commit index
    ///
    /// Used by LinearizableRead to ensure the state machine has applied all committed
    /// entries before serving reads. Typical apply latency is <1ms on local SSD.
    /// Default: 10ms (safe buffer for single-node local deployments)
    #[serde(default = "default_state_machine_sync_timeout_ms")]
    pub state_machine_sync_timeout_ms: u64,
}

impl Default for ReadConsistencyConfig {
    fn default() -> Self {
        Self {
            default_policy: ReadConsistencyPolicy::default(),
            lease_duration_ms: default_lease_duration_ms(),
            allow_client_override: default_allow_client_override(),
            state_machine_sync_timeout_ms: default_state_machine_sync_timeout_ms(),
        }
    }
}

fn default_lease_duration_ms() -> u64 {
    // Conservative default: half of a typical heartbeat interval (~300ms)
    250
}

fn default_allow_client_override() -> bool {
    // Allow flexibility by default — clients can choose stronger consistency when needed
    true
}

fn default_state_machine_sync_timeout_ms() -> u64 {
    10 // 10ms is safe for typical <1ms apply latency on local SSD
}

impl ReadConsistencyConfig {
    fn validate(&self) -> Result<()> {
        // Validate read consistency configuration
        if self.lease_duration_ms == 0 {
            return Err(Error::Config(ConfigError::Message(
                "read_consistency.lease_duration_ms must be greater than 0".into(),
            )));
        }
        Ok(())
    }
}

impl From<d_engine_proto::client::ReadConsistencyPolicy> for ReadConsistencyPolicy {
    fn from(proto_policy: d_engine_proto::client::ReadConsistencyPolicy) -> Self {
        match proto_policy {
            d_engine_proto::client::ReadConsistencyPolicy::LeaseRead => Self::LeaseRead,
            d_engine_proto::client::ReadConsistencyPolicy::LinearizableRead => {
                Self::LinearizableRead
            }
            d_engine_proto::client::ReadConsistencyPolicy::EventualConsistency => {
                Self::EventualConsistency
            }
        }
    }
}

impl From<ReadConsistencyPolicy> for d_engine_proto::client::ReadConsistencyPolicy {
    fn from(config_policy: ReadConsistencyPolicy) -> Self {
        match config_policy {
            ReadConsistencyPolicy::LeaseRead => {
                d_engine_proto::client::ReadConsistencyPolicy::LeaseRead
            }
            ReadConsistencyPolicy::LinearizableRead => {
                d_engine_proto::client::ReadConsistencyPolicy::LinearizableRead
            }
            ReadConsistencyPolicy::EventualConsistency => {
                d_engine_proto::client::ReadConsistencyPolicy::EventualConsistency
            }
        }
    }
}

/// Configuration for controlling gRPC compression settings per service type
///
/// Provides fine-grained control over when to enable compression based on
/// the RPC service type and deployment environment. Each service can be
/// independently configured to use compression based on its data
/// characteristics and frequency.
#[derive(Debug, Clone, Serialize, Deserialize)]
pub struct RpcCompressionConfig {
    /// Controls compression for Raft replication response data
    ///
    /// Replication traffic is typically high-frequency with small payloads
    /// in LAN environments, making compression less beneficial. In WAN
    /// deployments with bandwidth constraints, enabling may help.
    ///
    /// **Default**: `false` - Optimized for LAN/same-VPC deployments
    #[serde(default = "default_replication_compression")]
    pub replication_response: bool,

    /// Controls compression for Raft election response data
    ///
    /// Election traffic is low-frequency but time-sensitive. Compression
    /// rarely benefits election traffic due to small payload size.
    ///
    /// **Default**: `true` for backward compatibility
    #[serde(default = "default_election_compression")]
    pub election_response: bool,

    /// Controls compression for snapshot transfer response data
    ///
    /// Snapshot transfers involve large data volumes where compression
    /// is typically beneficial, even in low-latency environments.
    ///
    /// **Default**: `true` - Recommended for all environments
    #[serde(default = "default_snapshot_compression")]
    pub snapshot_response: bool,

    /// Controls compression for cluster management response data
    ///
    /// Cluster operations are infrequent but may contain configuration data.
    /// Compression is generally beneficial for these operations.
    ///
    /// **Default**: `true` for backward compatibility
    #[serde(default = "default_cluster_compression")]
    pub cluster_response: bool,

    /// Controls compression for client request response data
    ///
    /// Client responses may vary in size. In LAN/VPC environments,
    /// compression CPU overhead typically outweighs network benefits.
    ///
    /// **Default**: `false` - Optimized for LAN/same-VPC deployments
    #[serde(default = "default_client_compression")]
    pub client_response: bool,
}

impl Default for RpcCompressionConfig {
    fn default() -> Self {
        Self {
            replication_response: default_replication_compression(),
            election_response: default_election_compression(),
            snapshot_response: default_snapshot_compression(),
            cluster_response: default_cluster_compression(),
            client_response: default_client_compression(),
        }
    }
}

// Default values for RPC compression settings
fn default_replication_compression() -> bool {
    // Replication traffic is high-frequency with typically small payloads
    // For LAN/VPC deployments, compression adds CPU overhead without significant benefit
    false
}

fn default_election_compression() -> bool {
    // Kept enabled for backward compatibility, though minimal benefit
    true
}

fn default_snapshot_compression() -> bool {
    // Snapshot data is large and benefits from compression in all environments
    true
}

fn default_cluster_compression() -> bool {
    // Kept enabled for backward compatibility
    true
}

fn default_client_compression() -> bool {
    // Client responses in LAN/VPC environments typically benefit from no compression
    false
}

/// Configuration for the Watch mechanism that monitors key changes
///
/// The watch system allows clients to monitor specific keys for changes with
/// minimal overhead on the write path. It uses a lock-free event queue and
/// configurable buffer sizes to balance performance and memory usage.
///
/// # Performance Characteristics
///
/// - Write path overhead: < 0.01% with 100+ watchers
/// - Event notification latency: typically < 100μs end-to-end
/// - Memory per watcher: ~2.4KB with default buffer size
///
/// # Configuration Example
///
/// ```toml
/// [raft.watch]
/// event_queue_size = 1000
/// watcher_buffer_size = 10
/// enable_metrics = false
/// ```
#[derive(Debug, Serialize, Deserialize, Clone)]
pub struct WatchConfig {
    /// Buffer size for the global event queue shared across all watchers
    ///
    /// This queue sits between the write path and the dispatcher thread.
    /// A larger queue reduces the chance of dropped events under burst load,
    /// but increases memory usage.
    ///
    /// **Performance Impact**:
    /// - Memory: ~24 bytes per slot (key + value pointers + event type)
    /// - Default 1000 slots ≈ 24KB memory
    ///
    /// **Tuning Guidelines**:
    /// - Low traffic (< 1K writes/sec): 500-1000
    /// - Medium traffic (1K-10K writes/sec): 1000-2000
    /// - High traffic (> 10K writes/sec): 2000-5000
    ///
    /// **Default**: 1000
    #[serde(default = "default_event_queue_size")]
    pub event_queue_size: usize,

    /// Buffer size for each individual watcher's channel
    ///
    /// Each registered watcher gets its own channel to receive events.
    /// Smaller buffers reduce memory usage but increase the risk of
    /// dropping events for slow consumers.
    ///
    /// **Performance Impact**:
    /// - Memory: ~240 bytes per slot per watcher
    /// - 10 slots × 100 watchers = ~240KB total
    ///
    /// **Tuning Guidelines**:
    /// - Fast consumers (< 1ms processing): 5-10
    /// - Normal consumers (1-10ms processing): 10-20
    /// - Slow consumers (> 10ms processing): 20-50
    ///
    /// **Default**: 10
    #[serde(default = "default_watcher_buffer_size")]
    pub watcher_buffer_size: usize,

    /// Enable detailed metrics and logging for watch operations
    ///
    /// When enabled, logs warnings for dropped events and tracks watch
    /// performance metrics. Adds minimal overhead (~0.001%) but useful
    /// for debugging and monitoring.
    ///
    /// **Default**: false (minimal overhead in production)
    #[serde(default = "default_enable_watch_metrics")]
    pub enable_metrics: bool,
}

impl Default for WatchConfig {
    fn default() -> Self {
        Self {
            event_queue_size: default_event_queue_size(),
            watcher_buffer_size: default_watcher_buffer_size(),
            enable_metrics: default_enable_watch_metrics(),
        }
    }
}

impl WatchConfig {
    /// Validates watch configuration parameters
    pub fn validate(&self) -> Result<()> {
        if self.event_queue_size == 0 {
            return Err(Error::Config(ConfigError::Message(
                "watch.event_queue_size must be greater than 0".into(),
            )));
        }

        if self.event_queue_size > 100_000 {
            warn!(
                "watch.event_queue_size ({}) is very large and may consume significant memory (~{}MB)",
                self.event_queue_size,
                (self.event_queue_size * 24) / 1_000_000
            );
        }

        if self.watcher_buffer_size == 0 {
            return Err(Error::Config(ConfigError::Message(
                "watch.watcher_buffer_size must be greater than 0".into(),
            )));
        }

        if self.watcher_buffer_size > 1000 {
            warn!(
                "watch.watcher_buffer_size ({}) is very large. Each watcher will consume ~{}KB memory",
                self.watcher_buffer_size,
                (self.watcher_buffer_size * 240) / 1000
            );
        }

        Ok(())
    }
}

const fn default_event_queue_size() -> usize {
    1000
}

const fn default_watcher_buffer_size() -> usize {
    10
}

const fn default_enable_watch_metrics() -> bool {
    false
}

/// Performance metrics configuration
///
/// Controls emission of observability metrics. Disabling metrics reduces overhead
/// in hot paths but decreases system visibility.
///
/// # Example
/// ```toml
/// [raft.metrics]
/// enable_backpressure = true
/// enable_batch = true
/// sample_rate = 1  # No sampling (record every event)
/// ```
#[derive(Debug, Serialize, Deserialize, Clone)]
pub struct MetricsConfig {
    /// Enable backpressure metrics (rejections, buffer utilization)
    ///
    /// Tracks client request backpressure for capacity planning.
    /// Disable for absolute maximum performance in trusted environments.
    ///
    /// **Default**: false
    #[serde(default = "default_enable_backpressure_metrics")]
    pub enable_backpressure: bool,

    /// Enable buffer length gauge (batch.buffer_length for propose and linearizable buffers)
    ///
    /// Tracks buffer utilization for capacity planning.
    /// Disable for absolute maximum performance in trusted environments.
    ///
    /// **Default**: false
    #[serde(default = "default_enable_batch_metrics")]
    pub enable_batch: bool,

    /// Sample rate for high-frequency gauge metrics
    ///
    /// - `1` = record every event (no sampling)
    /// - `10` = record 1 out of 10 events (10% sampling)
    /// - `100` = record 1 out of 100 events (1% sampling)
    ///
    /// Applies to: buffer_utilization, buffer_length gauges.
    /// Does NOT apply to: counters (rejections, drain/heartbeat triggers).
    ///
    /// **Default**: 1 (no sampling)
    #[serde(default = "default_metrics_sample_rate")]
    pub sample_rate: u32,
}

impl Default for MetricsConfig {
    fn default() -> Self {
        Self {
            enable_backpressure: default_enable_backpressure_metrics(),
            enable_batch: default_enable_batch_metrics(),
            sample_rate: default_metrics_sample_rate(),
        }
    }
}

fn default_enable_backpressure_metrics() -> bool {
    false
}

fn default_enable_batch_metrics() -> bool {
    false
}

fn default_metrics_sample_rate() -> u32 {
    1 // No sampling by default
}