raft.go 73 KB

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