raft.go 72 KB

1234567891011121314151617181920212223242526272829303132333435363738394041424344454647484950515253545556575859606162636465666768697071727374757677787980818283848586878889909192939495969798991001011021031041051061071081091101111121131141151161171181191201211221231241251261271281291301311321331341351361371381391401411421431441451461471481491501511521531541551561571581591601611621631641651661671681691701711721731741751761771781791801811821831841851861871881891901911921931941951961971981992002012022032042052062072082092102112122132142152162172182192202212222232242252262272282292302312322332342352362372382392402412422432442452462472482492502512522532542552562572582592602612622632642652662672682692702712722732742752762772782792802812822832842852862872882892902912922932942952962972982993003013023033043053063073083093103113123133143153163173183193203213223233243253263273283293303313323333343353363373383393403413423433443453463473483493503513523533543553563573583593603613623633643653663673683693703713723733743753763773783793803813823833843853863873883893903913923933943953963973983994004014024034044054064074084094104114124134144154164174184194204214224234244254264274284294304314324334344354364374384394404414424434444454464474484494504514524534544554564574584594604614624634644654664674684694704714724734744754764774784794804814824834844854864874884894904914924934944954964974984995005015025035045055065075085095105115125135145155165175185195205215225235245255265275285295305315325335345355365375385395405415425435445455465475485495505515525535545555565575585595605615625635645655665675685695705715725735745755765775785795805815825835845855865875885895905915925935945955965975985996006016026036046056066076086096106116126136146156166176186196206216226236246256266276286296306316326336346356366376386396406416426436446456466476486496506516526536546556566576586596606616626636646656666676686696706716726736746756766776786796806816826836846856866876886896906916926936946956966976986997007017027037047057067077087097107117127137147157167177187197207217227237247257267277287297307317327337347357367377387397407417427437447457467477487497507517527537547557567577587597607617627637647657667677687697707717727737747757767777787797807817827837847857867877887897907917927937947957967977987998008018028038048058068078088098108118128138148158168178188198208218228238248258268278288298308318328338348358368378388398408418428438448458468478488498508518528538548558568578588598608618628638648658668678688698708718728738748758768778788798808818828838848858868878888898908918928938948958968978988999009019029039049059069079089099109119129139149159169179189199209219229239249259269279289299309319329339349359369379389399409419429439449459469479489499509519529539549559569579589599609619629639649659669679689699709719729739749759769779789799809819829839849859869879889899909919929939949959969979989991000100110021003100410051006100710081009101010111012101310141015101610171018101910201021102210231024102510261027102810291030103110321033103410351036103710381039104010411042104310441045104610471048104910501051105210531054105510561057105810591060106110621063106410651066106710681069107010711072107310741075107610771078107910801081108210831084108510861087108810891090109110921093109410951096109710981099110011011102110311041105110611071108110911101111111211131114111511161117111811191120112111221123112411251126112711281129113011311132113311341135113611371138113911401141114211431144114511461147114811491150115111521153115411551156115711581159116011611162116311641165116611671168116911701171117211731174117511761177117811791180118111821183118411851186118711881189119011911192119311941195119611971198119912001201120212031204120512061207120812091210121112121213121412151216121712181219122012211222122312241225122612271228122912301231123212331234123512361237123812391240124112421243124412451246124712481249125012511252125312541255125612571258125912601261126212631264126512661267126812691270127112721273127412751276127712781279128012811282128312841285128612871288128912901291129212931294129512961297129812991300130113021303130413051306130713081309131013111312131313141315131613171318131913201321132213231324132513261327132813291330133113321333133413351336133713381339134013411342134313441345134613471348134913501351135213531354135513561357135813591360136113621363136413651366136713681369137013711372137313741375137613771378137913801381138213831384138513861387138813891390139113921393139413951396139713981399140014011402140314041405140614071408140914101411141214131414141514161417141814191420142114221423142414251426142714281429143014311432143314341435143614371438143914401441144214431444144514461447144814491450145114521453145414551456145714581459146014611462146314641465146614671468146914701471147214731474147514761477147814791480148114821483148414851486148714881489149014911492149314941495149614971498149915001501150215031504150515061507150815091510151115121513151415151516151715181519152015211522152315241525152615271528152915301531153215331534153515361537153815391540154115421543154415451546154715481549155015511552155315541555155615571558155915601561156215631564156515661567156815691570157115721573157415751576157715781579158015811582158315841585158615871588158915901591159215931594159515961597159815991600160116021603160416051606160716081609161016111612161316141615161616171618161916201621162216231624162516261627162816291630163116321633163416351636163716381639164016411642164316441645164616471648164916501651165216531654165516561657165816591660166116621663166416651666166716681669167016711672167316741675167616771678167916801681168216831684168516861687168816891690169116921693169416951696169716981699170017011702170317041705170617071708170917101711171217131714171517161717171817191720172117221723172417251726172717281729173017311732173317341735173617371738173917401741174217431744174517461747174817491750175117521753175417551756175717581759176017611762176317641765176617671768176917701771177217731774177517761777177817791780178117821783178417851786178717881789179017911792179317941795179617971798179918001801180218031804180518061807180818091810181118121813181418151816181718181819182018211822182318241825182618271828182918301831183218331834183518361837183818391840184118421843184418451846184718481849185018511852185318541855185618571858185918601861186218631864186518661867186818691870187118721873187418751876187718781879188018811882188318841885188618871888188918901891189218931894189518961897189818991900190119021903190419051906190719081909191019111912191319141915191619171918191919201921192219231924192519261927192819291930193119321933193419351936193719381939194019411942194319441945194619471948194919501951195219531954195519561957195819591960196119621963196419651966196719681969197019711972197319741975197619771978197919801981198219831984198519861987198819891990199119921993199419951996199719981999200020012002200320042005200620072008200920102011201220132014201520162017201820192020202120222023202420252026202720282029203020312032203320342035203620372038203920402041204220432044204520462047204820492050205120522053205420552056205720582059206020612062206320642065206620672068206920702071207220732074207520762077207820792080208120822083208420852086208720882089209020912092209320942095209620972098209921002101210221032104210521062107210821092110211121122113211421152116211721182119212021212122212321242125212621272128212921302131213221332134213521362137213821392140214121422143214421452146214721482149215021512152215321542155215621572158215921602161216221632164216521662167216821692170217121722173217421752176217721782179218021812182218321842185218621872188218921902191219221932194219521962197219821992200220122022203220422052206220722082209221022112212221322142215221622172218221922202221222222232224222522262227222822292230223122322233223422352236223722382239224022412242224322442245224622472248224922502251225222532254225522562257225822592260226122622263226422652266226722682269227022712272227322742275227622772278227922802281228222832284228522862287228822892290229122922293229422952296229722982299230023012302230323042305230623072308230923102311231223132314231523162317231823192320232123222323232423252326232723282329233023312332233323342335233623372338233923402341234223432344234523462347234823492350235123522353235423552356235723582359236023612362236323642365236623672368236923702371237223732374237523762377237823792380238123822383238423852386238723882389239023912392239323942395239623972398239924002401240224032404240524062407240824092410241124122413241424152416241724182419242024212422242324242425242624272428242924302431243224332434243524362437243824392440244124422443244424452446244724482449245024512452245324542455245624572458245924602461246224632464246524662467246824692470247124722473247424752476247724782479248024812482248324842485248624872488248924902491249224932494249524962497249824992500250125022503250425052506250725082509251025112512251325142515251625172518251925202521252225232524252525262527252825292530253125322533253425352536253725382539254025412542254325442545254625472548254925502551255225532554255525562557255825592560256125622563256425652566256725682569257025712572257325742575257625772578257925802581258225832584258525862587258825892590259125922593259425952596259725982599260026012602260326042605260626072608260926102611261226132614261526162617261826192620262126222623262426252626262726282629263026312632263326342635263626372638263926402641264226432644264526462647264826492650265126522653265426552656265726582659266026612662266326642665266626672668266926702671267226732674267526762677267826792680268126822683268426852686268726882689269026912692269326942695269626972698269927002701270227032704270527062707270827092710271127122713271427152716271727182719272027212722272327242725272627272728272927302731273227332734273527362737273827392740274127422743274427452746274727482749275027512752275327542755275627572758275927602761276227632764276527662767276827692770277127722773277427752776277727782779278027812782278327842785278627872788278927902791279227932794279527962797279827992800280128022803280428052806280728082809281028112812281328142815281628172818281928202821282228232824282528262827282828292830283128322833
  1. package raft
  2. import (
  3. "context"
  4. "fmt"
  5. "math/rand"
  6. "os"
  7. "strings"
  8. "sync"
  9. "sync/atomic"
  10. "time"
  11. )
  12. // Raft represents a Raft consensus node
  13. type Raft struct {
  14. mu sync.RWMutex
  15. // Node identity
  16. nodeID string
  17. peers []string
  18. // Cluster membership - maps nodeID to address
  19. clusterNodes map[string]string
  20. // Current state
  21. state NodeState
  22. currentTerm uint64
  23. votedFor string
  24. leaderID string
  25. // Log management
  26. log *LogManager
  27. storage Storage
  28. commitIndex uint64
  29. lastApplied uint64
  30. // Leader state
  31. nextIndex map[string]uint64
  32. matchIndex map[string]uint64
  33. // Configuration
  34. config *Config
  35. // Communication
  36. transport Transport
  37. // Channels
  38. applyCh chan ApplyMsg
  39. stopCh chan struct{}
  40. commitCh chan struct{}
  41. // Election timer
  42. electionTimer *time.Timer
  43. heartbeatTimer *time.Timer
  44. // Statistics (deprecated, use metrics instead)
  45. stats Stats
  46. // Metrics for monitoring
  47. metrics Metrics
  48. // Logger
  49. logger Logger
  50. // Running flag
  51. running int32
  52. // Replication trigger channel - used to batch replication requests
  53. replicationCh chan struct{}
  54. // Pending config change - only one config change can be pending at a time
  55. pendingConfigChange bool
  56. // Old cluster nodes - used for majority calculation during config change
  57. // This ensures we use the old cluster size until the config change is committed
  58. oldClusterNodes map[string]string
  59. // Index of the pending config change entry
  60. configChangeIndex uint64
  61. // Joining cluster state - when a standalone node is being added to a cluster
  62. // This prevents the node from starting elections while syncing
  63. joiningCluster bool
  64. joiningClusterTime time.Time
  65. // Compaction state - prevents concurrent compaction
  66. compacting int32
  67. // Dynamic compaction threshold - updated after each compaction
  68. // Next compaction triggers when log size >= this value
  69. // Formula: lastCompactionSize * 1.5 (or initial SnapshotThreshold)
  70. nextCompactionThreshold uint64
  71. // WaitGroup to track goroutines for clean shutdown
  72. wg sync.WaitGroup
  73. // Leadership transfer state
  74. transferring bool
  75. transferTarget string
  76. transferDeadline time.Time
  77. // Last heartbeat received (for health check)
  78. lastHeartbeat time.Time
  79. // Start time (for uptime calculation)
  80. startTime time.Time
  81. // ReadIndex waiting queue
  82. readIndexCh chan *readIndexRequest
  83. // Snapshot receiving state (for chunked transfer)
  84. pendingSnapshot *pendingSnapshotState
  85. // Last contact time for each peer (for leader check)
  86. lastContact map[string]time.Time
  87. }
  88. // readIndexRequest represents a pending read index request
  89. type readIndexRequest struct {
  90. readIndex uint64
  91. done chan error
  92. }
  93. // pendingSnapshotState tracks chunked snapshot reception
  94. type pendingSnapshotState struct {
  95. lastIncludedIndex uint64
  96. lastIncludedTerm uint64
  97. data []byte
  98. receivedBytes uint64
  99. }
  100. // Stats holds runtime statistics
  101. type Stats struct {
  102. Term uint64
  103. State string
  104. FirstLogIndex uint64 // Added for monitoring compaction
  105. LastLogIndex uint64
  106. LastLogTerm uint64
  107. CommitIndex uint64
  108. LastApplied uint64
  109. LeaderID string
  110. VotesReceived int
  111. AppendsSent int64
  112. AppendsReceived int64
  113. ClusterSize int // Number of nodes in cluster
  114. ClusterNodes map[string]string // NodeID -> Address mapping
  115. }
  116. // ConsoleLogger implements Logger with console output
  117. type ConsoleLogger struct {
  118. Prefix string
  119. Level int // 0=debug, 1=info, 2=warn, 3=error
  120. mu sync.Mutex
  121. }
  122. func NewConsoleLogger(prefix string, level int) *ConsoleLogger {
  123. return &ConsoleLogger{Prefix: prefix, Level: level}
  124. }
  125. func (l *ConsoleLogger) log(level int, levelStr, format string, args ...interface{}) {
  126. if level < l.Level {
  127. return
  128. }
  129. l.mu.Lock()
  130. defer l.mu.Unlock()
  131. msg := fmt.Sprintf(format, args...)
  132. fmt.Printf("[%s] %s [%s] %s\n", time.Now().Format("15:04:05.000"), l.Prefix, levelStr, msg)
  133. }
  134. func (l *ConsoleLogger) Debug(format string, args ...interface{}) {
  135. l.log(0, "DEBUG", format, args...)
  136. }
  137. func (l *ConsoleLogger) Info(format string, args ...interface{}) {
  138. l.log(1, "INFO", format, args...)
  139. }
  140. func (l *ConsoleLogger) Warn(format string, args ...interface{}) {
  141. l.log(2, "WARN", format, args...)
  142. }
  143. func (l *ConsoleLogger) Error(format string, args ...interface{}) {
  144. l.log(3, "ERROR", format, args...)
  145. }
  146. // NewRaft creates a new Raft node
  147. func NewRaft(config *Config, transport Transport, applyCh chan ApplyMsg) (*Raft, error) {
  148. if config.Logger == nil {
  149. config.Logger = NewConsoleLogger(config.NodeID, 1)
  150. }
  151. // Create storage
  152. storage, err := NewHybridStorage(config.DataDir, config.MemoryLogCapacity, config.Logger)
  153. if err != nil {
  154. return nil, fmt.Errorf("failed to create storage: %w", err)
  155. }
  156. // Create log manager
  157. logMgr := NewLogManager(storage, config.Logger)
  158. // Load persistent state
  159. state, err := storage.GetState()
  160. if err != nil {
  161. return nil, fmt.Errorf("failed to load state: %w", err)
  162. }
  163. // Load or initialize cluster configuration
  164. clusterNodes := make(map[string]string)
  165. // Try to load saved cluster config first
  166. savedConfig, err := storage.GetClusterConfig()
  167. if err != nil {
  168. return nil, fmt.Errorf("failed to load cluster config: %w", err)
  169. }
  170. if savedConfig != nil && len(savedConfig.Nodes) > 0 {
  171. // Use saved config
  172. clusterNodes = savedConfig.Nodes
  173. config.Logger.Info("Loaded cluster config with %d nodes", len(clusterNodes))
  174. } else {
  175. // Initialize from config
  176. if config.ClusterNodes != nil && len(config.ClusterNodes) > 0 {
  177. for k, v := range config.ClusterNodes {
  178. clusterNodes[k] = v
  179. }
  180. } else {
  181. // Build from Peers + self (backward compatibility)
  182. clusterNodes[config.NodeID] = config.ListenAddr
  183. if config.PeerMap != nil {
  184. for k, v := range config.PeerMap {
  185. clusterNodes[k] = v
  186. }
  187. }
  188. }
  189. // Save initial config
  190. if len(clusterNodes) > 0 {
  191. if err := storage.SaveClusterConfig(&ClusterConfig{Nodes: clusterNodes}); err != nil {
  192. config.Logger.Warn("Failed to save initial cluster config: %v", err)
  193. }
  194. }
  195. }
  196. // Build peers list from cluster nodes (excluding self)
  197. var peers []string
  198. for nodeID, addr := range clusterNodes {
  199. if nodeID != config.NodeID {
  200. peers = append(peers, addr)
  201. }
  202. }
  203. r := &Raft{
  204. nodeID: config.NodeID,
  205. peers: peers,
  206. clusterNodes: clusterNodes,
  207. state: Follower,
  208. currentTerm: state.CurrentTerm,
  209. votedFor: state.VotedFor,
  210. log: logMgr,
  211. storage: storage,
  212. config: config,
  213. transport: transport,
  214. applyCh: applyCh,
  215. stopCh: make(chan struct{}),
  216. commitCh: make(chan struct{}, 100), // Increased buffer for event-driven apply
  217. replicationCh: make(chan struct{}, 1),
  218. nextIndex: make(map[string]uint64),
  219. matchIndex: make(map[string]uint64),
  220. logger: config.Logger,
  221. readIndexCh: make(chan *readIndexRequest, 100),
  222. lastHeartbeat: time.Now(),
  223. lastContact: make(map[string]time.Time),
  224. }
  225. // Initialize metrics
  226. r.metrics.Term = state.CurrentTerm
  227. // Set RPC handler
  228. transport.SetRPCHandler(r)
  229. return r, nil
  230. }
  231. // Start starts the Raft node
  232. func (r *Raft) Start() error {
  233. if !atomic.CompareAndSwapInt32(&r.running, 0, 1) {
  234. return fmt.Errorf("already running")
  235. }
  236. // Record start time
  237. r.startTime = time.Now()
  238. // Start transport
  239. if err := r.transport.Start(); err != nil {
  240. return fmt.Errorf("failed to start transport: %w", err)
  241. }
  242. // Restore FSM from snapshot if exists
  243. // This must happen before starting apply loop to ensure FSM state is restored
  244. if err := r.restoreFromSnapshot(); err != nil {
  245. // Suppress warning if it's just a missing file (first start)
  246. if !os.IsNotExist(err) && !strings.Contains(err.Error(), "no such file") {
  247. r.logger.Warn("Failed to restore from snapshot: %v", err)
  248. } else {
  249. r.logger.Info("No snapshot found, starting with empty state")
  250. }
  251. // Continue anyway - the node can still function, just without historical state
  252. }
  253. // Start election timer
  254. r.resetElectionTimer()
  255. // Start background goroutines with WaitGroup tracking
  256. r.wg.Add(4)
  257. go func() {
  258. defer r.wg.Done()
  259. r.applyLoop()
  260. }()
  261. go func() {
  262. defer r.wg.Done()
  263. r.replicationLoop()
  264. }()
  265. go func() {
  266. defer r.wg.Done()
  267. r.mainLoop()
  268. }()
  269. go func() {
  270. defer r.wg.Done()
  271. r.readIndexLoop()
  272. }()
  273. r.logger.Info("Raft node started")
  274. return nil
  275. }
  276. // Stop stops the Raft node
  277. func (r *Raft) Stop() error {
  278. if !atomic.CompareAndSwapInt32(&r.running, 1, 0) {
  279. return fmt.Errorf("not running")
  280. }
  281. // Signal all goroutines to stop
  282. close(r.stopCh)
  283. // Stop timers first to prevent new operations
  284. r.mu.Lock()
  285. if r.electionTimer != nil {
  286. r.electionTimer.Stop()
  287. }
  288. if r.heartbeatTimer != nil {
  289. r.heartbeatTimer.Stop()
  290. }
  291. r.mu.Unlock()
  292. // Wait for all goroutines to finish with timeout
  293. done := make(chan struct{})
  294. go func() {
  295. r.wg.Wait()
  296. close(done)
  297. }()
  298. select {
  299. case <-done:
  300. // All goroutines exited cleanly
  301. case <-time.After(3 * time.Second):
  302. r.logger.Warn("Timeout waiting for goroutines to stop")
  303. }
  304. // Stop transport (has its own timeout)
  305. if err := r.transport.Stop(); err != nil {
  306. r.logger.Error("Failed to stop transport: %v", err)
  307. }
  308. if err := r.storage.Close(); err != nil {
  309. r.logger.Error("Failed to close storage: %v", err)
  310. }
  311. r.logger.Info("Raft node stopped")
  312. return nil
  313. }
  314. // mainLoop is the main event loop
  315. func (r *Raft) mainLoop() {
  316. for {
  317. select {
  318. case <-r.stopCh:
  319. return
  320. default:
  321. }
  322. r.mu.RLock()
  323. state := r.state
  324. r.mu.RUnlock()
  325. switch state {
  326. case Follower:
  327. r.runFollower()
  328. case Candidate:
  329. r.runCandidate()
  330. case Leader:
  331. r.runLeader()
  332. }
  333. }
  334. }
  335. // runFollower handles follower behavior
  336. func (r *Raft) runFollower() {
  337. r.logger.Debug("Running as follower in term %d", r.currentTerm)
  338. for {
  339. select {
  340. case <-r.stopCh:
  341. return
  342. case <-r.electionTimer.C:
  343. r.mu.Lock()
  344. if r.state == Follower {
  345. // If we're joining a cluster, don't start elections
  346. // Give time for the leader to sync us up
  347. if r.joiningCluster {
  348. // Allow joining for up to 30 seconds
  349. if time.Since(r.joiningClusterTime) < 30*time.Second {
  350. r.logger.Debug("Suppressing election during cluster join")
  351. r.resetElectionTimer()
  352. r.mu.Unlock()
  353. continue
  354. }
  355. // Timeout - clear joining state
  356. r.joiningCluster = false
  357. r.logger.Warn("Cluster join timeout, resuming normal election behavior")
  358. }
  359. r.logger.Debug("Election timeout, becoming candidate")
  360. r.state = Candidate
  361. r.leaderID = "" // Clear leaderID so we can vote for self in Pre-Vote
  362. }
  363. r.mu.Unlock()
  364. return
  365. }
  366. }
  367. }
  368. // runCandidate handles candidate behavior (leader election)
  369. // Implements Pre-Vote mechanism to prevent term explosion
  370. func (r *Raft) runCandidate() {
  371. // Phase 1: Pre-Vote (don't increment term yet)
  372. if !r.runPreVote() {
  373. // Pre-vote failed, wait for timeout before retrying
  374. r.mu.Lock()
  375. r.resetElectionTimer()
  376. r.mu.Unlock()
  377. select {
  378. case <-r.stopCh:
  379. return
  380. case <-r.electionTimer.C:
  381. // Timer expired, will retry pre-vote
  382. }
  383. return
  384. }
  385. // Phase 2: Actual election (pre-vote succeeded, now increment term)
  386. r.mu.Lock()
  387. r.currentTerm++
  388. r.votedFor = r.nodeID
  389. r.leaderID = ""
  390. currentTerm := r.currentTerm
  391. // Update metrics
  392. atomic.StoreUint64(&r.metrics.Term, currentTerm)
  393. if err := r.persistState(); err != nil {
  394. r.logger.Error("Failed to persist state during election: %v", err)
  395. r.mu.Unlock()
  396. return // Cannot proceed without persisting state
  397. }
  398. atomic.AddUint64(&r.metrics.ElectionsStarted, 1)
  399. r.resetElectionTimer()
  400. r.mu.Unlock()
  401. r.logger.Debug("Starting election for term %d", currentTerm)
  402. // Get current peers list
  403. r.mu.RLock()
  404. currentPeers := make([]string, len(r.peers))
  405. copy(currentPeers, r.peers)
  406. clusterSize := len(r.clusterNodes)
  407. if clusterSize == 0 {
  408. clusterSize = len(r.peers) + 1
  409. }
  410. r.mu.RUnlock()
  411. // Request actual votes from all peers
  412. votes := 1 // Vote for self
  413. voteCh := make(chan bool, len(currentPeers))
  414. lastLogIndex, lastLogTerm := r.log.LastIndexAndTerm()
  415. for _, peer := range currentPeers {
  416. go func(peer string) {
  417. args := &RequestVoteArgs{
  418. Term: currentTerm,
  419. CandidateID: r.nodeID,
  420. LastLogIndex: lastLogIndex,
  421. LastLogTerm: lastLogTerm,
  422. PreVote: false,
  423. }
  424. ctx, cancel := context.WithTimeout(context.Background(), 500*time.Millisecond)
  425. defer cancel()
  426. reply, err := r.transport.RequestVote(ctx, peer, args)
  427. if err != nil {
  428. r.logger.Debug("RequestVote to %s failed: %v", peer, err)
  429. voteCh <- false
  430. return
  431. }
  432. r.mu.Lock()
  433. defer r.mu.Unlock()
  434. if reply.Term > r.currentTerm {
  435. r.becomeFollower(reply.Term)
  436. voteCh <- false
  437. return
  438. }
  439. voteCh <- reply.VoteGranted
  440. }(peer)
  441. }
  442. // Wait for votes - majority is (clusterSize/2) + 1
  443. needed := clusterSize/2 + 1
  444. // If we already have enough votes (single-node cluster), become leader immediately
  445. if votes >= needed {
  446. r.mu.Lock()
  447. if r.state == Candidate && r.currentTerm == currentTerm {
  448. r.becomeLeader()
  449. }
  450. r.mu.Unlock()
  451. return
  452. }
  453. for i := 0; i < len(currentPeers); i++ {
  454. select {
  455. case <-r.stopCh:
  456. return
  457. case <-r.electionTimer.C:
  458. r.logger.Debug("Election timeout, will start new election")
  459. return
  460. case granted := <-voteCh:
  461. if granted {
  462. votes++
  463. if votes >= needed {
  464. r.mu.Lock()
  465. if r.state == Candidate && r.currentTerm == currentTerm {
  466. r.becomeLeader()
  467. }
  468. r.mu.Unlock()
  469. return
  470. }
  471. }
  472. }
  473. // Check if we're still a candidate
  474. r.mu.RLock()
  475. if r.state != Candidate {
  476. r.mu.RUnlock()
  477. return
  478. }
  479. r.mu.RUnlock()
  480. }
  481. // Election failed, wait for timeout
  482. r.mu.RLock()
  483. stillCandidate := r.state == Candidate
  484. r.mu.RUnlock()
  485. if stillCandidate {
  486. r.logger.Debug("Election failed, waiting for timeout (got %d/%d votes)", votes, needed)
  487. select {
  488. case <-r.stopCh:
  489. return
  490. case <-r.electionTimer.C:
  491. // Timer expired, will start new election
  492. }
  493. }
  494. }
  495. // runPreVote sends pre-vote requests to check if we can win an election
  496. // Returns true if we got majority pre-votes, false otherwise
  497. func (r *Raft) runPreVote() bool {
  498. r.mu.RLock()
  499. currentTerm := r.currentTerm
  500. currentPeers := make([]string, len(r.peers))
  501. copy(currentPeers, r.peers)
  502. clusterSize := len(r.clusterNodes)
  503. if clusterSize == 0 {
  504. clusterSize = len(r.peers) + 1
  505. }
  506. leaderID := r.leaderID
  507. r.mu.RUnlock()
  508. // Pre-vote uses term+1 but doesn't actually increment it
  509. preVoteTerm := currentTerm + 1
  510. lastLogIndex, lastLogTerm := r.log.LastIndexAndTerm()
  511. r.logger.Debug("Starting pre-vote for term %d", preVoteTerm)
  512. // Per Raft Pre-Vote optimization (§9.6): if we have a known leader,
  513. // don't vote for self. This prevents standalone nodes from constantly
  514. // electing themselves when they should be joining an existing cluster.
  515. preVotes := 0
  516. if leaderID == "" {
  517. preVotes = 1 // Vote for self only if we don't know of a leader
  518. } else {
  519. r.logger.Debug("Pre-vote: not self-voting because we have leader %s", leaderID)
  520. }
  521. preVoteCh := make(chan bool, len(currentPeers))
  522. for _, peer := range currentPeers {
  523. go func(peer string) {
  524. args := &RequestVoteArgs{
  525. Term: preVoteTerm,
  526. CandidateID: r.nodeID,
  527. LastLogIndex: lastLogIndex,
  528. LastLogTerm: lastLogTerm,
  529. PreVote: true,
  530. }
  531. ctx, cancel := context.WithTimeout(context.Background(), 300*time.Millisecond)
  532. defer cancel()
  533. reply, err := r.transport.RequestVote(ctx, peer, args)
  534. if err != nil {
  535. r.logger.Debug("PreVote to %s failed: %v", peer, err)
  536. preVoteCh <- false
  537. return
  538. }
  539. // For pre-vote, we don't step down even if reply.Term > currentTerm
  540. // because the other node might also be doing pre-vote
  541. preVoteCh <- reply.VoteGranted
  542. }(peer)
  543. }
  544. // Wait for pre-votes with a shorter timeout - majority is (clusterSize/2) + 1
  545. needed := clusterSize/2 + 1
  546. // If we already have enough votes (single-node cluster with no known leader), succeed immediately
  547. if preVotes >= needed {
  548. r.logger.Debug("Pre-vote succeeded immediately (got %d/%d pre-votes)", preVotes, needed)
  549. return true
  550. }
  551. timeout := time.After(500 * time.Millisecond)
  552. for i := 0; i < len(currentPeers); i++ {
  553. select {
  554. case <-r.stopCh:
  555. return false
  556. case <-timeout:
  557. r.logger.Debug("Pre-vote timeout (got %d/%d pre-votes)", preVotes, needed)
  558. return false
  559. case granted := <-preVoteCh:
  560. if granted {
  561. preVotes++
  562. if preVotes >= needed {
  563. r.logger.Debug("Pre-vote succeeded (got %d/%d pre-votes)", preVotes, needed)
  564. return true
  565. }
  566. }
  567. }
  568. // Check if we're still a candidate
  569. r.mu.RLock()
  570. if r.state != Candidate {
  571. r.mu.RUnlock()
  572. return false
  573. }
  574. r.mu.RUnlock()
  575. }
  576. r.logger.Debug("Pre-vote failed (got %d/%d pre-votes)", preVotes, needed)
  577. return false
  578. }
  579. // runLeader handles leader behavior
  580. func (r *Raft) runLeader() {
  581. r.logger.Debug("Running as leader in term %d", r.currentTerm)
  582. // Send initial heartbeat
  583. r.sendHeartbeats()
  584. // Start heartbeat timer
  585. r.mu.Lock()
  586. r.heartbeatTimer = time.NewTimer(r.config.HeartbeatInterval)
  587. r.mu.Unlock()
  588. for {
  589. select {
  590. case <-r.stopCh:
  591. return
  592. case <-r.heartbeatTimer.C:
  593. r.mu.RLock()
  594. if r.state != Leader {
  595. r.mu.RUnlock()
  596. return
  597. }
  598. r.mu.RUnlock()
  599. r.sendHeartbeats()
  600. r.heartbeatTimer.Reset(r.config.HeartbeatInterval)
  601. case <-r.commitCh:
  602. r.updateCommitIndex()
  603. }
  604. }
  605. }
  606. // becomeFollower transitions to follower state
  607. func (r *Raft) becomeFollower(term uint64) {
  608. oldState := r.state
  609. oldTerm := r.currentTerm
  610. r.state = Follower
  611. r.currentTerm = term
  612. // Update metrics
  613. atomic.StoreUint64(&r.metrics.Term, term)
  614. r.votedFor = ""
  615. r.leaderID = ""
  616. // Must persist before responding - use mustPersistState for critical transitions
  617. r.mustPersistState()
  618. r.resetElectionTimer()
  619. // Clear leadership transfer state
  620. r.transferring = false
  621. r.transferTarget = ""
  622. // Only log significant state changes
  623. if oldState == Leader && term > oldTerm {
  624. // Stepping down from leader is notable
  625. r.logger.Debug("Stepped down from leader in term %d", term)
  626. }
  627. }
  628. // becomeLeader transitions to leader state
  629. func (r *Raft) becomeLeader() {
  630. r.state = Leader
  631. r.leaderID = r.nodeID
  632. // Update metrics
  633. atomic.AddUint64(&r.metrics.ElectionsWon, 1)
  634. // Initialize leader state for all peers
  635. lastIndex := r.log.LastIndex()
  636. for nodeID, addr := range r.clusterNodes {
  637. if nodeID != r.nodeID {
  638. r.nextIndex[addr] = lastIndex + 1
  639. r.matchIndex[addr] = 0
  640. }
  641. }
  642. // Also handle legacy peers
  643. for _, peer := range r.peers {
  644. if _, exists := r.nextIndex[peer]; !exists {
  645. r.nextIndex[peer] = lastIndex + 1
  646. r.matchIndex[peer] = 0
  647. }
  648. }
  649. r.logger.Info("Became leader in term %d with %d peers", r.currentTerm, len(r.clusterNodes)-1)
  650. // Append a no-op entry to commit entries from previous terms
  651. // This is a standard Raft optimization - the leader appends a no-op
  652. // entry in its current term to quickly commit all pending entries
  653. noopEntry := LogEntry{
  654. Index: r.log.LastIndex() + 1,
  655. Term: r.currentTerm,
  656. Type: EntryNoop,
  657. Command: nil,
  658. }
  659. if err := r.log.Append(noopEntry); err != nil {
  660. r.logger.Error("Failed to append no-op entry: %v", err)
  661. } else {
  662. r.logger.Debug("Appended no-op entry at index %d for term %d", noopEntry.Index, noopEntry.Term)
  663. }
  664. }
  665. // sendHeartbeats sends AppendEntries RPCs to all peers
  666. func (r *Raft) sendHeartbeats() {
  667. r.mu.RLock()
  668. if r.state != Leader {
  669. r.mu.RUnlock()
  670. return
  671. }
  672. currentTerm := r.currentTerm
  673. leaderCommit := r.commitIndex
  674. // Get all peer addresses
  675. peerAddrs := make([]string, 0, len(r.clusterNodes))
  676. for nodeID, addr := range r.clusterNodes {
  677. if nodeID != r.nodeID {
  678. peerAddrs = append(peerAddrs, addr)
  679. }
  680. }
  681. // Also include legacy peers not in clusterNodes
  682. for _, peer := range r.peers {
  683. found := false
  684. for _, addr := range peerAddrs {
  685. if addr == peer {
  686. found = true
  687. break
  688. }
  689. }
  690. if !found {
  691. peerAddrs = append(peerAddrs, peer)
  692. }
  693. }
  694. r.mu.RUnlock()
  695. for _, peer := range peerAddrs {
  696. go r.replicateToPeer(peer, currentTerm, leaderCommit)
  697. }
  698. }
  699. // replicateToPeer sends AppendEntries to a specific peer
  700. func (r *Raft) replicateToPeer(peer string, term, leaderCommit uint64) {
  701. r.mu.RLock()
  702. if r.state != Leader || r.currentTerm != term {
  703. r.mu.RUnlock()
  704. return
  705. }
  706. nextIndex := r.nextIndex[peer]
  707. r.mu.RUnlock()
  708. // Get entries to send
  709. entries, prevLogIndex, prevLogTerm, err := r.log.GetEntriesForFollower(nextIndex, r.config.MaxLogEntriesPerRequest)
  710. if err == ErrCompacted {
  711. // Need to send snapshot
  712. r.sendSnapshot(peer)
  713. return
  714. }
  715. if err != nil {
  716. r.logger.Debug("Failed to get entries for %s: %v", peer, err)
  717. return
  718. }
  719. args := &AppendEntriesArgs{
  720. Term: term,
  721. LeaderID: r.nodeID,
  722. PrevLogIndex: prevLogIndex,
  723. PrevLogTerm: prevLogTerm,
  724. Entries: entries,
  725. LeaderCommit: leaderCommit,
  726. }
  727. ctx, cancel := context.WithTimeout(context.Background(), 500*time.Millisecond)
  728. defer cancel()
  729. reply, err := r.transport.AppendEntries(ctx, peer, args)
  730. if err != nil {
  731. r.logger.Debug("AppendEntries to %s failed: %v", peer, err)
  732. return
  733. }
  734. atomic.AddInt64(&r.stats.AppendsSent, 1)
  735. r.mu.Lock()
  736. defer r.mu.Unlock()
  737. if reply.Term > r.currentTerm {
  738. r.becomeFollower(reply.Term)
  739. return
  740. }
  741. if r.state != Leader || r.currentTerm != term {
  742. return
  743. }
  744. if reply.Success {
  745. // Update contact time
  746. r.lastContact[peer] = time.Now()
  747. // Update nextIndex and matchIndex
  748. if len(entries) > 0 {
  749. newMatchIndex := entries[len(entries)-1].Index
  750. if newMatchIndex > r.matchIndex[peer] {
  751. r.matchIndex[peer] = newMatchIndex
  752. r.nextIndex[peer] = newMatchIndex + 1
  753. // Try to update commit index
  754. select {
  755. case r.commitCh <- struct{}{}:
  756. default:
  757. }
  758. }
  759. }
  760. } else {
  761. // Even if failed (log conflict), we contacted the peer
  762. r.lastContact[peer] = time.Now()
  763. // Decrement nextIndex and retry
  764. if reply.ConflictTerm > 0 {
  765. // Find the last entry of ConflictTerm in our log
  766. found := false
  767. for idx := r.log.LastIndex(); idx >= r.log.FirstIndex(); idx-- {
  768. t, err := r.log.GetTerm(idx)
  769. if err != nil {
  770. break
  771. }
  772. if t == reply.ConflictTerm {
  773. r.nextIndex[peer] = idx + 1
  774. found = true
  775. break
  776. }
  777. if t < reply.ConflictTerm {
  778. break
  779. }
  780. }
  781. if !found {
  782. r.nextIndex[peer] = reply.ConflictIndex
  783. }
  784. } else if reply.ConflictIndex > 0 {
  785. r.nextIndex[peer] = reply.ConflictIndex
  786. } else {
  787. r.nextIndex[peer] = max(1, r.nextIndex[peer]-1)
  788. }
  789. }
  790. }
  791. // sendSnapshot sends a snapshot to a peer with chunked transfer support
  792. func (r *Raft) sendSnapshot(peer string) {
  793. r.mu.RLock()
  794. if r.state != Leader {
  795. r.mu.RUnlock()
  796. return
  797. }
  798. term := r.currentTerm
  799. chunkSize := r.config.SnapshotChunkSize
  800. if chunkSize <= 0 {
  801. chunkSize = 1024 * 1024 // Default 1MB
  802. }
  803. r.mu.RUnlock()
  804. data, lastIndex, lastTerm, err := r.storage.GetSnapshot()
  805. if err != nil {
  806. r.logger.Error("Failed to get snapshot: %v", err)
  807. return
  808. }
  809. atomic.AddUint64(&r.metrics.SnapshotsSent, 1)
  810. r.logger.Info("Sending snapshot to %s: %d bytes, lastIndex=%d, lastTerm=%d",
  811. peer, len(data), lastIndex, lastTerm)
  812. // Send snapshot in chunks
  813. totalSize := len(data)
  814. offset := 0
  815. for offset < totalSize {
  816. // Check if we're still leader
  817. r.mu.RLock()
  818. if r.state != Leader || r.currentTerm != term {
  819. r.mu.RUnlock()
  820. r.logger.Debug("Aborting snapshot send: no longer leader")
  821. return
  822. }
  823. r.mu.RUnlock()
  824. // Calculate chunk size
  825. end := offset + chunkSize
  826. if end > totalSize {
  827. end = totalSize
  828. }
  829. chunk := data[offset:end]
  830. done := end >= totalSize
  831. args := &InstallSnapshotArgs{
  832. Term: term,
  833. LeaderID: r.nodeID,
  834. LastIncludedIndex: lastIndex,
  835. LastIncludedTerm: lastTerm,
  836. Offset: uint64(offset),
  837. Data: chunk,
  838. Done: done,
  839. }
  840. ctx, cancel := context.WithTimeout(context.Background(), r.config.SnapshotRPCTimeout)
  841. reply, err := r.transport.InstallSnapshot(ctx, peer, args)
  842. cancel()
  843. if err != nil {
  844. r.logger.Error("InstallSnapshot chunk to %s failed at offset %d: %v", peer, offset, err)
  845. return
  846. }
  847. if reply.Term > r.currentTerm {
  848. r.mu.Lock()
  849. r.becomeFollower(reply.Term)
  850. r.mu.Unlock()
  851. return
  852. }
  853. if !reply.Success {
  854. r.logger.Error("InstallSnapshot chunk rejected by %s at offset %d", peer, offset)
  855. return
  856. }
  857. offset = end
  858. }
  859. // Snapshot fully sent and accepted
  860. r.mu.Lock()
  861. defer r.mu.Unlock()
  862. if r.state != Leader || r.currentTerm != term {
  863. return
  864. }
  865. r.nextIndex[peer] = lastIndex + 1
  866. r.matchIndex[peer] = lastIndex
  867. r.logger.Info("Snapshot to %s completed, nextIndex=%d", peer, lastIndex+1)
  868. }
  869. // updateCommitIndex updates the commit index based on matchIndex
  870. func (r *Raft) updateCommitIndex() {
  871. r.mu.Lock()
  872. defer r.mu.Unlock()
  873. if r.state != Leader {
  874. return
  875. }
  876. // For config change entries, use the OLD cluster for majority calculation
  877. // This ensures that adding a node doesn't require the new node's vote
  878. // until the config change itself is committed
  879. votingNodes := r.clusterNodes
  880. if r.pendingConfigChange && r.oldClusterNodes != nil {
  881. votingNodes = r.oldClusterNodes
  882. }
  883. // Get current cluster size from voting nodes
  884. clusterSize := len(votingNodes)
  885. if clusterSize == 0 {
  886. clusterSize = len(r.peers) + 1
  887. }
  888. // Find the highest index replicated on a majority
  889. for n := r.log.LastIndex(); n > r.commitIndex; n-- {
  890. term, err := r.log.GetTerm(n)
  891. if err != nil {
  892. continue
  893. }
  894. // Only commit entries from current term
  895. if term != r.currentTerm {
  896. continue
  897. }
  898. // Count replicas (including self)
  899. count := 1 // Self
  900. for nodeID, addr := range votingNodes {
  901. if nodeID != r.nodeID {
  902. if r.matchIndex[addr] >= n {
  903. count++
  904. }
  905. }
  906. }
  907. // Also check legacy peers
  908. for _, peer := range r.peers {
  909. // Avoid double counting if peer is already in votingNodes
  910. alreadyCounted := false
  911. for _, addr := range votingNodes {
  912. if addr == peer {
  913. alreadyCounted = true
  914. break
  915. }
  916. }
  917. if !alreadyCounted && r.matchIndex[peer] >= n {
  918. count++
  919. }
  920. }
  921. // Majority is (n/2) + 1
  922. needed := clusterSize/2 + 1
  923. if count >= needed {
  924. r.commitIndex = n
  925. break
  926. }
  927. }
  928. }
  929. // applyLoop applies committed entries to the state machine
  930. // Uses event-driven model with fallback polling for reliability
  931. func (r *Raft) applyLoop() {
  932. // Use a ticker as fallback for missed signals (matches original 10ms polling)
  933. ticker := time.NewTicker(10 * time.Millisecond)
  934. defer ticker.Stop()
  935. for {
  936. select {
  937. case <-r.stopCh:
  938. return
  939. case <-r.commitCh:
  940. // Event-driven: triggered when commitIndex is updated
  941. r.applyCommitted()
  942. case <-ticker.C:
  943. // Fallback: check periodically in case we missed a signal
  944. r.applyCommitted()
  945. }
  946. }
  947. }
  948. // applyCommitted applies all committed but not yet applied entries
  949. func (r *Raft) applyCommitted() {
  950. r.mu.Lock()
  951. commitIndex := r.commitIndex
  952. lastApplied := r.lastApplied
  953. firstIndex := r.log.FirstIndex()
  954. lastLogIndex := r.log.LastIndex()
  955. r.mu.Unlock()
  956. // Safety check: ensure lastApplied is within valid range
  957. // If lastApplied is below firstIndex (due to snapshot), skip to firstIndex
  958. if lastApplied < firstIndex && firstIndex > 0 {
  959. r.mu.Lock()
  960. r.lastApplied = firstIndex
  961. lastApplied = firstIndex
  962. r.mu.Unlock()
  963. r.logger.Debug("Adjusted lastApplied to firstIndex %d after compaction", firstIndex)
  964. }
  965. // Safety check: don't try to apply beyond what we have in log
  966. if commitIndex > lastLogIndex {
  967. commitIndex = lastLogIndex
  968. }
  969. for lastApplied < commitIndex {
  970. lastApplied++
  971. // Skip if entry has been compacted
  972. if lastApplied < firstIndex {
  973. continue
  974. }
  975. entry, err := r.log.GetEntry(lastApplied)
  976. if err != nil {
  977. // If entry is compacted, skip ahead
  978. if err == ErrCompacted {
  979. r.mu.Lock()
  980. newFirstIndex := r.log.FirstIndex()
  981. if lastApplied < newFirstIndex {
  982. r.lastApplied = newFirstIndex
  983. lastApplied = newFirstIndex
  984. }
  985. r.mu.Unlock()
  986. continue
  987. }
  988. r.logger.Error("Failed to get entry %d: %v (firstIndex=%d, lastIndex=%d)",
  989. lastApplied, err, r.log.FirstIndex(), r.log.LastIndex())
  990. break
  991. }
  992. // Handle config change entries
  993. if entry.Type == EntryConfig {
  994. r.applyConfigChange(entry)
  995. r.mu.Lock()
  996. r.lastApplied = lastApplied
  997. r.mu.Unlock()
  998. continue
  999. }
  1000. // Handle no-op entries (just update lastApplied, don't send to state machine)
  1001. if entry.Type == EntryNoop {
  1002. r.mu.Lock()
  1003. r.lastApplied = lastApplied
  1004. r.mu.Unlock()
  1005. continue
  1006. }
  1007. // Normal command entry
  1008. msg := ApplyMsg{
  1009. CommandValid: true,
  1010. Command: entry.Command,
  1011. CommandIndex: entry.Index,
  1012. CommandTerm: entry.Term,
  1013. }
  1014. select {
  1015. case r.applyCh <- msg:
  1016. case <-r.stopCh:
  1017. return
  1018. }
  1019. r.mu.Lock()
  1020. r.lastApplied = lastApplied
  1021. r.mu.Unlock()
  1022. }
  1023. // Check if log compaction is needed
  1024. r.maybeCompactLog()
  1025. }
  1026. // maybeCompactLog checks if automatic log compaction should be triggered
  1027. // It uses a dynamic threshold to prevent "compaction thrashing":
  1028. // - First compaction triggers at SnapshotThreshold (default 100,000)
  1029. // - After compaction, next threshold = current_log_size * 1.5
  1030. // - This prevents every new entry from triggering compaction if log stays large
  1031. func (r *Raft) maybeCompactLog() {
  1032. // Skip if no snapshot provider is configured
  1033. if r.config.SnapshotProvider == nil {
  1034. return
  1035. }
  1036. // Check if compaction is already in progress (atomic check-and-set)
  1037. if !atomic.CompareAndSwapInt32(&r.compacting, 0, 1) {
  1038. return
  1039. }
  1040. // Ensure we release the compacting flag when done
  1041. defer atomic.StoreInt32(&r.compacting, 0)
  1042. r.mu.RLock()
  1043. lastApplied := r.lastApplied
  1044. firstIndex := r.log.FirstIndex()
  1045. initialThreshold := r.config.SnapshotThreshold
  1046. minRetention := r.config.SnapshotMinRetention
  1047. isLeader := r.state == Leader
  1048. dynamicThreshold := r.nextCompactionThreshold
  1049. r.mu.RUnlock()
  1050. // Guard against underflow: ensure lastApplied > firstIndex
  1051. if lastApplied <= firstIndex {
  1052. return
  1053. }
  1054. // Calculate current log size
  1055. logSize := lastApplied - firstIndex
  1056. // Determine effective threshold:
  1057. // - Use initial threshold if no compaction has occurred yet (dynamicThreshold == 0)
  1058. // - Otherwise use the dynamic threshold
  1059. effectiveThreshold := initialThreshold
  1060. if dynamicThreshold > 0 {
  1061. effectiveThreshold = dynamicThreshold
  1062. }
  1063. // Check if we have enough entries to warrant compaction
  1064. if logSize <= effectiveThreshold {
  1065. return
  1066. }
  1067. // Guard against underflow: ensure lastApplied > minRetention
  1068. if lastApplied <= minRetention {
  1069. return
  1070. }
  1071. // Calculate the safe compaction point
  1072. // We need to retain at least minRetention entries for follower catch-up
  1073. compactUpTo := lastApplied - minRetention
  1074. if compactUpTo <= firstIndex {
  1075. return // Not enough entries to compact while maintaining retention
  1076. }
  1077. // For leader, also consider the minimum nextIndex of all followers
  1078. // to avoid compacting entries that followers still need
  1079. if isLeader {
  1080. r.mu.RLock()
  1081. minNextIndex := lastApplied
  1082. for _, nextIdx := range r.nextIndex {
  1083. if nextIdx < minNextIndex {
  1084. minNextIndex = nextIdx
  1085. }
  1086. }
  1087. r.mu.RUnlock()
  1088. // Don't compact entries that followers still need
  1089. // Keep a buffer of minRetention entries before the slowest follower
  1090. if minNextIndex > minRetention {
  1091. followerSafePoint := minNextIndex - minRetention
  1092. if followerSafePoint < compactUpTo {
  1093. compactUpTo = followerSafePoint
  1094. }
  1095. } else {
  1096. // Slowest follower is too far behind, don't compact
  1097. // They will need a full snapshot anyway
  1098. compactUpTo = firstIndex // Effectively skip compaction
  1099. }
  1100. }
  1101. // Final check - make sure we're actually compacting something meaningful
  1102. if compactUpTo <= firstIndex {
  1103. return
  1104. }
  1105. // Get snapshot from application layer
  1106. snapshotData, err := r.config.SnapshotProvider()
  1107. if err != nil {
  1108. r.logger.Error("Failed to get snapshot from provider: %v", err)
  1109. return
  1110. }
  1111. // Get the term at the compaction point
  1112. term, err := r.log.GetTerm(compactUpTo)
  1113. if err != nil {
  1114. r.logger.Error("Failed to get term for compaction index %d: %v", compactUpTo, err)
  1115. return
  1116. }
  1117. // Save snapshot
  1118. if err := r.storage.SaveSnapshot(snapshotData, compactUpTo, term); err != nil {
  1119. r.logger.Error("Failed to save snapshot: %v", err)
  1120. return
  1121. }
  1122. // Compact the log
  1123. // CRITICAL: We must hold the write lock during compaction to prevent
  1124. // concurrent writes (AppendEntries) unless we can guarantee underlying
  1125. // file integrity with concurrent modification. The requirement is to
  1126. // pause writes during compaction.
  1127. r.mu.Lock()
  1128. if err := r.log.Compact(compactUpTo); err != nil {
  1129. r.mu.Unlock()
  1130. r.logger.Error("Failed to compact log: %v", err)
  1131. return
  1132. }
  1133. r.mu.Unlock()
  1134. // Calculate new log size after compaction
  1135. newLogSize := lastApplied - compactUpTo
  1136. // Update dynamic threshold for next compaction: current size * 1.5
  1137. // This prevents "compaction thrashing" where every entry triggers compaction
  1138. r.mu.Lock()
  1139. r.nextCompactionThreshold = newLogSize + newLogSize/2 // newLogSize * 1.5
  1140. // Ensure threshold doesn't go below the initial threshold
  1141. if r.nextCompactionThreshold < initialThreshold {
  1142. r.nextCompactionThreshold = initialThreshold
  1143. }
  1144. r.mu.Unlock()
  1145. r.logger.Info("Auto compaction completed: compacted up to index %d (term %d), log size %d -> %d, next threshold: %d",
  1146. compactUpTo, term, logSize, newLogSize, r.nextCompactionThreshold)
  1147. }
  1148. // resetElectionTimer resets the election timeout
  1149. func (r *Raft) resetElectionTimer() {
  1150. timeout := r.config.ElectionTimeoutMin +
  1151. time.Duration(rand.Int63n(int64(r.config.ElectionTimeoutMax-r.config.ElectionTimeoutMin)))
  1152. if r.electionTimer == nil {
  1153. r.electionTimer = time.NewTimer(timeout)
  1154. } else {
  1155. if !r.electionTimer.Stop() {
  1156. select {
  1157. case <-r.electionTimer.C:
  1158. default:
  1159. }
  1160. }
  1161. r.electionTimer.Reset(timeout)
  1162. }
  1163. }
  1164. // persistState saves the current state to stable storage
  1165. // Returns error if persistence fails - caller MUST handle this for safety
  1166. func (r *Raft) persistState() error {
  1167. state := &PersistentState{
  1168. CurrentTerm: r.currentTerm,
  1169. VotedFor: r.votedFor,
  1170. }
  1171. if err := r.storage.SaveState(state); err != nil {
  1172. r.logger.Error("Failed to persist state: %v", err)
  1173. return fmt.Errorf("%w: %v", ErrPersistFailed, err)
  1174. }
  1175. return nil
  1176. }
  1177. // mustPersistState saves state and panics on failure
  1178. // Use this only in critical paths where failure is unrecoverable
  1179. func (r *Raft) mustPersistState() {
  1180. if err := r.persistState(); err != nil {
  1181. // In production, you might want to trigger a graceful shutdown instead
  1182. r.logger.Error("CRITICAL: Failed to persist state, node may be in inconsistent state: %v", err)
  1183. panic(err)
  1184. }
  1185. }
  1186. // HandleRequestVote handles RequestVote RPCs (including pre-vote)
  1187. func (r *Raft) HandleRequestVote(args *RequestVoteArgs) *RequestVoteReply {
  1188. r.mu.Lock()
  1189. defer r.mu.Unlock()
  1190. reply := &RequestVoteReply{
  1191. Term: r.currentTerm,
  1192. VoteGranted: false,
  1193. }
  1194. // Handle pre-vote separately
  1195. if args.PreVote {
  1196. return r.handlePreVote(args)
  1197. }
  1198. // Reply false if term < currentTerm
  1199. if args.Term < r.currentTerm {
  1200. return reply
  1201. }
  1202. // If term > currentTerm, become follower
  1203. if args.Term > r.currentTerm {
  1204. r.becomeFollower(args.Term)
  1205. }
  1206. reply.Term = r.currentTerm
  1207. // Check if we can vote for this candidate
  1208. if (r.votedFor == "" || r.votedFor == args.CandidateID) &&
  1209. r.log.IsUpToDate(args.LastLogIndex, args.LastLogTerm) {
  1210. r.votedFor = args.CandidateID
  1211. if err := r.persistState(); err != nil {
  1212. // Cannot grant vote if we can't persist the decision
  1213. r.logger.Error("Failed to persist vote for %s: %v", args.CandidateID, err)
  1214. return reply
  1215. }
  1216. r.resetElectionTimer()
  1217. reply.VoteGranted = true
  1218. r.logger.Debug("Granted vote to %s for term %d", args.CandidateID, args.Term)
  1219. }
  1220. return reply
  1221. }
  1222. // handlePreVote handles pre-vote requests
  1223. // Pre-vote doesn't change our state, just checks if we would vote
  1224. func (r *Raft) handlePreVote(args *RequestVoteArgs) *RequestVoteReply {
  1225. reply := &RequestVoteReply{
  1226. Term: r.currentTerm,
  1227. VoteGranted: false,
  1228. }
  1229. // For pre-vote, we check:
  1230. // 1. The candidate's term is at least as high as ours
  1231. // 2. The candidate's log is at least as up-to-date as ours
  1232. // 3. We don't have a current leader (or the candidate's term is higher)
  1233. if args.Term < r.currentTerm {
  1234. return reply
  1235. }
  1236. // Per Raft Pre-Vote optimization (§9.6): reject pre-vote if we have a current
  1237. // leader and the candidate's term is not higher than ours. This prevents
  1238. // disruptive elections when a partitioned node tries to rejoin.
  1239. if r.leaderID != "" && args.Term <= r.currentTerm {
  1240. r.logger.Debug("Rejecting pre-vote from %s: have leader %s", args.CandidateID, r.leaderID)
  1241. return reply
  1242. }
  1243. // Grant pre-vote if log is up-to-date
  1244. // Note: we don't check votedFor for pre-vote, and we don't update any state
  1245. if r.log.IsUpToDate(args.LastLogIndex, args.LastLogTerm) {
  1246. reply.VoteGranted = true
  1247. r.logger.Debug("Granted pre-vote to %s for term %d", args.CandidateID, args.Term)
  1248. }
  1249. return reply
  1250. }
  1251. // HandleAppendEntries handles AppendEntries RPCs
  1252. func (r *Raft) HandleAppendEntries(args *AppendEntriesArgs) *AppendEntriesReply {
  1253. r.mu.Lock()
  1254. defer r.mu.Unlock()
  1255. atomic.AddInt64(&r.stats.AppendsReceived, 1)
  1256. reply := &AppendEntriesReply{
  1257. Term: r.currentTerm,
  1258. Success: false,
  1259. }
  1260. // Check if we're a standalone node being added to a cluster
  1261. // A standalone node has only itself in clusterNodes
  1262. isStandalone := len(r.clusterNodes) == 1
  1263. if _, hasSelf := r.clusterNodes[r.nodeID]; isStandalone && hasSelf {
  1264. // We're standalone and receiving AppendEntries from an external leader
  1265. // This means we're being added to a cluster - suppress elections
  1266. if !r.joiningCluster {
  1267. r.joiningCluster = true
  1268. r.joiningClusterTime = time.Now()
  1269. r.logger.Info("Detected cluster join in progress, suppressing elections")
  1270. }
  1271. // When joining, accept higher terms from the leader to sync up
  1272. if args.Term > r.currentTerm {
  1273. r.becomeFollower(args.Term)
  1274. }
  1275. }
  1276. // Reply false if term < currentTerm
  1277. if args.Term < r.currentTerm {
  1278. // But still reset timer if we're joining a cluster to prevent elections
  1279. if r.joiningCluster {
  1280. r.resetElectionTimer()
  1281. }
  1282. return reply
  1283. }
  1284. // If term > currentTerm, or we're a candidate, or we're a leader receiving
  1285. // AppendEntries from another leader (split-brain scenario during cluster merge),
  1286. // become follower. In Raft, there can only be one leader per term.
  1287. if args.Term > r.currentTerm || r.state == Candidate || r.state == Leader {
  1288. r.becomeFollower(args.Term)
  1289. }
  1290. // Update leader info and reset election timer
  1291. r.leaderID = args.LeaderID
  1292. r.lastHeartbeat = time.Now()
  1293. r.resetElectionTimer()
  1294. reply.Term = r.currentTerm
  1295. // Try to append entries
  1296. success, conflictIndex, conflictTerm := r.log.AppendEntriesFromLeader(
  1297. args.PrevLogIndex, args.PrevLogTerm, args.Entries)
  1298. if !success {
  1299. reply.ConflictIndex = conflictIndex
  1300. reply.ConflictTerm = conflictTerm
  1301. return reply
  1302. }
  1303. reply.Success = true
  1304. // Update commit index safely
  1305. if args.LeaderCommit > r.commitIndex {
  1306. // Get our actual last log index
  1307. lastLogIndex := r.log.LastIndex()
  1308. // Calculate what index the entries would have reached
  1309. lastNewEntry := args.PrevLogIndex
  1310. if len(args.Entries) > 0 {
  1311. lastNewEntry = args.Entries[len(args.Entries)-1].Index
  1312. }
  1313. // Commit index should not exceed what we actually have in log
  1314. newCommitIndex := args.LeaderCommit
  1315. if newCommitIndex > lastNewEntry {
  1316. newCommitIndex = lastNewEntry
  1317. }
  1318. if newCommitIndex > lastLogIndex {
  1319. newCommitIndex = lastLogIndex
  1320. }
  1321. // Only advance commit index
  1322. if newCommitIndex > r.commitIndex {
  1323. r.commitIndex = newCommitIndex
  1324. }
  1325. }
  1326. return reply
  1327. }
  1328. // HandleInstallSnapshot handles InstallSnapshot RPCs with chunked transfer support
  1329. func (r *Raft) HandleInstallSnapshot(args *InstallSnapshotArgs) *InstallSnapshotReply {
  1330. r.mu.Lock()
  1331. defer r.mu.Unlock()
  1332. reply := &InstallSnapshotReply{
  1333. Term: r.currentTerm,
  1334. Success: false,
  1335. }
  1336. if args.Term < r.currentTerm {
  1337. return reply
  1338. }
  1339. if args.Term > r.currentTerm {
  1340. r.becomeFollower(args.Term)
  1341. }
  1342. r.leaderID = args.LeaderID
  1343. r.lastHeartbeat = time.Now()
  1344. r.resetElectionTimer()
  1345. reply.Term = r.currentTerm
  1346. // Skip if we already have this or a newer snapshot applied
  1347. if args.LastIncludedIndex <= r.lastApplied {
  1348. r.logger.Debug("Ignoring snapshot at index %d, already applied up to %d",
  1349. args.LastIncludedIndex, r.lastApplied)
  1350. reply.Success = true // Still success to let leader know we don't need it
  1351. return reply
  1352. }
  1353. // Handle chunked transfer
  1354. if args.Offset == 0 {
  1355. // First chunk - start new pending snapshot
  1356. r.pendingSnapshot = &pendingSnapshotState{
  1357. lastIncludedIndex: args.LastIncludedIndex,
  1358. lastIncludedTerm: args.LastIncludedTerm,
  1359. data: make([]byte, 0),
  1360. receivedBytes: 0,
  1361. }
  1362. r.logger.Info("Starting snapshot reception at index %d, term %d",
  1363. args.LastIncludedIndex, args.LastIncludedTerm)
  1364. }
  1365. // Validate we're receiving the expected snapshot
  1366. if r.pendingSnapshot == nil ||
  1367. r.pendingSnapshot.lastIncludedIndex != args.LastIncludedIndex ||
  1368. r.pendingSnapshot.lastIncludedTerm != args.LastIncludedTerm {
  1369. r.logger.Warn("Unexpected snapshot chunk: expected index %d, got %d",
  1370. r.pendingSnapshot.lastIncludedIndex, args.LastIncludedIndex)
  1371. return reply
  1372. }
  1373. // Validate offset matches what we've received
  1374. if uint64(args.Offset) != r.pendingSnapshot.receivedBytes {
  1375. r.logger.Warn("Unexpected chunk offset: expected %d, got %d",
  1376. r.pendingSnapshot.receivedBytes, args.Offset)
  1377. return reply
  1378. }
  1379. // Append chunk data
  1380. r.pendingSnapshot.data = append(r.pendingSnapshot.data, args.Data...)
  1381. r.pendingSnapshot.receivedBytes += uint64(len(args.Data))
  1382. reply.Success = true
  1383. r.logger.Debug("Received snapshot chunk: offset=%d, size=%d, done=%v",
  1384. args.Offset, len(args.Data), args.Done)
  1385. // If not done, wait for more chunks
  1386. if !args.Done {
  1387. return reply
  1388. }
  1389. // All chunks received - apply the snapshot
  1390. r.logger.Info("Installing complete snapshot: %d bytes at index %d, term %d",
  1391. len(r.pendingSnapshot.data), args.LastIncludedIndex, args.LastIncludedTerm)
  1392. atomic.AddUint64(&r.metrics.SnapshotsInstalled, 1)
  1393. // Save snapshot
  1394. if err := r.storage.SaveSnapshot(r.pendingSnapshot.data, args.LastIncludedIndex, args.LastIncludedTerm); err != nil {
  1395. r.logger.Error("Failed to save snapshot: %v", err)
  1396. r.pendingSnapshot = nil
  1397. reply.Success = false
  1398. return reply
  1399. }
  1400. // Compact log
  1401. if err := r.log.Compact(args.LastIncludedIndex); err != nil {
  1402. r.logger.Error("Failed to compact log: %v", err)
  1403. }
  1404. // Update state - must update both commitIndex and lastApplied
  1405. if args.LastIncludedIndex > r.commitIndex {
  1406. r.commitIndex = args.LastIncludedIndex
  1407. }
  1408. // Always update lastApplied to snapshot index to prevent trying to apply compacted entries
  1409. r.lastApplied = args.LastIncludedIndex
  1410. // Send snapshot to application (non-blocking with timeout)
  1411. // Use the complete pendingSnapshot data, not the last chunk
  1412. msg := ApplyMsg{
  1413. SnapshotValid: true,
  1414. Snapshot: r.pendingSnapshot.data,
  1415. SnapshotIndex: args.LastIncludedIndex,
  1416. SnapshotTerm: args.LastIncludedTerm,
  1417. }
  1418. // Clear pending snapshot
  1419. r.pendingSnapshot = nil
  1420. // Try to send, but don't block indefinitely
  1421. select {
  1422. case r.applyCh <- msg:
  1423. r.logger.Debug("Sent snapshot to application")
  1424. case <-time.After(100 * time.Millisecond):
  1425. r.logger.Warn("Timeout sending snapshot to application, will retry")
  1426. // The application will still get correct state via normal apply loop
  1427. }
  1428. return reply
  1429. }
  1430. // Propose proposes a new command to be replicated
  1431. func (r *Raft) Propose(command []byte) (uint64, uint64, bool) {
  1432. r.mu.Lock()
  1433. defer r.mu.Unlock()
  1434. if r.state != Leader {
  1435. return 0, 0, false
  1436. }
  1437. // Check connectivity (Lease Check)
  1438. // If we haven't heard from a majority of peers within ElectionTimeout,
  1439. // we shouldn't accept new commands because we might be partitioned.
  1440. if len(r.clusterNodes) > 1 {
  1441. activePeers := 1 // Self
  1442. now := time.Now()
  1443. timeout := r.config.ElectionTimeoutMax
  1444. for _, peer := range r.peers {
  1445. if last, ok := r.lastContact[peer]; ok && now.Sub(last) < timeout {
  1446. activePeers++
  1447. }
  1448. }
  1449. // Check majority
  1450. needed := len(r.clusterNodes)/2 + 1
  1451. if activePeers < needed {
  1452. r.logger.Warn("Rejecting Propose: lost contact with majority (active: %d, needed: %d)", activePeers, needed)
  1453. return 0, 0, false
  1454. }
  1455. }
  1456. index, err := r.log.AppendCommand(r.currentTerm, command)
  1457. if err != nil {
  1458. r.logger.Error("Failed to append command: %v", err)
  1459. return 0, 0, false
  1460. }
  1461. r.matchIndex[r.nodeID] = index
  1462. // For single-node cluster, we are the only voter and can commit immediately
  1463. // This fixes the issue where commitCh never gets triggered without other peers
  1464. if len(r.clusterNodes) <= 1 && len(r.peers) == 0 {
  1465. // Single node: self is majority, trigger commit immediately
  1466. select {
  1467. case r.commitCh <- struct{}{}:
  1468. default:
  1469. }
  1470. } else {
  1471. // Multi-node: trigger replication to other nodes
  1472. r.triggerReplication()
  1473. }
  1474. return index, r.currentTerm, true
  1475. }
  1476. // triggerReplication signals the replication loop to send heartbeats
  1477. // This uses a non-blocking send to batch replication requests
  1478. func (r *Raft) triggerReplication() {
  1479. select {
  1480. case r.replicationCh <- struct{}{}:
  1481. default:
  1482. // Replication already scheduled
  1483. }
  1484. }
  1485. // replicationLoop handles batched replication
  1486. // Uses simple delay-based batching: flush immediately when signaled, then wait
  1487. // to allow more requests to accumulate before the next flush.
  1488. func (r *Raft) replicationLoop() {
  1489. for {
  1490. select {
  1491. case <-r.stopCh:
  1492. return
  1493. case <-r.replicationCh:
  1494. // Flush and replicate immediately
  1495. r.flushAndReplicate()
  1496. // Wait briefly to allow batching of subsequent requests
  1497. // This gives time for more proposals to queue up before the next flush
  1498. time.Sleep(10 * time.Millisecond)
  1499. }
  1500. }
  1501. }
  1502. // flushAndReplicate flushes logs and sends heartbeats
  1503. func (r *Raft) flushAndReplicate() {
  1504. // Ensure logs are flushed to OS cache before sending to followers
  1505. // This implements Group Commit with Flush (fast) instead of Sync (slow)
  1506. if err := r.log.Flush(); err != nil {
  1507. r.logger.Error("Failed to flush log: %v", err)
  1508. }
  1509. r.sendHeartbeats()
  1510. }
  1511. // ProposeWithForward proposes a command, forwarding to leader if necessary
  1512. // This is the recommended method for applications to use
  1513. func (r *Raft) ProposeWithForward(command []byte) (index uint64, term uint64, err error) {
  1514. // Try local propose first
  1515. idx, t, isLeader := r.Propose(command)
  1516. if isLeader {
  1517. return idx, t, nil
  1518. }
  1519. // Not leader, forward to leader
  1520. r.mu.RLock()
  1521. leaderID := r.leaderID
  1522. // Use clusterNodes (dynamically maintained) to find leader address
  1523. leaderAddr := r.clusterNodes[leaderID]
  1524. r.mu.RUnlock()
  1525. if leaderID == "" {
  1526. return 0, 0, fmt.Errorf("no leader available")
  1527. }
  1528. if leaderAddr == "" {
  1529. return 0, 0, fmt.Errorf("leader %s address not found in cluster", leaderID)
  1530. }
  1531. // Forward to leader
  1532. ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second)
  1533. defer cancel()
  1534. args := &ProposeArgs{Command: command}
  1535. reply, err := r.transport.ForwardPropose(ctx, leaderAddr, args)
  1536. if err != nil {
  1537. return 0, 0, fmt.Errorf("forward failed: %w", err)
  1538. }
  1539. if !reply.Success {
  1540. return 0, 0, fmt.Errorf("leader rejected: %s", reply.Error)
  1541. }
  1542. return reply.Index, reply.Term, nil
  1543. }
  1544. // ForwardGet forwards a get request to the leader
  1545. func (r *Raft) ForwardGet(key string) (string, bool, error) {
  1546. // Check if we are leader (local read)
  1547. if r.state == Leader {
  1548. if r.config.GetHandler != nil {
  1549. val, found := r.config.GetHandler(key)
  1550. return val, found, nil
  1551. }
  1552. return "", false, fmt.Errorf("get handler not configured")
  1553. }
  1554. r.mu.RLock()
  1555. leaderID := r.leaderID
  1556. leaderAddr := r.clusterNodes[leaderID]
  1557. r.mu.RUnlock()
  1558. if leaderID == "" {
  1559. return "", false, ErrNoLeader
  1560. }
  1561. if leaderAddr == "" {
  1562. return "", false, fmt.Errorf("leader %s address not found", leaderID)
  1563. }
  1564. // Forward to leader
  1565. ctx, cancel := context.WithTimeout(context.Background(), r.config.RPCTimeout)
  1566. defer cancel()
  1567. args := &GetArgs{Key: key}
  1568. reply, err := r.transport.ForwardGet(ctx, leaderAddr, args)
  1569. if err != nil {
  1570. return "", false, fmt.Errorf("forward failed: %w", err)
  1571. }
  1572. return reply.Value, reply.Found, nil
  1573. }
  1574. // HandlePropose handles forwarded propose requests
  1575. func (r *Raft) HandlePropose(args *ProposeArgs) *ProposeReply {
  1576. index, term, isLeader := r.Propose(args.Command)
  1577. if !isLeader {
  1578. return &ProposeReply{
  1579. Success: false,
  1580. Error: "not leader",
  1581. }
  1582. }
  1583. return &ProposeReply{
  1584. Success: true,
  1585. Index: index,
  1586. Term: term,
  1587. }
  1588. }
  1589. // HandleAddNode handles forwarded AddNode requests
  1590. func (r *Raft) HandleAddNode(args *AddNodeArgs) *AddNodeReply {
  1591. err := r.AddNode(args.NodeID, args.Address)
  1592. if err != nil {
  1593. return &AddNodeReply{
  1594. Success: false,
  1595. Error: err.Error(),
  1596. }
  1597. }
  1598. return &AddNodeReply{
  1599. Success: true,
  1600. }
  1601. }
  1602. // HandleRemoveNode handles forwarded RemoveNode requests
  1603. func (r *Raft) HandleRemoveNode(args *RemoveNodeArgs) *RemoveNodeReply {
  1604. err := r.RemoveNode(args.NodeID)
  1605. if err != nil {
  1606. return &RemoveNodeReply{
  1607. Success: false,
  1608. Error: err.Error(),
  1609. }
  1610. }
  1611. return &RemoveNodeReply{
  1612. Success: true,
  1613. }
  1614. }
  1615. // GetState returns the current term and whether this node is leader
  1616. func (r *Raft) GetState() (uint64, bool) {
  1617. r.mu.RLock()
  1618. defer r.mu.RUnlock()
  1619. return r.currentTerm, r.state == Leader
  1620. }
  1621. // GetLeaderID returns the current leader ID
  1622. func (r *Raft) GetLeaderID() string {
  1623. r.mu.RLock()
  1624. defer r.mu.RUnlock()
  1625. return r.leaderID
  1626. }
  1627. // GetStats returns runtime statistics
  1628. func (r *Raft) GetStats() Stats {
  1629. r.mu.RLock()
  1630. defer r.mu.RUnlock()
  1631. lastIndex, lastTerm := r.log.LastIndexAndTerm()
  1632. firstIndex := r.log.FirstIndex()
  1633. // Copy cluster nodes
  1634. nodes := make(map[string]string)
  1635. for k, v := range r.clusterNodes {
  1636. nodes[k] = v
  1637. }
  1638. clusterSize := len(r.clusterNodes)
  1639. if clusterSize == 0 {
  1640. clusterSize = len(r.peers) + 1
  1641. }
  1642. return Stats{
  1643. Term: r.currentTerm,
  1644. State: r.state.String(),
  1645. FirstLogIndex: firstIndex,
  1646. LastLogIndex: lastIndex,
  1647. LastLogTerm: lastTerm,
  1648. CommitIndex: r.commitIndex,
  1649. LastApplied: r.lastApplied,
  1650. LeaderID: r.leaderID,
  1651. AppendsSent: atomic.LoadInt64(&r.stats.AppendsSent),
  1652. AppendsReceived: atomic.LoadInt64(&r.stats.AppendsReceived),
  1653. ClusterSize: clusterSize,
  1654. ClusterNodes: nodes,
  1655. }
  1656. }
  1657. // restoreFromSnapshot restores the FSM from a snapshot at startup
  1658. // This is called during Start() to ensure the FSM has the correct state
  1659. // before processing any new commands
  1660. func (r *Raft) restoreFromSnapshot() error {
  1661. // Get snapshot from storage
  1662. data, lastIndex, lastTerm, err := r.storage.GetSnapshot()
  1663. if err != nil {
  1664. return fmt.Errorf("failed to get snapshot: %w", err)
  1665. }
  1666. // No snapshot exists
  1667. if len(data) == 0 || lastIndex == 0 {
  1668. return nil
  1669. }
  1670. r.logger.Info("Restoring FSM from snapshot at index %d, term %d (%d bytes)",
  1671. lastIndex, lastTerm, len(data))
  1672. // Update lastApplied to snapshot index to prevent re-applying compacted entries
  1673. r.mu.Lock()
  1674. if lastIndex > r.lastApplied {
  1675. r.lastApplied = lastIndex
  1676. }
  1677. if lastIndex > r.commitIndex {
  1678. r.commitIndex = lastIndex
  1679. }
  1680. r.mu.Unlock()
  1681. // Send snapshot to FSM for restoration
  1682. // Use a goroutine with timeout to avoid blocking if applyCh is full
  1683. msg := ApplyMsg{
  1684. SnapshotValid: true,
  1685. Snapshot: data,
  1686. SnapshotIndex: lastIndex,
  1687. SnapshotTerm: lastTerm,
  1688. }
  1689. // Try to send with a timeout
  1690. select {
  1691. case r.applyCh <- msg:
  1692. r.logger.Info("FSM restoration triggered from snapshot at index %d", lastIndex)
  1693. case <-time.After(5 * time.Second):
  1694. return fmt.Errorf("timeout sending snapshot to applyCh")
  1695. }
  1696. return nil
  1697. }
  1698. // TakeSnapshot takes a snapshot of the current state
  1699. func (r *Raft) TakeSnapshot(data []byte, index uint64) error {
  1700. r.mu.Lock()
  1701. defer r.mu.Unlock()
  1702. if index > r.lastApplied {
  1703. return fmt.Errorf("snapshot index %d exceeds lastApplied %d", index, r.lastApplied)
  1704. }
  1705. term, err := r.log.GetTerm(index)
  1706. if err != nil {
  1707. return fmt.Errorf("failed to get term for index %d: %w", index, err)
  1708. }
  1709. if err := r.storage.SaveSnapshot(data, index, term); err != nil {
  1710. return fmt.Errorf("failed to save snapshot: %w", err)
  1711. }
  1712. if err := r.log.Compact(index); err != nil {
  1713. return fmt.Errorf("failed to compact log: %w", err)
  1714. }
  1715. r.logger.Info("Took snapshot at index %d, term %d", index, term)
  1716. return nil
  1717. }
  1718. func max(a, b uint64) uint64 {
  1719. if a > b {
  1720. return a
  1721. }
  1722. return b
  1723. }
  1724. // ==================== Membership Change API ====================
  1725. //
  1726. // This implementation uses Single-Node Membership Change (also known as one-at-a-time changes)
  1727. // as described in the Raft dissertation (§4.3). This is safe because:
  1728. //
  1729. // 1. We only allow one configuration change at a time (pendingConfigChange flag)
  1730. // 2. For commits, we use the OLD cluster majority until the config change is committed
  1731. // 3. The new node starts receiving entries immediately but doesn't affect majority calculation
  1732. //
  1733. // This approach is simpler than Joint Consensus and is sufficient for most use cases.
  1734. // The invariant maintained is: any two majorities (old or new) must overlap.
  1735. //
  1736. // For adding a node: old majority = N/2+1, new majority = (N+1)/2+1 = N/2+1 (overlaps)
  1737. // For removing a node: old majority = N/2+1, new majority = (N-1)/2+1 (overlaps if N > 1)
  1738. //
  1739. // WARNING: Avoid adding/removing multiple nodes rapidly. Wait for each change to be committed.
  1740. // AddNode adds a new node to the cluster
  1741. // This can only be called on the leader
  1742. // The new node must already be running and reachable
  1743. //
  1744. // Safety guarantees:
  1745. // - Only one config change can be in progress at a time
  1746. // - The old cluster majority is used until the config change is committed
  1747. // - Returns error if leadership is lost during the operation
  1748. func (r *Raft) AddNode(nodeID, address string) error {
  1749. r.mu.Lock()
  1750. // Must be leader
  1751. if r.state != Leader {
  1752. leaderID := r.leaderID
  1753. r.mu.Unlock()
  1754. return NewRaftError(ErrNotLeader, leaderID, r.config.RetryBackoff)
  1755. }
  1756. // Check if we're in the middle of a leadership transfer
  1757. if r.transferring {
  1758. r.mu.Unlock()
  1759. return fmt.Errorf("leadership transfer in progress")
  1760. }
  1761. // Check if there's already a pending config change
  1762. if r.pendingConfigChange {
  1763. r.mu.Unlock()
  1764. return ErrConfigInFlight
  1765. }
  1766. // Validate nodeID and address
  1767. if nodeID == "" {
  1768. r.mu.Unlock()
  1769. return fmt.Errorf("nodeID cannot be empty")
  1770. }
  1771. if address == "" {
  1772. r.mu.Unlock()
  1773. return fmt.Errorf("address cannot be empty")
  1774. }
  1775. // Check if node already exists
  1776. if _, exists := r.clusterNodes[nodeID]; exists {
  1777. r.mu.Unlock()
  1778. return fmt.Errorf("node %s already exists in cluster", nodeID)
  1779. }
  1780. // Check if address is already used by another node
  1781. for existingID, existingAddr := range r.clusterNodes {
  1782. if existingAddr == address {
  1783. r.mu.Unlock()
  1784. return fmt.Errorf("address %s is already used by node %s", address, existingID)
  1785. }
  1786. }
  1787. // Save old cluster nodes for majority calculation during config change
  1788. // This ensures we use the OLD cluster size until the config change is committed
  1789. r.oldClusterNodes = make(map[string]string)
  1790. for k, v := range r.clusterNodes {
  1791. r.oldClusterNodes[k] = v
  1792. }
  1793. // Create new config with the added node
  1794. newNodes := make(map[string]string)
  1795. for k, v := range r.clusterNodes {
  1796. newNodes[k] = v
  1797. }
  1798. newNodes[nodeID] = address
  1799. // Create config change entry with ClusterConfig
  1800. configIndex := r.log.LastIndex() + 1
  1801. entry := LogEntry{
  1802. Index: configIndex,
  1803. Term: r.currentTerm,
  1804. Type: EntryConfig,
  1805. Config: &ClusterConfig{Nodes: newNodes},
  1806. }
  1807. // Mark config change as pending and store the index
  1808. r.pendingConfigChange = true
  1809. r.configChangeIndex = configIndex
  1810. // Immediately apply the new configuration (for single-node changes, this is safe)
  1811. // The new node will start receiving AppendEntries immediately
  1812. r.clusterNodes[nodeID] = address
  1813. r.peers = append(r.peers, address)
  1814. // Set nextIndex to 1 (or firstIndex) so the new node syncs from the beginning
  1815. // This is crucial - the new node's log is empty, so we must start from index 1
  1816. firstIndex := r.log.FirstIndex()
  1817. if firstIndex == 0 {
  1818. firstIndex = 1
  1819. }
  1820. r.nextIndex[address] = firstIndex
  1821. r.matchIndex[address] = 0
  1822. r.mu.Unlock()
  1823. // Append the config change entry to log
  1824. if err := r.log.Append(entry); err != nil {
  1825. r.mu.Lock()
  1826. r.pendingConfigChange = false
  1827. r.oldClusterNodes = nil
  1828. r.configChangeIndex = 0
  1829. // Rollback
  1830. delete(r.clusterNodes, nodeID)
  1831. r.rebuildPeersList()
  1832. r.mu.Unlock()
  1833. return fmt.Errorf("failed to append config entry: %w", err)
  1834. }
  1835. r.logger.Info("Adding node %s (%s) to cluster", nodeID, address)
  1836. // Trigger immediate replication
  1837. r.triggerReplication()
  1838. return nil
  1839. }
  1840. // RemoveNode removes a node from the cluster
  1841. // This can only be called on the leader
  1842. // The node being removed can be any node except the leader itself
  1843. //
  1844. // Safety guarantees:
  1845. // - Only one config change can be in progress at a time
  1846. // - Cannot remove the leader (transfer leadership first)
  1847. // - Cannot reduce cluster to 0 nodes
  1848. // - The old cluster majority is used until the config change is committed
  1849. func (r *Raft) RemoveNode(nodeID string) error {
  1850. r.mu.Lock()
  1851. // Must be leader
  1852. if r.state != Leader {
  1853. leaderID := r.leaderID
  1854. r.mu.Unlock()
  1855. return NewRaftError(ErrNotLeader, leaderID, r.config.RetryBackoff)
  1856. }
  1857. // Check if we're in the middle of a leadership transfer
  1858. if r.transferring {
  1859. r.mu.Unlock()
  1860. return fmt.Errorf("leadership transfer in progress")
  1861. }
  1862. // Cannot remove self
  1863. if nodeID == r.nodeID {
  1864. r.mu.Unlock()
  1865. return fmt.Errorf("cannot remove self from cluster, use TransferLeadership first")
  1866. }
  1867. // Validate nodeID
  1868. if nodeID == "" {
  1869. r.mu.Unlock()
  1870. return fmt.Errorf("nodeID cannot be empty")
  1871. }
  1872. // Check if there's already a pending config change
  1873. if r.pendingConfigChange {
  1874. r.mu.Unlock()
  1875. return ErrConfigInFlight
  1876. }
  1877. // Check if node exists
  1878. if _, exists := r.clusterNodes[nodeID]; !exists {
  1879. r.mu.Unlock()
  1880. return fmt.Errorf("node %s not found in cluster", nodeID)
  1881. }
  1882. // Cannot reduce cluster below 1 node
  1883. if len(r.clusterNodes) <= 1 {
  1884. r.mu.Unlock()
  1885. return fmt.Errorf("cannot remove last node from cluster")
  1886. }
  1887. // Save old cluster nodes for majority calculation during config change
  1888. r.oldClusterNodes = make(map[string]string)
  1889. for k, v := range r.clusterNodes {
  1890. r.oldClusterNodes[k] = v
  1891. }
  1892. // Create new config without the removed node
  1893. newNodes := make(map[string]string)
  1894. for k, v := range r.clusterNodes {
  1895. if k != nodeID {
  1896. newNodes[k] = v
  1897. }
  1898. }
  1899. // Create config change entry with ClusterConfig
  1900. configIndex := r.log.LastIndex() + 1
  1901. entry := LogEntry{
  1902. Index: configIndex,
  1903. Term: r.currentTerm,
  1904. Type: EntryConfig,
  1905. Config: &ClusterConfig{Nodes: newNodes},
  1906. }
  1907. // Mark config change as pending and store the index
  1908. r.pendingConfigChange = true
  1909. r.configChangeIndex = configIndex
  1910. // Get the address of node being removed for cleanup
  1911. removedAddr := r.clusterNodes[nodeID]
  1912. // Immediately apply the new configuration
  1913. delete(r.clusterNodes, nodeID)
  1914. r.rebuildPeersList()
  1915. delete(r.nextIndex, removedAddr)
  1916. delete(r.matchIndex, removedAddr)
  1917. r.mu.Unlock()
  1918. // Append the config change entry to log
  1919. if err := r.log.Append(entry); err != nil {
  1920. r.mu.Lock()
  1921. r.pendingConfigChange = false
  1922. r.oldClusterNodes = nil
  1923. r.configChangeIndex = 0
  1924. // Rollback - this is tricky but we try our best
  1925. r.clusterNodes[nodeID] = removedAddr
  1926. r.rebuildPeersList()
  1927. r.mu.Unlock()
  1928. return fmt.Errorf("failed to append config entry: %w", err)
  1929. }
  1930. r.logger.Info("Removing node %s from cluster, config at index %d", nodeID, entry.Index)
  1931. // Trigger replication
  1932. go r.sendHeartbeats()
  1933. return nil
  1934. }
  1935. // AddNodeWithForward adds a node, forwarding to leader if necessary
  1936. // This is the recommended method for applications to use
  1937. func (r *Raft) AddNodeWithForward(nodeID, address string) error {
  1938. // Try local operation first
  1939. err := r.AddNode(nodeID, address)
  1940. if err == nil {
  1941. return nil
  1942. }
  1943. // Check if we're not the leader
  1944. r.mu.RLock()
  1945. state := r.state
  1946. leaderID := r.leaderID
  1947. leaderAddr := r.clusterNodes[leaderID]
  1948. r.mu.RUnlock()
  1949. if state == Leader {
  1950. // We are leader but AddNode failed for other reasons
  1951. return err
  1952. }
  1953. // Not leader, forward to leader
  1954. if leaderID == "" {
  1955. return fmt.Errorf("no leader available")
  1956. }
  1957. if leaderAddr == "" {
  1958. return fmt.Errorf("leader %s address not found in cluster", leaderID)
  1959. }
  1960. // Forward to leader
  1961. ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
  1962. defer cancel()
  1963. args := &AddNodeArgs{NodeID: nodeID, Address: address}
  1964. reply, err := r.transport.ForwardAddNode(ctx, leaderAddr, args)
  1965. if err != nil {
  1966. return fmt.Errorf("forward failed: %w", err)
  1967. }
  1968. if !reply.Success {
  1969. return fmt.Errorf("leader rejected: %s", reply.Error)
  1970. }
  1971. return nil
  1972. }
  1973. // RemoveNodeWithForward removes a node, forwarding to leader if necessary
  1974. // This is the recommended method for applications to use
  1975. func (r *Raft) RemoveNodeWithForward(nodeID string) error {
  1976. // Try local operation first
  1977. err := r.RemoveNode(nodeID)
  1978. if err == nil {
  1979. return nil
  1980. }
  1981. // Check if we're not the leader
  1982. r.mu.RLock()
  1983. state := r.state
  1984. leaderID := r.leaderID
  1985. leaderAddr := r.clusterNodes[leaderID]
  1986. r.mu.RUnlock()
  1987. if state == Leader {
  1988. // We are leader but RemoveNode failed for other reasons
  1989. return err
  1990. }
  1991. // Not leader, forward to leader
  1992. if leaderID == "" {
  1993. return fmt.Errorf("no leader available")
  1994. }
  1995. if leaderAddr == "" {
  1996. return fmt.Errorf("leader %s address not found in cluster", leaderID)
  1997. }
  1998. // Forward to leader
  1999. ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
  2000. defer cancel()
  2001. args := &RemoveNodeArgs{NodeID: nodeID}
  2002. reply, err := r.transport.ForwardRemoveNode(ctx, leaderAddr, args)
  2003. if err != nil {
  2004. return fmt.Errorf("forward failed: %w", err)
  2005. }
  2006. if !reply.Success {
  2007. return fmt.Errorf("leader rejected: %s", reply.Error)
  2008. }
  2009. return nil
  2010. }
  2011. // rebuildPeersList rebuilds the peers slice from clusterNodes
  2012. func (r *Raft) rebuildPeersList() {
  2013. r.peers = make([]string, 0, len(r.clusterNodes)-1)
  2014. for nodeID, addr := range r.clusterNodes {
  2015. if nodeID != r.nodeID {
  2016. r.peers = append(r.peers, addr)
  2017. }
  2018. }
  2019. }
  2020. // GetClusterNodes returns a copy of the current cluster membership
  2021. func (r *Raft) GetClusterNodes() map[string]string {
  2022. r.mu.RLock()
  2023. defer r.mu.RUnlock()
  2024. nodes := make(map[string]string)
  2025. for k, v := range r.clusterNodes {
  2026. nodes[k] = v
  2027. }
  2028. return nodes
  2029. }
  2030. // applyConfigChange applies a configuration change entry
  2031. func (r *Raft) applyConfigChange(entry *LogEntry) {
  2032. if entry.Config == nil || entry.Config.Nodes == nil {
  2033. r.logger.Warn("Invalid config change entry at index %d", entry.Index)
  2034. return
  2035. }
  2036. r.mu.Lock()
  2037. defer r.mu.Unlock()
  2038. // Update cluster configuration
  2039. r.clusterNodes = make(map[string]string)
  2040. for k, v := range entry.Config.Nodes {
  2041. r.clusterNodes[k] = v
  2042. }
  2043. r.rebuildPeersList()
  2044. // Persist the new configuration
  2045. if err := r.storage.SaveClusterConfig(&ClusterConfig{Nodes: r.clusterNodes}); err != nil {
  2046. r.logger.Error("Failed to persist cluster config: %v", err)
  2047. }
  2048. // Clear pending flag and old cluster state
  2049. r.pendingConfigChange = false
  2050. r.oldClusterNodes = nil
  2051. r.configChangeIndex = 0
  2052. // If we were joining a cluster and now have multiple nodes, we've successfully joined
  2053. if r.joiningCluster && len(r.clusterNodes) > 1 {
  2054. r.joiningCluster = false
  2055. r.logger.Info("Successfully joined cluster with %d nodes", len(r.clusterNodes))
  2056. }
  2057. r.logger.Info("Applied config change at index %d, cluster now has %d nodes", entry.Index, len(r.clusterNodes))
  2058. // If we're the leader, update leader state
  2059. if r.state == Leader {
  2060. // Initialize nextIndex/matchIndex for any new nodes
  2061. lastIndex := r.log.LastIndex()
  2062. for nodeID, addr := range r.clusterNodes {
  2063. if nodeID != r.nodeID {
  2064. if _, exists := r.nextIndex[addr]; !exists {
  2065. r.nextIndex[addr] = lastIndex + 1
  2066. r.matchIndex[addr] = 0
  2067. }
  2068. }
  2069. }
  2070. // Clean up removed nodes
  2071. validAddrs := make(map[string]bool)
  2072. for nodeID, addr := range r.clusterNodes {
  2073. if nodeID != r.nodeID {
  2074. validAddrs[addr] = true
  2075. }
  2076. }
  2077. for addr := range r.nextIndex {
  2078. if !validAddrs[addr] {
  2079. delete(r.nextIndex, addr)
  2080. delete(r.matchIndex, addr)
  2081. }
  2082. }
  2083. }
  2084. }
  2085. // ==================== ReadIndex (Linearizable Reads) ====================
  2086. // readIndexLoop handles read index requests
  2087. func (r *Raft) readIndexLoop() {
  2088. for {
  2089. select {
  2090. case <-r.stopCh:
  2091. return
  2092. case req := <-r.readIndexCh:
  2093. r.processReadIndexRequest(req)
  2094. }
  2095. }
  2096. }
  2097. // processReadIndexRequest processes a single read index request
  2098. func (r *Raft) processReadIndexRequest(req *readIndexRequest) {
  2099. r.mu.RLock()
  2100. if r.state != Leader {
  2101. r.mu.RUnlock()
  2102. req.done <- ErrNotLeader
  2103. return
  2104. }
  2105. r.mu.RUnlock()
  2106. // Confirm leadership by sending heartbeats and waiting for majority ack
  2107. if !r.confirmLeadership() {
  2108. req.done <- ErrLeadershipLost
  2109. return
  2110. }
  2111. // Wait for apply to catch up to readIndex
  2112. if err := r.waitApply(req.readIndex, r.config.ProposeTimeout); err != nil {
  2113. req.done <- err
  2114. return
  2115. }
  2116. req.done <- nil
  2117. }
  2118. // ReadIndex implements linearizable reads
  2119. // It ensures that the read sees all writes that were committed before the read started
  2120. func (r *Raft) ReadIndex() (uint64, error) {
  2121. r.mu.RLock()
  2122. if r.state != Leader {
  2123. leaderID := r.leaderID
  2124. r.mu.RUnlock()
  2125. return 0, NewRaftError(ErrNotLeader, leaderID, r.config.RetryBackoff)
  2126. }
  2127. readIndex := r.commitIndex
  2128. r.mu.RUnlock()
  2129. atomic.AddUint64(&r.metrics.ReadIndexRequests, 1)
  2130. // Create request and send to processing loop
  2131. req := &readIndexRequest{
  2132. readIndex: readIndex,
  2133. done: make(chan error, 1),
  2134. }
  2135. select {
  2136. case r.readIndexCh <- req:
  2137. case <-r.stopCh:
  2138. return 0, ErrShutdown
  2139. case <-time.After(r.config.ProposeTimeout):
  2140. return 0, ErrTimeout
  2141. }
  2142. // Wait for result
  2143. select {
  2144. case err := <-req.done:
  2145. if err != nil {
  2146. return 0, err
  2147. }
  2148. atomic.AddUint64(&r.metrics.ReadIndexSuccess, 1)
  2149. return readIndex, nil
  2150. case <-r.stopCh:
  2151. return 0, ErrShutdown
  2152. case <-time.After(r.config.ProposeTimeout):
  2153. return 0, ErrTimeout
  2154. }
  2155. }
  2156. // confirmLeadership confirms we're still leader by getting acks from majority
  2157. func (r *Raft) confirmLeadership() bool {
  2158. r.mu.RLock()
  2159. if r.state != Leader {
  2160. r.mu.RUnlock()
  2161. return false
  2162. }
  2163. currentTerm := r.currentTerm
  2164. clusterSize := len(r.clusterNodes)
  2165. if clusterSize == 0 {
  2166. clusterSize = len(r.peers) + 1
  2167. }
  2168. r.mu.RUnlock()
  2169. // Single node cluster - we're always the leader
  2170. if clusterSize == 1 {
  2171. return true
  2172. }
  2173. // Send heartbeats and count acks
  2174. r.sendHeartbeats()
  2175. // Wait briefly and check if we're still leader
  2176. time.Sleep(r.config.HeartbeatInterval)
  2177. r.mu.RLock()
  2178. stillLeader := r.state == Leader && r.currentTerm == currentTerm
  2179. r.mu.RUnlock()
  2180. return stillLeader
  2181. }
  2182. // waitApply waits until lastApplied >= index
  2183. func (r *Raft) waitApply(index uint64, timeout time.Duration) error {
  2184. deadline := time.Now().Add(timeout)
  2185. for {
  2186. r.mu.RLock()
  2187. lastApplied := r.lastApplied
  2188. r.mu.RUnlock()
  2189. if lastApplied >= index {
  2190. return nil
  2191. }
  2192. if time.Now().After(deadline) {
  2193. return ErrTimeout
  2194. }
  2195. time.Sleep(1 * time.Millisecond)
  2196. }
  2197. }
  2198. // ==================== Health Check ====================
  2199. // HealthCheck returns the current health status of the node
  2200. func (r *Raft) HealthCheck() HealthStatus {
  2201. r.mu.RLock()
  2202. defer r.mu.RUnlock()
  2203. clusterNodes := make(map[string]string)
  2204. for k, v := range r.clusterNodes {
  2205. clusterNodes[k] = v
  2206. }
  2207. clusterSize := len(r.clusterNodes)
  2208. if clusterSize == 0 {
  2209. clusterSize = len(r.peers) + 1
  2210. }
  2211. logBehind := uint64(0)
  2212. if r.commitIndex > r.lastApplied {
  2213. logBehind = r.commitIndex - r.lastApplied
  2214. }
  2215. // Consider healthy if we're leader or have a known leader
  2216. isHealthy := r.state == Leader || r.leaderID != ""
  2217. return HealthStatus{
  2218. NodeID: r.nodeID,
  2219. State: r.state.String(),
  2220. Term: r.currentTerm,
  2221. LeaderID: r.leaderID,
  2222. ClusterSize: clusterSize,
  2223. ClusterNodes: clusterNodes,
  2224. CommitIndex: r.commitIndex,
  2225. LastApplied: r.lastApplied,
  2226. LogBehind: logBehind,
  2227. LastHeartbeat: r.lastHeartbeat,
  2228. IsHealthy: isHealthy,
  2229. Uptime: time.Since(r.startTime),
  2230. }
  2231. }
  2232. // GetMetrics returns the current metrics
  2233. func (r *Raft) GetMetrics() Metrics {
  2234. return Metrics{
  2235. Term: atomic.LoadUint64(&r.metrics.Term),
  2236. ProposalsTotal: atomic.LoadUint64(&r.metrics.ProposalsTotal),
  2237. ProposalsSuccess: atomic.LoadUint64(&r.metrics.ProposalsSuccess),
  2238. ProposalsFailed: atomic.LoadUint64(&r.metrics.ProposalsFailed),
  2239. ProposalsForwarded: atomic.LoadUint64(&r.metrics.ProposalsForwarded),
  2240. AppendsSent: atomic.LoadUint64(&r.metrics.AppendsSent),
  2241. AppendsReceived: atomic.LoadUint64(&r.metrics.AppendsReceived),
  2242. AppendsSuccess: atomic.LoadUint64(&r.metrics.AppendsSuccess),
  2243. AppendsFailed: atomic.LoadUint64(&r.metrics.AppendsFailed),
  2244. ElectionsStarted: atomic.LoadUint64(&r.metrics.ElectionsStarted),
  2245. ElectionsWon: atomic.LoadUint64(&r.metrics.ElectionsWon),
  2246. PreVotesStarted: atomic.LoadUint64(&r.metrics.PreVotesStarted),
  2247. PreVotesGranted: atomic.LoadUint64(&r.metrics.PreVotesGranted),
  2248. SnapshotsTaken: atomic.LoadUint64(&r.metrics.SnapshotsTaken),
  2249. SnapshotsInstalled: atomic.LoadUint64(&r.metrics.SnapshotsInstalled),
  2250. SnapshotsSent: atomic.LoadUint64(&r.metrics.SnapshotsSent),
  2251. ReadIndexRequests: atomic.LoadUint64(&r.metrics.ReadIndexRequests),
  2252. ReadIndexSuccess: atomic.LoadUint64(&r.metrics.ReadIndexSuccess),
  2253. LeadershipTransfers: atomic.LoadUint64(&r.metrics.LeadershipTransfers),
  2254. LeadershipTransferSuccess: atomic.LoadUint64(&r.metrics.LeadershipTransferSuccess),
  2255. }
  2256. }
  2257. // ==================== Leadership Transfer ====================
  2258. // TransferLeadership transfers leadership to the specified node
  2259. func (r *Raft) TransferLeadership(targetID string) error {
  2260. r.mu.Lock()
  2261. if r.state != Leader {
  2262. r.mu.Unlock()
  2263. return ErrNotLeader
  2264. }
  2265. if targetID == r.nodeID {
  2266. r.mu.Unlock()
  2267. return fmt.Errorf("cannot transfer to self")
  2268. }
  2269. targetAddr, exists := r.clusterNodes[targetID]
  2270. if !exists {
  2271. r.mu.Unlock()
  2272. return fmt.Errorf("target node %s not in cluster", targetID)
  2273. }
  2274. if r.transferring {
  2275. r.mu.Unlock()
  2276. return fmt.Errorf("leadership transfer already in progress")
  2277. }
  2278. r.transferring = true
  2279. r.transferTarget = targetID
  2280. r.transferDeadline = time.Now().Add(r.config.ElectionTimeoutMax * 2)
  2281. currentTerm := r.currentTerm
  2282. atomic.AddUint64(&r.metrics.LeadershipTransfers, 1)
  2283. r.mu.Unlock()
  2284. r.logger.Info("Starting leadership transfer to %s", targetID)
  2285. // Step 1: Sync target to our log
  2286. if err := r.syncFollowerToLatest(targetAddr); err != nil {
  2287. r.mu.Lock()
  2288. r.transferring = false
  2289. r.transferTarget = ""
  2290. r.mu.Unlock()
  2291. return fmt.Errorf("failed to sync target: %w", err)
  2292. }
  2293. // Step 2: Send TimeoutNow RPC
  2294. args := &TimeoutNowArgs{
  2295. Term: currentTerm,
  2296. LeaderID: r.nodeID,
  2297. }
  2298. ctx, cancel := context.WithTimeout(context.Background(), r.config.RPCTimeout)
  2299. defer cancel()
  2300. reply, err := r.transport.TimeoutNow(ctx, targetAddr, args)
  2301. if err != nil {
  2302. r.mu.Lock()
  2303. r.transferring = false
  2304. r.transferTarget = ""
  2305. r.mu.Unlock()
  2306. return fmt.Errorf("TimeoutNow RPC failed: %w", err)
  2307. }
  2308. if !reply.Success {
  2309. r.mu.Lock()
  2310. r.transferring = false
  2311. r.transferTarget = ""
  2312. r.mu.Unlock()
  2313. return fmt.Errorf("target rejected leadership transfer")
  2314. }
  2315. atomic.AddUint64(&r.metrics.LeadershipTransferSuccess, 1)
  2316. r.logger.Info("Leadership transfer to %s initiated successfully", targetID)
  2317. // Note: We don't immediately step down; we wait for the target to win election
  2318. // and send us an AppendEntries with higher term
  2319. return nil
  2320. }
  2321. // syncFollowerToLatest ensures the follower is caught up to our log
  2322. func (r *Raft) syncFollowerToLatest(peerAddr string) error {
  2323. r.mu.RLock()
  2324. if r.state != Leader {
  2325. r.mu.RUnlock()
  2326. return ErrNotLeader
  2327. }
  2328. currentTerm := r.currentTerm
  2329. leaderCommit := r.commitIndex
  2330. lastIndex := r.log.LastIndex()
  2331. r.mu.RUnlock()
  2332. // Keep replicating until follower is caught up
  2333. deadline := time.Now().Add(r.config.ElectionTimeoutMax * 2)
  2334. for time.Now().Before(deadline) {
  2335. r.mu.RLock()
  2336. if r.state != Leader || r.currentTerm != currentTerm {
  2337. r.mu.RUnlock()
  2338. return ErrLeadershipLost
  2339. }
  2340. matchIndex := r.matchIndex[peerAddr]
  2341. r.mu.RUnlock()
  2342. if matchIndex >= lastIndex {
  2343. return nil // Caught up
  2344. }
  2345. // Trigger replication
  2346. r.replicateToPeer(peerAddr, currentTerm, leaderCommit)
  2347. time.Sleep(10 * time.Millisecond)
  2348. }
  2349. return ErrTimeout
  2350. }
  2351. // HandleTimeoutNow handles TimeoutNow RPC (for leadership transfer)
  2352. func (r *Raft) HandleTimeoutNow(args *TimeoutNowArgs) *TimeoutNowReply {
  2353. r.mu.Lock()
  2354. defer r.mu.Unlock()
  2355. reply := &TimeoutNowReply{
  2356. Term: r.currentTerm,
  2357. Success: false,
  2358. }
  2359. // Only accept if we're a follower and the term matches
  2360. if args.Term < r.currentTerm {
  2361. return reply
  2362. }
  2363. if r.state != Follower {
  2364. return reply
  2365. }
  2366. // Immediately start election
  2367. r.logger.Info("Received TimeoutNow from %s, starting immediate election", args.LeaderID)
  2368. r.state = Candidate
  2369. reply.Success = true
  2370. return reply
  2371. }
  2372. // HandleReadIndex handles ReadIndex RPC
  2373. func (r *Raft) HandleReadIndex(args *ReadIndexArgs) *ReadIndexReply {
  2374. reply := &ReadIndexReply{
  2375. Success: false,
  2376. }
  2377. readIndex, err := r.ReadIndex()
  2378. if err != nil {
  2379. reply.Error = err.Error()
  2380. return reply
  2381. }
  2382. reply.ReadIndex = readIndex
  2383. reply.Success = true
  2384. return reply
  2385. }
  2386. // HandleGet handles Get RPC for remote KV reads
  2387. func (r *Raft) HandleGet(args *GetArgs) *GetReply {
  2388. reply := &GetReply{
  2389. Found: false,
  2390. }
  2391. if r.config.GetHandler == nil {
  2392. reply.Error = "get handler not configured"
  2393. return reply
  2394. }
  2395. value, found := r.config.GetHandler(args.Key)
  2396. reply.Value = value
  2397. reply.Found = found
  2398. return reply
  2399. }