raft_test.go 125 KB

1234567891011121314151617181920212223242526272829303132333435363738394041424344454647484950515253545556575859606162636465666768697071727374757677787980818283848586878889909192939495969798991001011021031041051061071081091101111121131141151161171181191201211221231241251261271281291301311321331341351361371381391401411421431441451461471481491501511521531541551561571581591601611621631641651661671681691701711721731741751761771781791801811821831841851861871881891901911921931941951961971981992002012022032042052062072082092102112122132142152162172182192202212222232242252262272282292302312322332342352362372382392402412422432442452462472482492502512522532542552562572582592602612622632642652662672682692702712722732742752762772782792802812822832842852862872882892902912922932942952962972982993003013023033043053063073083093103113123133143153163173183193203213223233243253263273283293303313323333343353363373383393403413423433443453463473483493503513523533543553563573583593603613623633643653663673683693703713723733743753763773783793803813823833843853863873883893903913923933943953963973983994004014024034044054064074084094104114124134144154164174184194204214224234244254264274284294304314324334344354364374384394404414424434444454464474484494504514524534544554564574584594604614624634644654664674684694704714724734744754764774784794804814824834844854864874884894904914924934944954964974984995005015025035045055065075085095105115125135145155165175185195205215225235245255265275285295305315325335345355365375385395405415425435445455465475485495505515525535545555565575585595605615625635645655665675685695705715725735745755765775785795805815825835845855865875885895905915925935945955965975985996006016026036046056066076086096106116126136146156166176186196206216226236246256266276286296306316326336346356366376386396406416426436446456466476486496506516526536546556566576586596606616626636646656666676686696706716726736746756766776786796806816826836846856866876886896906916926936946956966976986997007017027037047057067077087097107117127137147157167177187197207217227237247257267277287297307317327337347357367377387397407417427437447457467477487497507517527537547557567577587597607617627637647657667677687697707717727737747757767777787797807817827837847857867877887897907917927937947957967977987998008018028038048058068078088098108118128138148158168178188198208218228238248258268278288298308318328338348358368378388398408418428438448458468478488498508518528538548558568578588598608618628638648658668678688698708718728738748758768778788798808818828838848858868878888898908918928938948958968978988999009019029039049059069079089099109119129139149159169179189199209219229239249259269279289299309319329339349359369379389399409419429439449459469479489499509519529539549559569579589599609619629639649659669679689699709719729739749759769779789799809819829839849859869879889899909919929939949959969979989991000100110021003100410051006100710081009101010111012101310141015101610171018101910201021102210231024102510261027102810291030103110321033103410351036103710381039104010411042104310441045104610471048104910501051105210531054105510561057105810591060106110621063106410651066106710681069107010711072107310741075107610771078107910801081108210831084108510861087108810891090109110921093109410951096109710981099110011011102110311041105110611071108110911101111111211131114111511161117111811191120112111221123112411251126112711281129113011311132113311341135113611371138113911401141114211431144114511461147114811491150115111521153115411551156115711581159116011611162116311641165116611671168116911701171117211731174117511761177117811791180118111821183118411851186118711881189119011911192119311941195119611971198119912001201120212031204120512061207120812091210121112121213121412151216121712181219122012211222122312241225122612271228122912301231123212331234123512361237123812391240124112421243124412451246124712481249125012511252125312541255125612571258125912601261126212631264126512661267126812691270127112721273127412751276127712781279128012811282128312841285128612871288128912901291129212931294129512961297129812991300130113021303130413051306130713081309131013111312131313141315131613171318131913201321132213231324132513261327132813291330133113321333133413351336133713381339134013411342134313441345134613471348134913501351135213531354135513561357135813591360136113621363136413651366136713681369137013711372137313741375137613771378137913801381138213831384138513861387138813891390139113921393139413951396139713981399140014011402140314041405140614071408140914101411141214131414141514161417141814191420142114221423142414251426142714281429143014311432143314341435143614371438143914401441144214431444144514461447144814491450145114521453145414551456145714581459146014611462146314641465146614671468146914701471147214731474147514761477147814791480148114821483148414851486148714881489149014911492149314941495149614971498149915001501150215031504150515061507150815091510151115121513151415151516151715181519152015211522152315241525152615271528152915301531153215331534153515361537153815391540154115421543154415451546154715481549155015511552155315541555155615571558155915601561156215631564156515661567156815691570157115721573157415751576157715781579158015811582158315841585158615871588158915901591159215931594159515961597159815991600160116021603160416051606160716081609161016111612161316141615161616171618161916201621162216231624162516261627162816291630163116321633163416351636163716381639164016411642164316441645164616471648164916501651165216531654165516561657165816591660166116621663166416651666166716681669167016711672167316741675167616771678167916801681168216831684168516861687168816891690169116921693169416951696169716981699170017011702170317041705170617071708170917101711171217131714171517161717171817191720172117221723172417251726172717281729173017311732173317341735173617371738173917401741174217431744174517461747174817491750175117521753175417551756175717581759176017611762176317641765176617671768176917701771177217731774177517761777177817791780178117821783178417851786178717881789179017911792179317941795179617971798179918001801180218031804180518061807180818091810181118121813181418151816181718181819182018211822182318241825182618271828182918301831183218331834183518361837183818391840184118421843184418451846184718481849185018511852185318541855185618571858185918601861186218631864186518661867186818691870187118721873187418751876187718781879188018811882188318841885188618871888188918901891189218931894189518961897189818991900190119021903190419051906190719081909191019111912191319141915191619171918191919201921192219231924192519261927192819291930193119321933193419351936193719381939194019411942194319441945194619471948194919501951195219531954195519561957195819591960196119621963196419651966196719681969197019711972197319741975197619771978197919801981198219831984198519861987198819891990199119921993199419951996199719981999200020012002200320042005200620072008200920102011201220132014201520162017201820192020202120222023202420252026202720282029203020312032203320342035203620372038203920402041204220432044204520462047204820492050205120522053205420552056205720582059206020612062206320642065206620672068206920702071207220732074207520762077207820792080208120822083208420852086208720882089209020912092209320942095209620972098209921002101210221032104210521062107210821092110211121122113211421152116211721182119212021212122212321242125212621272128212921302131213221332134213521362137213821392140214121422143214421452146214721482149215021512152215321542155215621572158215921602161216221632164216521662167216821692170217121722173217421752176217721782179218021812182218321842185218621872188218921902191219221932194219521962197219821992200220122022203220422052206220722082209221022112212221322142215221622172218221922202221222222232224222522262227222822292230223122322233223422352236223722382239224022412242224322442245224622472248224922502251225222532254225522562257225822592260226122622263226422652266226722682269227022712272227322742275227622772278227922802281228222832284228522862287228822892290229122922293229422952296229722982299230023012302230323042305230623072308230923102311231223132314231523162317231823192320232123222323232423252326232723282329233023312332233323342335233623372338233923402341234223432344234523462347234823492350235123522353235423552356235723582359236023612362236323642365236623672368236923702371237223732374237523762377237823792380238123822383238423852386238723882389239023912392239323942395239623972398239924002401240224032404240524062407240824092410241124122413241424152416241724182419242024212422242324242425242624272428242924302431243224332434243524362437243824392440244124422443244424452446244724482449245024512452245324542455245624572458245924602461246224632464246524662467246824692470247124722473247424752476247724782479248024812482248324842485248624872488248924902491249224932494249524962497249824992500250125022503250425052506250725082509251025112512251325142515251625172518251925202521252225232524252525262527252825292530253125322533253425352536253725382539254025412542254325442545254625472548254925502551255225532554255525562557255825592560256125622563256425652566256725682569257025712572257325742575257625772578257925802581258225832584258525862587258825892590259125922593259425952596259725982599260026012602260326042605260626072608260926102611261226132614261526162617261826192620262126222623262426252626262726282629263026312632263326342635263626372638263926402641264226432644264526462647264826492650265126522653265426552656265726582659266026612662266326642665266626672668266926702671267226732674267526762677267826792680268126822683268426852686268726882689269026912692269326942695269626972698269927002701270227032704270527062707270827092710271127122713271427152716271727182719272027212722272327242725272627272728272927302731273227332734273527362737273827392740274127422743274427452746274727482749275027512752275327542755275627572758275927602761276227632764276527662767276827692770277127722773277427752776277727782779278027812782278327842785278627872788278927902791279227932794279527962797279827992800280128022803280428052806280728082809281028112812281328142815281628172818281928202821282228232824282528262827282828292830283128322833283428352836283728382839284028412842284328442845284628472848284928502851285228532854285528562857285828592860286128622863286428652866286728682869287028712872287328742875287628772878287928802881288228832884288528862887288828892890289128922893289428952896289728982899290029012902290329042905290629072908290929102911291229132914291529162917291829192920292129222923292429252926292729282929293029312932293329342935293629372938293929402941294229432944294529462947294829492950295129522953295429552956295729582959296029612962296329642965296629672968296929702971297229732974297529762977297829792980298129822983298429852986298729882989299029912992299329942995299629972998299930003001300230033004300530063007300830093010301130123013301430153016301730183019302030213022302330243025302630273028302930303031303230333034303530363037303830393040304130423043304430453046304730483049305030513052305330543055305630573058305930603061306230633064306530663067306830693070307130723073307430753076307730783079308030813082308330843085308630873088308930903091309230933094309530963097309830993100310131023103310431053106310731083109311031113112311331143115311631173118311931203121312231233124312531263127312831293130313131323133313431353136313731383139314031413142314331443145314631473148314931503151315231533154315531563157315831593160316131623163316431653166316731683169317031713172317331743175317631773178317931803181318231833184318531863187318831893190319131923193319431953196319731983199320032013202320332043205320632073208320932103211321232133214321532163217321832193220322132223223322432253226322732283229323032313232323332343235323632373238323932403241324232433244324532463247324832493250325132523253325432553256325732583259326032613262326332643265326632673268326932703271327232733274327532763277327832793280328132823283328432853286328732883289329032913292329332943295329632973298329933003301330233033304330533063307330833093310331133123313331433153316331733183319332033213322332333243325332633273328332933303331333233333334333533363337333833393340334133423343334433453346334733483349335033513352335333543355335633573358335933603361336233633364336533663367336833693370337133723373337433753376337733783379338033813382338333843385338633873388338933903391339233933394339533963397339833993400340134023403340434053406340734083409341034113412341334143415341634173418341934203421342234233424342534263427342834293430343134323433343434353436343734383439344034413442344334443445344634473448344934503451345234533454345534563457345834593460346134623463346434653466346734683469347034713472347334743475347634773478347934803481348234833484348534863487348834893490349134923493349434953496349734983499350035013502350335043505350635073508350935103511351235133514351535163517351835193520352135223523352435253526352735283529353035313532353335343535353635373538353935403541354235433544354535463547354835493550355135523553355435553556355735583559356035613562356335643565356635673568356935703571357235733574357535763577357835793580358135823583358435853586358735883589359035913592359335943595359635973598359936003601360236033604360536063607360836093610361136123613361436153616361736183619362036213622362336243625362636273628362936303631363236333634363536363637363836393640364136423643364436453646364736483649365036513652365336543655365636573658365936603661366236633664366536663667366836693670367136723673367436753676367736783679368036813682368336843685368636873688368936903691369236933694369536963697369836993700370137023703370437053706370737083709371037113712371337143715371637173718371937203721372237233724372537263727372837293730373137323733373437353736373737383739374037413742374337443745374637473748374937503751375237533754375537563757375837593760376137623763376437653766376737683769377037713772377337743775377637773778377937803781378237833784378537863787378837893790379137923793379437953796379737983799380038013802380338043805380638073808380938103811381238133814381538163817381838193820382138223823382438253826382738283829383038313832383338343835383638373838383938403841384238433844384538463847384838493850385138523853385438553856385738583859386038613862386338643865386638673868386938703871387238733874387538763877387838793880388138823883388438853886388738883889389038913892389338943895389638973898389939003901390239033904390539063907390839093910391139123913391439153916391739183919392039213922392339243925392639273928392939303931393239333934393539363937393839393940394139423943394439453946394739483949395039513952395339543955395639573958395939603961396239633964396539663967396839693970397139723973397439753976397739783979398039813982398339843985398639873988398939903991399239933994399539963997399839994000400140024003400440054006400740084009401040114012401340144015401640174018401940204021402240234024402540264027402840294030403140324033403440354036403740384039404040414042404340444045404640474048404940504051405240534054405540564057405840594060406140624063406440654066406740684069407040714072407340744075407640774078407940804081408240834084408540864087408840894090409140924093409440954096409740984099410041014102410341044105410641074108410941104111411241134114411541164117411841194120412141224123412441254126412741284129413041314132413341344135413641374138413941404141414241434144414541464147414841494150415141524153415441554156415741584159416041614162416341644165416641674168416941704171417241734174417541764177417841794180418141824183418441854186418741884189419041914192419341944195419641974198419942004201420242034204420542064207420842094210421142124213421442154216421742184219422042214222422342244225422642274228422942304231423242334234423542364237423842394240424142424243424442454246424742484249425042514252425342544255425642574258425942604261426242634264426542664267426842694270427142724273427442754276427742784279428042814282428342844285428642874288428942904291429242934294429542964297429842994300430143024303430443054306430743084309431043114312431343144315431643174318431943204321
  1. // Copyright 2015 The etcd Authors
  2. //
  3. // Licensed under the Apache License, Version 2.0 (the "License");
  4. // you may not use this file except in compliance with the License.
  5. // You may obtain a copy of the License at
  6. //
  7. // http://www.apache.org/licenses/LICENSE-2.0
  8. //
  9. // Unless required by applicable law or agreed to in writing, software
  10. // distributed under the License is distributed on an "AS IS" BASIS,
  11. // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  12. // See the License for the specific language governing permissions and
  13. // limitations under the License.
  14. package raft
  15. import (
  16. "bytes"
  17. "fmt"
  18. "math"
  19. "math/rand"
  20. "reflect"
  21. "strings"
  22. "testing"
  23. pb "go.etcd.io/etcd/raft/raftpb"
  24. )
  25. // nextEnts returns the appliable entries and updates the applied index
  26. func nextEnts(r *raft, s *MemoryStorage) (ents []pb.Entry) {
  27. // Transfer all unstable entries to "stable" storage.
  28. s.Append(r.raftLog.unstableEntries())
  29. r.raftLog.stableTo(r.raftLog.lastIndex(), r.raftLog.lastTerm())
  30. ents = r.raftLog.nextEnts()
  31. r.raftLog.appliedTo(r.raftLog.committed)
  32. return ents
  33. }
  34. func mustAppendEntry(r *raft, ents ...pb.Entry) {
  35. if !r.appendEntry(ents...) {
  36. panic("entry unexpectedly dropped")
  37. }
  38. }
  39. type stateMachine interface {
  40. Step(m pb.Message) error
  41. readMessages() []pb.Message
  42. }
  43. func (r *raft) readMessages() []pb.Message {
  44. msgs := r.msgs
  45. r.msgs = make([]pb.Message, 0)
  46. return msgs
  47. }
  48. func TestProgressBecomeProbe(t *testing.T) {
  49. match := uint64(1)
  50. tests := []struct {
  51. p *Progress
  52. wnext uint64
  53. }{
  54. {
  55. &Progress{State: ProgressStateReplicate, Match: match, Next: 5, ins: newInflights(256)},
  56. 2,
  57. },
  58. {
  59. // snapshot finish
  60. &Progress{State: ProgressStateSnapshot, Match: match, Next: 5, PendingSnapshot: 10, ins: newInflights(256)},
  61. 11,
  62. },
  63. {
  64. // snapshot failure
  65. &Progress{State: ProgressStateSnapshot, Match: match, Next: 5, PendingSnapshot: 0, ins: newInflights(256)},
  66. 2,
  67. },
  68. }
  69. for i, tt := range tests {
  70. tt.p.becomeProbe()
  71. if tt.p.State != ProgressStateProbe {
  72. t.Errorf("#%d: state = %s, want %s", i, tt.p.State, ProgressStateProbe)
  73. }
  74. if tt.p.Match != match {
  75. t.Errorf("#%d: match = %d, want %d", i, tt.p.Match, match)
  76. }
  77. if tt.p.Next != tt.wnext {
  78. t.Errorf("#%d: next = %d, want %d", i, tt.p.Next, tt.wnext)
  79. }
  80. }
  81. }
  82. func TestProgressBecomeReplicate(t *testing.T) {
  83. p := &Progress{State: ProgressStateProbe, Match: 1, Next: 5, ins: newInflights(256)}
  84. p.becomeReplicate()
  85. if p.State != ProgressStateReplicate {
  86. t.Errorf("state = %s, want %s", p.State, ProgressStateReplicate)
  87. }
  88. if p.Match != 1 {
  89. t.Errorf("match = %d, want 1", p.Match)
  90. }
  91. if w := p.Match + 1; p.Next != w {
  92. t.Errorf("next = %d, want %d", p.Next, w)
  93. }
  94. }
  95. func TestProgressBecomeSnapshot(t *testing.T) {
  96. p := &Progress{State: ProgressStateProbe, Match: 1, Next: 5, ins: newInflights(256)}
  97. p.becomeSnapshot(10)
  98. if p.State != ProgressStateSnapshot {
  99. t.Errorf("state = %s, want %s", p.State, ProgressStateSnapshot)
  100. }
  101. if p.Match != 1 {
  102. t.Errorf("match = %d, want 1", p.Match)
  103. }
  104. if p.PendingSnapshot != 10 {
  105. t.Errorf("pendingSnapshot = %d, want 10", p.PendingSnapshot)
  106. }
  107. }
  108. func TestProgressUpdate(t *testing.T) {
  109. prevM, prevN := uint64(3), uint64(5)
  110. tests := []struct {
  111. update uint64
  112. wm uint64
  113. wn uint64
  114. wok bool
  115. }{
  116. {prevM - 1, prevM, prevN, false}, // do not decrease match, next
  117. {prevM, prevM, prevN, false}, // do not decrease next
  118. {prevM + 1, prevM + 1, prevN, true}, // increase match, do not decrease next
  119. {prevM + 2, prevM + 2, prevN + 1, true}, // increase match, next
  120. }
  121. for i, tt := range tests {
  122. p := &Progress{
  123. Match: prevM,
  124. Next: prevN,
  125. }
  126. ok := p.maybeUpdate(tt.update)
  127. if ok != tt.wok {
  128. t.Errorf("#%d: ok= %v, want %v", i, ok, tt.wok)
  129. }
  130. if p.Match != tt.wm {
  131. t.Errorf("#%d: match= %d, want %d", i, p.Match, tt.wm)
  132. }
  133. if p.Next != tt.wn {
  134. t.Errorf("#%d: next= %d, want %d", i, p.Next, tt.wn)
  135. }
  136. }
  137. }
  138. func TestProgressMaybeDecr(t *testing.T) {
  139. tests := []struct {
  140. state ProgressStateType
  141. m uint64
  142. n uint64
  143. rejected uint64
  144. last uint64
  145. w bool
  146. wn uint64
  147. }{
  148. {
  149. // state replicate and rejected is not greater than match
  150. ProgressStateReplicate, 5, 10, 5, 5, false, 10,
  151. },
  152. {
  153. // state replicate and rejected is not greater than match
  154. ProgressStateReplicate, 5, 10, 4, 4, false, 10,
  155. },
  156. {
  157. // state replicate and rejected is greater than match
  158. // directly decrease to match+1
  159. ProgressStateReplicate, 5, 10, 9, 9, true, 6,
  160. },
  161. {
  162. // next-1 != rejected is always false
  163. ProgressStateProbe, 0, 0, 0, 0, false, 0,
  164. },
  165. {
  166. // next-1 != rejected is always false
  167. ProgressStateProbe, 0, 10, 5, 5, false, 10,
  168. },
  169. {
  170. // next>1 = decremented by 1
  171. ProgressStateProbe, 0, 10, 9, 9, true, 9,
  172. },
  173. {
  174. // next>1 = decremented by 1
  175. ProgressStateProbe, 0, 2, 1, 1, true, 1,
  176. },
  177. {
  178. // next<=1 = reset to 1
  179. ProgressStateProbe, 0, 1, 0, 0, true, 1,
  180. },
  181. {
  182. // decrease to min(rejected, last+1)
  183. ProgressStateProbe, 0, 10, 9, 2, true, 3,
  184. },
  185. {
  186. // rejected < 1, reset to 1
  187. ProgressStateProbe, 0, 10, 9, 0, true, 1,
  188. },
  189. }
  190. for i, tt := range tests {
  191. p := &Progress{
  192. State: tt.state,
  193. Match: tt.m,
  194. Next: tt.n,
  195. }
  196. if g := p.maybeDecrTo(tt.rejected, tt.last); g != tt.w {
  197. t.Errorf("#%d: maybeDecrTo= %t, want %t", i, g, tt.w)
  198. }
  199. if gm := p.Match; gm != tt.m {
  200. t.Errorf("#%d: match= %d, want %d", i, gm, tt.m)
  201. }
  202. if gn := p.Next; gn != tt.wn {
  203. t.Errorf("#%d: next= %d, want %d", i, gn, tt.wn)
  204. }
  205. }
  206. }
  207. func TestProgressIsPaused(t *testing.T) {
  208. tests := []struct {
  209. state ProgressStateType
  210. paused bool
  211. w bool
  212. }{
  213. {ProgressStateProbe, false, false},
  214. {ProgressStateProbe, true, true},
  215. {ProgressStateReplicate, false, false},
  216. {ProgressStateReplicate, true, false},
  217. {ProgressStateSnapshot, false, true},
  218. {ProgressStateSnapshot, true, true},
  219. }
  220. for i, tt := range tests {
  221. p := &Progress{
  222. State: tt.state,
  223. Paused: tt.paused,
  224. ins: newInflights(256),
  225. }
  226. if g := p.IsPaused(); g != tt.w {
  227. t.Errorf("#%d: paused= %t, want %t", i, g, tt.w)
  228. }
  229. }
  230. }
  231. // TestProgressResume ensures that progress.maybeUpdate and progress.maybeDecrTo
  232. // will reset progress.paused.
  233. func TestProgressResume(t *testing.T) {
  234. p := &Progress{
  235. Next: 2,
  236. Paused: true,
  237. }
  238. p.maybeDecrTo(1, 1)
  239. if p.Paused {
  240. t.Errorf("paused= %v, want false", p.Paused)
  241. }
  242. p.Paused = true
  243. p.maybeUpdate(2)
  244. if p.Paused {
  245. t.Errorf("paused= %v, want false", p.Paused)
  246. }
  247. }
  248. // TestProgressResumeByHeartbeatResp ensures raft.heartbeat reset progress.paused by heartbeat response.
  249. func TestProgressResumeByHeartbeatResp(t *testing.T) {
  250. r := newTestRaft(1, []uint64{1, 2}, 5, 1, NewMemoryStorage())
  251. r.becomeCandidate()
  252. r.becomeLeader()
  253. r.prs[2].Paused = true
  254. r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgBeat})
  255. if !r.prs[2].Paused {
  256. t.Errorf("paused = %v, want true", r.prs[2].Paused)
  257. }
  258. r.prs[2].becomeReplicate()
  259. r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgHeartbeatResp})
  260. if r.prs[2].Paused {
  261. t.Errorf("paused = %v, want false", r.prs[2].Paused)
  262. }
  263. }
  264. func TestProgressPaused(t *testing.T) {
  265. r := newTestRaft(1, []uint64{1, 2}, 5, 1, NewMemoryStorage())
  266. r.becomeCandidate()
  267. r.becomeLeader()
  268. r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
  269. r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
  270. r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
  271. ms := r.readMessages()
  272. if len(ms) != 1 {
  273. t.Errorf("len(ms) = %d, want 1", len(ms))
  274. }
  275. }
  276. func TestProgressFlowControl(t *testing.T) {
  277. cfg := newTestConfig(1, []uint64{1, 2}, 5, 1, NewMemoryStorage())
  278. cfg.MaxInflightMsgs = 3
  279. cfg.MaxSizePerMsg = 2048
  280. r := newRaft(cfg)
  281. r.becomeCandidate()
  282. r.becomeLeader()
  283. // Throw away all the messages relating to the initial election.
  284. r.readMessages()
  285. // While node 2 is in probe state, propose a bunch of entries.
  286. r.prs[2].becomeProbe()
  287. blob := []byte(strings.Repeat("a", 1000))
  288. for i := 0; i < 10; i++ {
  289. r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: blob}}})
  290. }
  291. ms := r.readMessages()
  292. // First append has two entries: the empty entry to confirm the
  293. // election, and the first proposal (only one proposal gets sent
  294. // because we're in probe state).
  295. if len(ms) != 1 || ms[0].Type != pb.MsgApp {
  296. t.Fatalf("expected 1 MsgApp, got %v", ms)
  297. }
  298. if len(ms[0].Entries) != 2 {
  299. t.Fatalf("expected 2 entries, got %d", len(ms[0].Entries))
  300. }
  301. if len(ms[0].Entries[0].Data) != 0 || len(ms[0].Entries[1].Data) != 1000 {
  302. t.Fatalf("unexpected entry sizes: %v", ms[0].Entries)
  303. }
  304. // When this append is acked, we change to replicate state and can
  305. // send multiple messages at once.
  306. r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgAppResp, Index: ms[0].Entries[1].Index})
  307. ms = r.readMessages()
  308. if len(ms) != 3 {
  309. t.Fatalf("expected 3 messages, got %d", len(ms))
  310. }
  311. for i, m := range ms {
  312. if m.Type != pb.MsgApp {
  313. t.Errorf("%d: expected MsgApp, got %s", i, m.Type)
  314. }
  315. if len(m.Entries) != 2 {
  316. t.Errorf("%d: expected 2 entries, got %d", i, len(m.Entries))
  317. }
  318. }
  319. // Ack all three of those messages together and get the last two
  320. // messages (containing three entries).
  321. r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgAppResp, Index: ms[2].Entries[1].Index})
  322. ms = r.readMessages()
  323. if len(ms) != 2 {
  324. t.Fatalf("expected 2 messages, got %d", len(ms))
  325. }
  326. for i, m := range ms {
  327. if m.Type != pb.MsgApp {
  328. t.Errorf("%d: expected MsgApp, got %s", i, m.Type)
  329. }
  330. }
  331. if len(ms[0].Entries) != 2 {
  332. t.Errorf("%d: expected 2 entries, got %d", 0, len(ms[0].Entries))
  333. }
  334. if len(ms[1].Entries) != 1 {
  335. t.Errorf("%d: expected 1 entry, got %d", 1, len(ms[1].Entries))
  336. }
  337. }
  338. func TestUncommittedEntryLimit(t *testing.T) {
  339. // Use a relatively large number of entries here to prevent regression of a
  340. // bug which computed the size before it was fixed. This test would fail
  341. // with the bug, either because we'd get dropped proposals earlier than we
  342. // expect them, or because the final tally ends up nonzero. (At the time of
  343. // writing, the former).
  344. const maxEntries = 1024
  345. testEntry := pb.Entry{Data: []byte("testdata")}
  346. maxEntrySize := maxEntries * PayloadSize(testEntry)
  347. cfg := newTestConfig(1, []uint64{1, 2, 3}, 5, 1, NewMemoryStorage())
  348. cfg.MaxUncommittedEntriesSize = uint64(maxEntrySize)
  349. cfg.MaxInflightMsgs = 2 * 1024 // avoid interference
  350. r := newRaft(cfg)
  351. r.becomeCandidate()
  352. r.becomeLeader()
  353. if n := r.uncommittedSize; n != 0 {
  354. t.Fatalf("expected zero uncommitted size, got %d bytes", n)
  355. }
  356. // Set the two followers to the replicate state. Commit to tail of log.
  357. const numFollowers = 2
  358. r.prs[2].becomeReplicate()
  359. r.prs[3].becomeReplicate()
  360. r.uncommittedSize = 0
  361. // Send proposals to r1. The first 5 entries should be appended to the log.
  362. propMsg := pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{testEntry}}
  363. propEnts := make([]pb.Entry, maxEntries)
  364. for i := 0; i < maxEntries; i++ {
  365. if err := r.Step(propMsg); err != nil {
  366. t.Fatalf("proposal resulted in error: %v", err)
  367. }
  368. propEnts[i] = testEntry
  369. }
  370. // Send one more proposal to r1. It should be rejected.
  371. if err := r.Step(propMsg); err != ErrProposalDropped {
  372. t.Fatalf("proposal not dropped: %v", err)
  373. }
  374. // Read messages and reduce the uncommitted size as if we had committed
  375. // these entries.
  376. ms := r.readMessages()
  377. if e := maxEntries * numFollowers; len(ms) != e {
  378. t.Fatalf("expected %d messages, got %d", e, len(ms))
  379. }
  380. r.reduceUncommittedSize(propEnts)
  381. if r.uncommittedSize != 0 {
  382. t.Fatalf("committed everything, but still tracking %d", r.uncommittedSize)
  383. }
  384. // Send a single large proposal to r1. Should be accepted even though it
  385. // pushes us above the limit because we were beneath it before the proposal.
  386. propEnts = make([]pb.Entry, 2*maxEntries)
  387. for i := range propEnts {
  388. propEnts[i] = testEntry
  389. }
  390. propMsgLarge := pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: propEnts}
  391. if err := r.Step(propMsgLarge); err != nil {
  392. t.Fatalf("proposal resulted in error: %v", err)
  393. }
  394. // Send one more proposal to r1. It should be rejected, again.
  395. if err := r.Step(propMsg); err != ErrProposalDropped {
  396. t.Fatalf("proposal not dropped: %v", err)
  397. }
  398. // Read messages and reduce the uncommitted size as if we had committed
  399. // these entries.
  400. ms = r.readMessages()
  401. if e := 1 * numFollowers; len(ms) != e {
  402. t.Fatalf("expected %d messages, got %d", e, len(ms))
  403. }
  404. r.reduceUncommittedSize(propEnts)
  405. if n := r.uncommittedSize; n != 0 {
  406. t.Fatalf("expected zero uncommitted size, got %d", n)
  407. }
  408. }
  409. func TestLeaderElection(t *testing.T) {
  410. testLeaderElection(t, false)
  411. }
  412. func TestLeaderElectionPreVote(t *testing.T) {
  413. testLeaderElection(t, true)
  414. }
  415. func testLeaderElection(t *testing.T, preVote bool) {
  416. var cfg func(*Config)
  417. candState := StateCandidate
  418. candTerm := uint64(1)
  419. if preVote {
  420. cfg = preVoteConfig
  421. // In pre-vote mode, an election that fails to complete
  422. // leaves the node in pre-candidate state without advancing
  423. // the term.
  424. candState = StatePreCandidate
  425. candTerm = 0
  426. }
  427. tests := []struct {
  428. *network
  429. state StateType
  430. expTerm uint64
  431. }{
  432. {newNetworkWithConfig(cfg, nil, nil, nil), StateLeader, 1},
  433. {newNetworkWithConfig(cfg, nil, nil, nopStepper), StateLeader, 1},
  434. {newNetworkWithConfig(cfg, nil, nopStepper, nopStepper), candState, candTerm},
  435. {newNetworkWithConfig(cfg, nil, nopStepper, nopStepper, nil), candState, candTerm},
  436. {newNetworkWithConfig(cfg, nil, nopStepper, nopStepper, nil, nil), StateLeader, 1},
  437. // three logs further along than 0, but in the same term so rejections
  438. // are returned instead of the votes being ignored.
  439. {newNetworkWithConfig(cfg,
  440. nil, entsWithConfig(cfg, 1), entsWithConfig(cfg, 1), entsWithConfig(cfg, 1, 1), nil),
  441. StateFollower, 1},
  442. }
  443. for i, tt := range tests {
  444. tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  445. sm := tt.network.peers[1].(*raft)
  446. if sm.state != tt.state {
  447. t.Errorf("#%d: state = %s, want %s", i, sm.state, tt.state)
  448. }
  449. if g := sm.Term; g != tt.expTerm {
  450. t.Errorf("#%d: term = %d, want %d", i, g, tt.expTerm)
  451. }
  452. }
  453. }
  454. // TestLearnerElectionTimeout verfies that the leader should not start election even
  455. // when times out.
  456. func TestLearnerElectionTimeout(t *testing.T) {
  457. n1 := newTestLearnerRaft(1, []uint64{1}, []uint64{2}, 10, 1, NewMemoryStorage())
  458. n2 := newTestLearnerRaft(2, []uint64{1}, []uint64{2}, 10, 1, NewMemoryStorage())
  459. n1.becomeFollower(1, None)
  460. n2.becomeFollower(1, None)
  461. // n2 is learner. Learner should not start election even when times out.
  462. setRandomizedElectionTimeout(n2, n2.electionTimeout)
  463. for i := 0; i < n2.electionTimeout; i++ {
  464. n2.tick()
  465. }
  466. if n2.state != StateFollower {
  467. t.Errorf("peer 2 state: %s, want %s", n2.state, StateFollower)
  468. }
  469. }
  470. // TestLearnerPromotion verifies that the learner should not election until
  471. // it is promoted to a normal peer.
  472. func TestLearnerPromotion(t *testing.T) {
  473. n1 := newTestLearnerRaft(1, []uint64{1}, []uint64{2}, 10, 1, NewMemoryStorage())
  474. n2 := newTestLearnerRaft(2, []uint64{1}, []uint64{2}, 10, 1, NewMemoryStorage())
  475. n1.becomeFollower(1, None)
  476. n2.becomeFollower(1, None)
  477. nt := newNetwork(n1, n2)
  478. if n1.state == StateLeader {
  479. t.Error("peer 1 state is leader, want not", n1.state)
  480. }
  481. // n1 should become leader
  482. setRandomizedElectionTimeout(n1, n1.electionTimeout)
  483. for i := 0; i < n1.electionTimeout; i++ {
  484. n1.tick()
  485. }
  486. if n1.state != StateLeader {
  487. t.Errorf("peer 1 state: %s, want %s", n1.state, StateLeader)
  488. }
  489. if n2.state != StateFollower {
  490. t.Errorf("peer 2 state: %s, want %s", n2.state, StateFollower)
  491. }
  492. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgBeat})
  493. n1.addNode(2)
  494. n2.addNode(2)
  495. if n2.isLearner {
  496. t.Error("peer 2 is learner, want not")
  497. }
  498. // n2 start election, should become leader
  499. setRandomizedElectionTimeout(n2, n2.electionTimeout)
  500. for i := 0; i < n2.electionTimeout; i++ {
  501. n2.tick()
  502. }
  503. nt.send(pb.Message{From: 2, To: 2, Type: pb.MsgBeat})
  504. if n1.state != StateFollower {
  505. t.Errorf("peer 1 state: %s, want %s", n1.state, StateFollower)
  506. }
  507. if n2.state != StateLeader {
  508. t.Errorf("peer 2 state: %s, want %s", n2.state, StateLeader)
  509. }
  510. }
  511. // TestLearnerCannotVote checks that a learner can't vote even it receives a valid Vote request.
  512. func TestLearnerCannotVote(t *testing.T) {
  513. n2 := newTestLearnerRaft(2, []uint64{1}, []uint64{2}, 10, 1, NewMemoryStorage())
  514. n2.becomeFollower(1, None)
  515. n2.Step(pb.Message{From: 1, To: 2, Term: 2, Type: pb.MsgVote, LogTerm: 11, Index: 11})
  516. if len(n2.msgs) != 0 {
  517. t.Errorf("expect learner not to vote, but received %v messages", n2.msgs)
  518. }
  519. }
  520. func TestLeaderCycle(t *testing.T) {
  521. testLeaderCycle(t, false)
  522. }
  523. func TestLeaderCyclePreVote(t *testing.T) {
  524. testLeaderCycle(t, true)
  525. }
  526. // testLeaderCycle verifies that each node in a cluster can campaign
  527. // and be elected in turn. This ensures that elections (including
  528. // pre-vote) work when not starting from a clean slate (as they do in
  529. // TestLeaderElection)
  530. func testLeaderCycle(t *testing.T, preVote bool) {
  531. var cfg func(*Config)
  532. if preVote {
  533. cfg = preVoteConfig
  534. }
  535. n := newNetworkWithConfig(cfg, nil, nil, nil)
  536. for campaignerID := uint64(1); campaignerID <= 3; campaignerID++ {
  537. n.send(pb.Message{From: campaignerID, To: campaignerID, Type: pb.MsgHup})
  538. for _, peer := range n.peers {
  539. sm := peer.(*raft)
  540. if sm.id == campaignerID && sm.state != StateLeader {
  541. t.Errorf("preVote=%v: campaigning node %d state = %v, want StateLeader",
  542. preVote, sm.id, sm.state)
  543. } else if sm.id != campaignerID && sm.state != StateFollower {
  544. t.Errorf("preVote=%v: after campaign of node %d, "+
  545. "node %d had state = %v, want StateFollower",
  546. preVote, campaignerID, sm.id, sm.state)
  547. }
  548. }
  549. }
  550. }
  551. // TestLeaderElectionOverwriteNewerLogs tests a scenario in which a
  552. // newly-elected leader does *not* have the newest (i.e. highest term)
  553. // log entries, and must overwrite higher-term log entries with
  554. // lower-term ones.
  555. func TestLeaderElectionOverwriteNewerLogs(t *testing.T) {
  556. testLeaderElectionOverwriteNewerLogs(t, false)
  557. }
  558. func TestLeaderElectionOverwriteNewerLogsPreVote(t *testing.T) {
  559. testLeaderElectionOverwriteNewerLogs(t, true)
  560. }
  561. func testLeaderElectionOverwriteNewerLogs(t *testing.T, preVote bool) {
  562. var cfg func(*Config)
  563. if preVote {
  564. cfg = preVoteConfig
  565. }
  566. // This network represents the results of the following sequence of
  567. // events:
  568. // - Node 1 won the election in term 1.
  569. // - Node 1 replicated a log entry to node 2 but died before sending
  570. // it to other nodes.
  571. // - Node 3 won the second election in term 2.
  572. // - Node 3 wrote an entry to its logs but died without sending it
  573. // to any other nodes.
  574. //
  575. // At this point, nodes 1, 2, and 3 all have uncommitted entries in
  576. // their logs and could win an election at term 3. The winner's log
  577. // entry overwrites the losers'. (TestLeaderSyncFollowerLog tests
  578. // the case where older log entries are overwritten, so this test
  579. // focuses on the case where the newer entries are lost).
  580. n := newNetworkWithConfig(cfg,
  581. entsWithConfig(cfg, 1), // Node 1: Won first election
  582. entsWithConfig(cfg, 1), // Node 2: Got logs from node 1
  583. entsWithConfig(cfg, 2), // Node 3: Won second election
  584. votedWithConfig(cfg, 3, 2), // Node 4: Voted but didn't get logs
  585. votedWithConfig(cfg, 3, 2)) // Node 5: Voted but didn't get logs
  586. // Node 1 campaigns. The election fails because a quorum of nodes
  587. // know about the election that already happened at term 2. Node 1's
  588. // term is pushed ahead to 2.
  589. n.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  590. sm1 := n.peers[1].(*raft)
  591. if sm1.state != StateFollower {
  592. t.Errorf("state = %s, want StateFollower", sm1.state)
  593. }
  594. if sm1.Term != 2 {
  595. t.Errorf("term = %d, want 2", sm1.Term)
  596. }
  597. // Node 1 campaigns again with a higher term. This time it succeeds.
  598. n.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  599. if sm1.state != StateLeader {
  600. t.Errorf("state = %s, want StateLeader", sm1.state)
  601. }
  602. if sm1.Term != 3 {
  603. t.Errorf("term = %d, want 3", sm1.Term)
  604. }
  605. // Now all nodes agree on a log entry with term 1 at index 1 (and
  606. // term 3 at index 2).
  607. for i := range n.peers {
  608. sm := n.peers[i].(*raft)
  609. entries := sm.raftLog.allEntries()
  610. if len(entries) != 2 {
  611. t.Fatalf("node %d: len(entries) == %d, want 2", i, len(entries))
  612. }
  613. if entries[0].Term != 1 {
  614. t.Errorf("node %d: term at index 1 == %d, want 1", i, entries[0].Term)
  615. }
  616. if entries[1].Term != 3 {
  617. t.Errorf("node %d: term at index 2 == %d, want 3", i, entries[1].Term)
  618. }
  619. }
  620. }
  621. func TestVoteFromAnyState(t *testing.T) {
  622. testVoteFromAnyState(t, pb.MsgVote)
  623. }
  624. func TestPreVoteFromAnyState(t *testing.T) {
  625. testVoteFromAnyState(t, pb.MsgPreVote)
  626. }
  627. func testVoteFromAnyState(t *testing.T, vt pb.MessageType) {
  628. for st := StateType(0); st < numStates; st++ {
  629. r := newTestRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  630. r.Term = 1
  631. switch st {
  632. case StateFollower:
  633. r.becomeFollower(r.Term, 3)
  634. case StatePreCandidate:
  635. r.becomePreCandidate()
  636. case StateCandidate:
  637. r.becomeCandidate()
  638. case StateLeader:
  639. r.becomeCandidate()
  640. r.becomeLeader()
  641. }
  642. // Note that setting our state above may have advanced r.Term
  643. // past its initial value.
  644. origTerm := r.Term
  645. newTerm := r.Term + 1
  646. msg := pb.Message{
  647. From: 2,
  648. To: 1,
  649. Type: vt,
  650. Term: newTerm,
  651. LogTerm: newTerm,
  652. Index: 42,
  653. }
  654. if err := r.Step(msg); err != nil {
  655. t.Errorf("%s,%s: Step failed: %s", vt, st, err)
  656. }
  657. if len(r.msgs) != 1 {
  658. t.Errorf("%s,%s: %d response messages, want 1: %+v", vt, st, len(r.msgs), r.msgs)
  659. } else {
  660. resp := r.msgs[0]
  661. if resp.Type != voteRespMsgType(vt) {
  662. t.Errorf("%s,%s: response message is %s, want %s",
  663. vt, st, resp.Type, voteRespMsgType(vt))
  664. }
  665. if resp.Reject {
  666. t.Errorf("%s,%s: unexpected rejection", vt, st)
  667. }
  668. }
  669. // If this was a real vote, we reset our state and term.
  670. if vt == pb.MsgVote {
  671. if r.state != StateFollower {
  672. t.Errorf("%s,%s: state %s, want %s", vt, st, r.state, StateFollower)
  673. }
  674. if r.Term != newTerm {
  675. t.Errorf("%s,%s: term %d, want %d", vt, st, r.Term, newTerm)
  676. }
  677. if r.Vote != 2 {
  678. t.Errorf("%s,%s: vote %d, want 2", vt, st, r.Vote)
  679. }
  680. } else {
  681. // In a prevote, nothing changes.
  682. if r.state != st {
  683. t.Errorf("%s,%s: state %s, want %s", vt, st, r.state, st)
  684. }
  685. if r.Term != origTerm {
  686. t.Errorf("%s,%s: term %d, want %d", vt, st, r.Term, origTerm)
  687. }
  688. // if st == StateFollower or StatePreCandidate, r hasn't voted yet.
  689. // In StateCandidate or StateLeader, it's voted for itself.
  690. if r.Vote != None && r.Vote != 1 {
  691. t.Errorf("%s,%s: vote %d, want %d or 1", vt, st, r.Vote, None)
  692. }
  693. }
  694. }
  695. }
  696. func TestLogReplication(t *testing.T) {
  697. tests := []struct {
  698. *network
  699. msgs []pb.Message
  700. wcommitted uint64
  701. }{
  702. {
  703. newNetwork(nil, nil, nil),
  704. []pb.Message{
  705. {From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}},
  706. },
  707. 2,
  708. },
  709. {
  710. newNetwork(nil, nil, nil),
  711. []pb.Message{
  712. {From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}},
  713. {From: 1, To: 2, Type: pb.MsgHup},
  714. {From: 1, To: 2, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}},
  715. },
  716. 4,
  717. },
  718. }
  719. for i, tt := range tests {
  720. tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  721. for _, m := range tt.msgs {
  722. tt.send(m)
  723. }
  724. for j, x := range tt.network.peers {
  725. sm := x.(*raft)
  726. if sm.raftLog.committed != tt.wcommitted {
  727. t.Errorf("#%d.%d: committed = %d, want %d", i, j, sm.raftLog.committed, tt.wcommitted)
  728. }
  729. ents := []pb.Entry{}
  730. for _, e := range nextEnts(sm, tt.network.storage[j]) {
  731. if e.Data != nil {
  732. ents = append(ents, e)
  733. }
  734. }
  735. props := []pb.Message{}
  736. for _, m := range tt.msgs {
  737. if m.Type == pb.MsgProp {
  738. props = append(props, m)
  739. }
  740. }
  741. for k, m := range props {
  742. if !bytes.Equal(ents[k].Data, m.Entries[0].Data) {
  743. t.Errorf("#%d.%d: data = %d, want %d", i, j, ents[k].Data, m.Entries[0].Data)
  744. }
  745. }
  746. }
  747. }
  748. }
  749. // TestLearnerLogReplication tests that a learner can receive entries from the leader.
  750. func TestLearnerLogReplication(t *testing.T) {
  751. n1 := newTestLearnerRaft(1, []uint64{1}, []uint64{2}, 10, 1, NewMemoryStorage())
  752. n2 := newTestLearnerRaft(2, []uint64{1}, []uint64{2}, 10, 1, NewMemoryStorage())
  753. nt := newNetwork(n1, n2)
  754. n1.becomeFollower(1, None)
  755. n2.becomeFollower(1, None)
  756. setRandomizedElectionTimeout(n1, n1.electionTimeout)
  757. for i := 0; i < n1.electionTimeout; i++ {
  758. n1.tick()
  759. }
  760. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgBeat})
  761. // n1 is leader and n2 is learner
  762. if n1.state != StateLeader {
  763. t.Errorf("peer 1 state: %s, want %s", n1.state, StateLeader)
  764. }
  765. if !n2.isLearner {
  766. t.Error("peer 2 state: not learner, want yes")
  767. }
  768. nextCommitted := n1.raftLog.committed + 1
  769. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
  770. if n1.raftLog.committed != nextCommitted {
  771. t.Errorf("peer 1 wants committed to %d, but still %d", nextCommitted, n1.raftLog.committed)
  772. }
  773. if n1.raftLog.committed != n2.raftLog.committed {
  774. t.Errorf("peer 2 wants committed to %d, but still %d", n1.raftLog.committed, n2.raftLog.committed)
  775. }
  776. match := n1.getProgress(2).Match
  777. if match != n2.raftLog.committed {
  778. t.Errorf("progress 2 of leader 1 wants match %d, but got %d", n2.raftLog.committed, match)
  779. }
  780. }
  781. func TestSingleNodeCommit(t *testing.T) {
  782. tt := newNetwork(nil)
  783. tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  784. tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}})
  785. tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}})
  786. sm := tt.peers[1].(*raft)
  787. if sm.raftLog.committed != 3 {
  788. t.Errorf("committed = %d, want %d", sm.raftLog.committed, 3)
  789. }
  790. }
  791. // TestCannotCommitWithoutNewTermEntry tests the entries cannot be committed
  792. // when leader changes, no new proposal comes in and ChangeTerm proposal is
  793. // filtered.
  794. func TestCannotCommitWithoutNewTermEntry(t *testing.T) {
  795. tt := newNetwork(nil, nil, nil, nil, nil)
  796. tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  797. // 0 cannot reach 2,3,4
  798. tt.cut(1, 3)
  799. tt.cut(1, 4)
  800. tt.cut(1, 5)
  801. tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}})
  802. tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}})
  803. sm := tt.peers[1].(*raft)
  804. if sm.raftLog.committed != 1 {
  805. t.Errorf("committed = %d, want %d", sm.raftLog.committed, 1)
  806. }
  807. // network recovery
  808. tt.recover()
  809. // avoid committing ChangeTerm proposal
  810. tt.ignore(pb.MsgApp)
  811. // elect 2 as the new leader with term 2
  812. tt.send(pb.Message{From: 2, To: 2, Type: pb.MsgHup})
  813. // no log entries from previous term should be committed
  814. sm = tt.peers[2].(*raft)
  815. if sm.raftLog.committed != 1 {
  816. t.Errorf("committed = %d, want %d", sm.raftLog.committed, 1)
  817. }
  818. tt.recover()
  819. // send heartbeat; reset wait
  820. tt.send(pb.Message{From: 2, To: 2, Type: pb.MsgBeat})
  821. // append an entry at current term
  822. tt.send(pb.Message{From: 2, To: 2, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}})
  823. // expect the committed to be advanced
  824. if sm.raftLog.committed != 5 {
  825. t.Errorf("committed = %d, want %d", sm.raftLog.committed, 5)
  826. }
  827. }
  828. // TestCommitWithoutNewTermEntry tests the entries could be committed
  829. // when leader changes, no new proposal comes in.
  830. func TestCommitWithoutNewTermEntry(t *testing.T) {
  831. tt := newNetwork(nil, nil, nil, nil, nil)
  832. tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  833. // 0 cannot reach 2,3,4
  834. tt.cut(1, 3)
  835. tt.cut(1, 4)
  836. tt.cut(1, 5)
  837. tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}})
  838. tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}})
  839. sm := tt.peers[1].(*raft)
  840. if sm.raftLog.committed != 1 {
  841. t.Errorf("committed = %d, want %d", sm.raftLog.committed, 1)
  842. }
  843. // network recovery
  844. tt.recover()
  845. // elect 2 as the new leader with term 2
  846. // after append a ChangeTerm entry from the current term, all entries
  847. // should be committed
  848. tt.send(pb.Message{From: 2, To: 2, Type: pb.MsgHup})
  849. if sm.raftLog.committed != 4 {
  850. t.Errorf("committed = %d, want %d", sm.raftLog.committed, 4)
  851. }
  852. }
  853. func TestDuelingCandidates(t *testing.T) {
  854. a := newTestRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  855. b := newTestRaft(2, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  856. c := newTestRaft(3, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  857. nt := newNetwork(a, b, c)
  858. nt.cut(1, 3)
  859. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  860. nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
  861. // 1 becomes leader since it receives votes from 1 and 2
  862. sm := nt.peers[1].(*raft)
  863. if sm.state != StateLeader {
  864. t.Errorf("state = %s, want %s", sm.state, StateLeader)
  865. }
  866. // 3 stays as candidate since it receives a vote from 3 and a rejection from 2
  867. sm = nt.peers[3].(*raft)
  868. if sm.state != StateCandidate {
  869. t.Errorf("state = %s, want %s", sm.state, StateCandidate)
  870. }
  871. nt.recover()
  872. // candidate 3 now increases its term and tries to vote again
  873. // we expect it to disrupt the leader 1 since it has a higher term
  874. // 3 will be follower again since both 1 and 2 rejects its vote request since 3 does not have a long enough log
  875. nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
  876. wlog := &raftLog{
  877. storage: &MemoryStorage{ents: []pb.Entry{{}, {Data: nil, Term: 1, Index: 1}}},
  878. committed: 1,
  879. unstable: unstable{offset: 2},
  880. }
  881. tests := []struct {
  882. sm *raft
  883. state StateType
  884. term uint64
  885. raftLog *raftLog
  886. }{
  887. {a, StateFollower, 2, wlog},
  888. {b, StateFollower, 2, wlog},
  889. {c, StateFollower, 2, newLog(NewMemoryStorage(), raftLogger)},
  890. }
  891. for i, tt := range tests {
  892. if g := tt.sm.state; g != tt.state {
  893. t.Errorf("#%d: state = %s, want %s", i, g, tt.state)
  894. }
  895. if g := tt.sm.Term; g != tt.term {
  896. t.Errorf("#%d: term = %d, want %d", i, g, tt.term)
  897. }
  898. base := ltoa(tt.raftLog)
  899. if sm, ok := nt.peers[1+uint64(i)].(*raft); ok {
  900. l := ltoa(sm.raftLog)
  901. if g := diffu(base, l); g != "" {
  902. t.Errorf("#%d: diff:\n%s", i, g)
  903. }
  904. } else {
  905. t.Logf("#%d: empty log", i)
  906. }
  907. }
  908. }
  909. func TestDuelingPreCandidates(t *testing.T) {
  910. cfgA := newTestConfig(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  911. cfgB := newTestConfig(2, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  912. cfgC := newTestConfig(3, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  913. cfgA.PreVote = true
  914. cfgB.PreVote = true
  915. cfgC.PreVote = true
  916. a := newRaft(cfgA)
  917. b := newRaft(cfgB)
  918. c := newRaft(cfgC)
  919. nt := newNetwork(a, b, c)
  920. nt.cut(1, 3)
  921. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  922. nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
  923. // 1 becomes leader since it receives votes from 1 and 2
  924. sm := nt.peers[1].(*raft)
  925. if sm.state != StateLeader {
  926. t.Errorf("state = %s, want %s", sm.state, StateLeader)
  927. }
  928. // 3 campaigns then reverts to follower when its PreVote is rejected
  929. sm = nt.peers[3].(*raft)
  930. if sm.state != StateFollower {
  931. t.Errorf("state = %s, want %s", sm.state, StateFollower)
  932. }
  933. nt.recover()
  934. // Candidate 3 now increases its term and tries to vote again.
  935. // With PreVote, it does not disrupt the leader.
  936. nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
  937. wlog := &raftLog{
  938. storage: &MemoryStorage{ents: []pb.Entry{{}, {Data: nil, Term: 1, Index: 1}}},
  939. committed: 1,
  940. unstable: unstable{offset: 2},
  941. }
  942. tests := []struct {
  943. sm *raft
  944. state StateType
  945. term uint64
  946. raftLog *raftLog
  947. }{
  948. {a, StateLeader, 1, wlog},
  949. {b, StateFollower, 1, wlog},
  950. {c, StateFollower, 1, newLog(NewMemoryStorage(), raftLogger)},
  951. }
  952. for i, tt := range tests {
  953. if g := tt.sm.state; g != tt.state {
  954. t.Errorf("#%d: state = %s, want %s", i, g, tt.state)
  955. }
  956. if g := tt.sm.Term; g != tt.term {
  957. t.Errorf("#%d: term = %d, want %d", i, g, tt.term)
  958. }
  959. base := ltoa(tt.raftLog)
  960. if sm, ok := nt.peers[1+uint64(i)].(*raft); ok {
  961. l := ltoa(sm.raftLog)
  962. if g := diffu(base, l); g != "" {
  963. t.Errorf("#%d: diff:\n%s", i, g)
  964. }
  965. } else {
  966. t.Logf("#%d: empty log", i)
  967. }
  968. }
  969. }
  970. func TestCandidateConcede(t *testing.T) {
  971. tt := newNetwork(nil, nil, nil)
  972. tt.isolate(1)
  973. tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  974. tt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
  975. // heal the partition
  976. tt.recover()
  977. // send heartbeat; reset wait
  978. tt.send(pb.Message{From: 3, To: 3, Type: pb.MsgBeat})
  979. data := []byte("force follower")
  980. // send a proposal to 3 to flush out a MsgApp to 1
  981. tt.send(pb.Message{From: 3, To: 3, Type: pb.MsgProp, Entries: []pb.Entry{{Data: data}}})
  982. // send heartbeat; flush out commit
  983. tt.send(pb.Message{From: 3, To: 3, Type: pb.MsgBeat})
  984. a := tt.peers[1].(*raft)
  985. if g := a.state; g != StateFollower {
  986. t.Errorf("state = %s, want %s", g, StateFollower)
  987. }
  988. if g := a.Term; g != 1 {
  989. t.Errorf("term = %d, want %d", g, 1)
  990. }
  991. wantLog := ltoa(&raftLog{
  992. storage: &MemoryStorage{
  993. ents: []pb.Entry{{}, {Data: nil, Term: 1, Index: 1}, {Term: 1, Index: 2, Data: data}},
  994. },
  995. unstable: unstable{offset: 3},
  996. committed: 2,
  997. })
  998. for i, p := range tt.peers {
  999. if sm, ok := p.(*raft); ok {
  1000. l := ltoa(sm.raftLog)
  1001. if g := diffu(wantLog, l); g != "" {
  1002. t.Errorf("#%d: diff:\n%s", i, g)
  1003. }
  1004. } else {
  1005. t.Logf("#%d: empty log", i)
  1006. }
  1007. }
  1008. }
  1009. func TestSingleNodeCandidate(t *testing.T) {
  1010. tt := newNetwork(nil)
  1011. tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  1012. sm := tt.peers[1].(*raft)
  1013. if sm.state != StateLeader {
  1014. t.Errorf("state = %d, want %d", sm.state, StateLeader)
  1015. }
  1016. }
  1017. func TestSingleNodePreCandidate(t *testing.T) {
  1018. tt := newNetworkWithConfig(preVoteConfig, nil)
  1019. tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  1020. sm := tt.peers[1].(*raft)
  1021. if sm.state != StateLeader {
  1022. t.Errorf("state = %d, want %d", sm.state, StateLeader)
  1023. }
  1024. }
  1025. func TestOldMessages(t *testing.T) {
  1026. tt := newNetwork(nil, nil, nil)
  1027. // make 0 leader @ term 3
  1028. tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  1029. tt.send(pb.Message{From: 2, To: 2, Type: pb.MsgHup})
  1030. tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  1031. // pretend we're an old leader trying to make progress; this entry is expected to be ignored.
  1032. tt.send(pb.Message{From: 2, To: 1, Type: pb.MsgApp, Term: 2, Entries: []pb.Entry{{Index: 3, Term: 2}}})
  1033. // commit a new entry
  1034. tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
  1035. ilog := &raftLog{
  1036. storage: &MemoryStorage{
  1037. ents: []pb.Entry{
  1038. {}, {Data: nil, Term: 1, Index: 1},
  1039. {Data: nil, Term: 2, Index: 2}, {Data: nil, Term: 3, Index: 3},
  1040. {Data: []byte("somedata"), Term: 3, Index: 4},
  1041. },
  1042. },
  1043. unstable: unstable{offset: 5},
  1044. committed: 4,
  1045. }
  1046. base := ltoa(ilog)
  1047. for i, p := range tt.peers {
  1048. if sm, ok := p.(*raft); ok {
  1049. l := ltoa(sm.raftLog)
  1050. if g := diffu(base, l); g != "" {
  1051. t.Errorf("#%d: diff:\n%s", i, g)
  1052. }
  1053. } else {
  1054. t.Logf("#%d: empty log", i)
  1055. }
  1056. }
  1057. }
  1058. // TestOldMessagesReply - optimization - reply with new term.
  1059. func TestProposal(t *testing.T) {
  1060. tests := []struct {
  1061. *network
  1062. success bool
  1063. }{
  1064. {newNetwork(nil, nil, nil), true},
  1065. {newNetwork(nil, nil, nopStepper), true},
  1066. {newNetwork(nil, nopStepper, nopStepper), false},
  1067. {newNetwork(nil, nopStepper, nopStepper, nil), false},
  1068. {newNetwork(nil, nopStepper, nopStepper, nil, nil), true},
  1069. }
  1070. for j, tt := range tests {
  1071. send := func(m pb.Message) {
  1072. defer func() {
  1073. // only recover if we expect it to panic (success==false)
  1074. if !tt.success {
  1075. e := recover()
  1076. if e != nil {
  1077. t.Logf("#%d: err: %s", j, e)
  1078. }
  1079. }
  1080. }()
  1081. tt.send(m)
  1082. }
  1083. data := []byte("somedata")
  1084. // promote 1 to become leader
  1085. send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  1086. send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: data}}})
  1087. wantLog := newLog(NewMemoryStorage(), raftLogger)
  1088. if tt.success {
  1089. wantLog = &raftLog{
  1090. storage: &MemoryStorage{
  1091. ents: []pb.Entry{{}, {Data: nil, Term: 1, Index: 1}, {Term: 1, Index: 2, Data: data}},
  1092. },
  1093. unstable: unstable{offset: 3},
  1094. committed: 2}
  1095. }
  1096. base := ltoa(wantLog)
  1097. for i, p := range tt.peers {
  1098. if sm, ok := p.(*raft); ok {
  1099. l := ltoa(sm.raftLog)
  1100. if g := diffu(base, l); g != "" {
  1101. t.Errorf("#%d: diff:\n%s", i, g)
  1102. }
  1103. } else {
  1104. t.Logf("#%d: empty log", i)
  1105. }
  1106. }
  1107. sm := tt.network.peers[1].(*raft)
  1108. if g := sm.Term; g != 1 {
  1109. t.Errorf("#%d: term = %d, want %d", j, g, 1)
  1110. }
  1111. }
  1112. }
  1113. func TestProposalByProxy(t *testing.T) {
  1114. data := []byte("somedata")
  1115. tests := []*network{
  1116. newNetwork(nil, nil, nil),
  1117. newNetwork(nil, nil, nopStepper),
  1118. }
  1119. for j, tt := range tests {
  1120. // promote 0 the leader
  1121. tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  1122. // propose via follower
  1123. tt.send(pb.Message{From: 2, To: 2, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
  1124. wantLog := &raftLog{
  1125. storage: &MemoryStorage{
  1126. ents: []pb.Entry{{}, {Data: nil, Term: 1, Index: 1}, {Term: 1, Data: data, Index: 2}},
  1127. },
  1128. unstable: unstable{offset: 3},
  1129. committed: 2}
  1130. base := ltoa(wantLog)
  1131. for i, p := range tt.peers {
  1132. if sm, ok := p.(*raft); ok {
  1133. l := ltoa(sm.raftLog)
  1134. if g := diffu(base, l); g != "" {
  1135. t.Errorf("#%d: diff:\n%s", i, g)
  1136. }
  1137. } else {
  1138. t.Logf("#%d: empty log", i)
  1139. }
  1140. }
  1141. sm := tt.peers[1].(*raft)
  1142. if g := sm.Term; g != 1 {
  1143. t.Errorf("#%d: term = %d, want %d", j, g, 1)
  1144. }
  1145. }
  1146. }
  1147. func TestCommit(t *testing.T) {
  1148. tests := []struct {
  1149. matches []uint64
  1150. logs []pb.Entry
  1151. smTerm uint64
  1152. w uint64
  1153. }{
  1154. // single
  1155. {[]uint64{1}, []pb.Entry{{Index: 1, Term: 1}}, 1, 1},
  1156. {[]uint64{1}, []pb.Entry{{Index: 1, Term: 1}}, 2, 0},
  1157. {[]uint64{2}, []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}}, 2, 2},
  1158. {[]uint64{1}, []pb.Entry{{Index: 1, Term: 2}}, 2, 1},
  1159. // odd
  1160. {[]uint64{2, 1, 1}, []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}}, 1, 1},
  1161. {[]uint64{2, 1, 1}, []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 1}}, 2, 0},
  1162. {[]uint64{2, 1, 2}, []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}}, 2, 2},
  1163. {[]uint64{2, 1, 2}, []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 1}}, 2, 0},
  1164. // even
  1165. {[]uint64{2, 1, 1, 1}, []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}}, 1, 1},
  1166. {[]uint64{2, 1, 1, 1}, []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 1}}, 2, 0},
  1167. {[]uint64{2, 1, 1, 2}, []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}}, 1, 1},
  1168. {[]uint64{2, 1, 1, 2}, []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 1}}, 2, 0},
  1169. {[]uint64{2, 1, 2, 2}, []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}}, 2, 2},
  1170. {[]uint64{2, 1, 2, 2}, []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 1}}, 2, 0},
  1171. }
  1172. for i, tt := range tests {
  1173. storage := NewMemoryStorage()
  1174. storage.Append(tt.logs)
  1175. storage.hardState = pb.HardState{Term: tt.smTerm}
  1176. sm := newTestRaft(1, []uint64{1}, 10, 2, storage)
  1177. for j := 0; j < len(tt.matches); j++ {
  1178. sm.setProgress(uint64(j)+1, tt.matches[j], tt.matches[j]+1, false)
  1179. }
  1180. sm.maybeCommit()
  1181. if g := sm.raftLog.committed; g != tt.w {
  1182. t.Errorf("#%d: committed = %d, want %d", i, g, tt.w)
  1183. }
  1184. }
  1185. }
  1186. func TestPastElectionTimeout(t *testing.T) {
  1187. tests := []struct {
  1188. elapse int
  1189. wprobability float64
  1190. round bool
  1191. }{
  1192. {5, 0, false},
  1193. {10, 0.1, true},
  1194. {13, 0.4, true},
  1195. {15, 0.6, true},
  1196. {18, 0.9, true},
  1197. {20, 1, false},
  1198. }
  1199. for i, tt := range tests {
  1200. sm := newTestRaft(1, []uint64{1}, 10, 1, NewMemoryStorage())
  1201. sm.electionElapsed = tt.elapse
  1202. c := 0
  1203. for j := 0; j < 10000; j++ {
  1204. sm.resetRandomizedElectionTimeout()
  1205. if sm.pastElectionTimeout() {
  1206. c++
  1207. }
  1208. }
  1209. got := float64(c) / 10000.0
  1210. if tt.round {
  1211. got = math.Floor(got*10+0.5) / 10.0
  1212. }
  1213. if got != tt.wprobability {
  1214. t.Errorf("#%d: probability = %v, want %v", i, got, tt.wprobability)
  1215. }
  1216. }
  1217. }
  1218. // ensure that the Step function ignores the message from old term and does not pass it to the
  1219. // actual stepX function.
  1220. func TestStepIgnoreOldTermMsg(t *testing.T) {
  1221. called := false
  1222. fakeStep := func(r *raft, m pb.Message) error {
  1223. called = true
  1224. return nil
  1225. }
  1226. sm := newTestRaft(1, []uint64{1}, 10, 1, NewMemoryStorage())
  1227. sm.step = fakeStep
  1228. sm.Term = 2
  1229. sm.Step(pb.Message{Type: pb.MsgApp, Term: sm.Term - 1})
  1230. if called {
  1231. t.Errorf("stepFunc called = %v , want %v", called, false)
  1232. }
  1233. }
  1234. // TestHandleMsgApp ensures:
  1235. // 1. Reply false if log doesn’t contain an entry at prevLogIndex whose term matches prevLogTerm.
  1236. // 2. If an existing entry conflicts with a new one (same index but different terms),
  1237. // delete the existing entry and all that follow it; append any new entries not already in the log.
  1238. // 3. If leaderCommit > commitIndex, set commitIndex = min(leaderCommit, index of last new entry).
  1239. func TestHandleMsgApp(t *testing.T) {
  1240. tests := []struct {
  1241. m pb.Message
  1242. wIndex uint64
  1243. wCommit uint64
  1244. wReject bool
  1245. }{
  1246. // Ensure 1
  1247. {pb.Message{Type: pb.MsgApp, Term: 2, LogTerm: 3, Index: 2, Commit: 3}, 2, 0, true}, // previous log mismatch
  1248. {pb.Message{Type: pb.MsgApp, Term: 2, LogTerm: 3, Index: 3, Commit: 3}, 2, 0, true}, // previous log non-exist
  1249. // Ensure 2
  1250. {pb.Message{Type: pb.MsgApp, Term: 2, LogTerm: 1, Index: 1, Commit: 1}, 2, 1, false},
  1251. {pb.Message{Type: pb.MsgApp, Term: 2, LogTerm: 0, Index: 0, Commit: 1, Entries: []pb.Entry{{Index: 1, Term: 2}}}, 1, 1, false},
  1252. {pb.Message{Type: pb.MsgApp, Term: 2, LogTerm: 2, Index: 2, Commit: 3, Entries: []pb.Entry{{Index: 3, Term: 2}, {Index: 4, Term: 2}}}, 4, 3, false},
  1253. {pb.Message{Type: pb.MsgApp, Term: 2, LogTerm: 2, Index: 2, Commit: 4, Entries: []pb.Entry{{Index: 3, Term: 2}}}, 3, 3, false},
  1254. {pb.Message{Type: pb.MsgApp, Term: 2, LogTerm: 1, Index: 1, Commit: 4, Entries: []pb.Entry{{Index: 2, Term: 2}}}, 2, 2, false},
  1255. // Ensure 3
  1256. {pb.Message{Type: pb.MsgApp, Term: 1, LogTerm: 1, Index: 1, Commit: 3}, 2, 1, false}, // match entry 1, commit up to last new entry 1
  1257. {pb.Message{Type: pb.MsgApp, Term: 1, LogTerm: 1, Index: 1, Commit: 3, Entries: []pb.Entry{{Index: 2, Term: 2}}}, 2, 2, false}, // match entry 1, commit up to last new entry 2
  1258. {pb.Message{Type: pb.MsgApp, Term: 2, LogTerm: 2, Index: 2, Commit: 3}, 2, 2, false}, // match entry 2, commit up to last new entry 2
  1259. {pb.Message{Type: pb.MsgApp, Term: 2, LogTerm: 2, Index: 2, Commit: 4}, 2, 2, false}, // commit up to log.last()
  1260. }
  1261. for i, tt := range tests {
  1262. storage := NewMemoryStorage()
  1263. storage.Append([]pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}})
  1264. sm := newTestRaft(1, []uint64{1}, 10, 1, storage)
  1265. sm.becomeFollower(2, None)
  1266. sm.handleAppendEntries(tt.m)
  1267. if sm.raftLog.lastIndex() != tt.wIndex {
  1268. t.Errorf("#%d: lastIndex = %d, want %d", i, sm.raftLog.lastIndex(), tt.wIndex)
  1269. }
  1270. if sm.raftLog.committed != tt.wCommit {
  1271. t.Errorf("#%d: committed = %d, want %d", i, sm.raftLog.committed, tt.wCommit)
  1272. }
  1273. m := sm.readMessages()
  1274. if len(m) != 1 {
  1275. t.Fatalf("#%d: msg = nil, want 1", i)
  1276. }
  1277. if m[0].Reject != tt.wReject {
  1278. t.Errorf("#%d: reject = %v, want %v", i, m[0].Reject, tt.wReject)
  1279. }
  1280. }
  1281. }
  1282. // TestHandleHeartbeat ensures that the follower commits to the commit in the message.
  1283. func TestHandleHeartbeat(t *testing.T) {
  1284. commit := uint64(2)
  1285. tests := []struct {
  1286. m pb.Message
  1287. wCommit uint64
  1288. }{
  1289. {pb.Message{From: 2, To: 1, Type: pb.MsgHeartbeat, Term: 2, Commit: commit + 1}, commit + 1},
  1290. {pb.Message{From: 2, To: 1, Type: pb.MsgHeartbeat, Term: 2, Commit: commit - 1}, commit}, // do not decrease commit
  1291. }
  1292. for i, tt := range tests {
  1293. storage := NewMemoryStorage()
  1294. storage.Append([]pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}})
  1295. sm := newTestRaft(1, []uint64{1, 2}, 5, 1, storage)
  1296. sm.becomeFollower(2, 2)
  1297. sm.raftLog.commitTo(commit)
  1298. sm.handleHeartbeat(tt.m)
  1299. if sm.raftLog.committed != tt.wCommit {
  1300. t.Errorf("#%d: committed = %d, want %d", i, sm.raftLog.committed, tt.wCommit)
  1301. }
  1302. m := sm.readMessages()
  1303. if len(m) != 1 {
  1304. t.Fatalf("#%d: msg = nil, want 1", i)
  1305. }
  1306. if m[0].Type != pb.MsgHeartbeatResp {
  1307. t.Errorf("#%d: type = %v, want MsgHeartbeatResp", i, m[0].Type)
  1308. }
  1309. }
  1310. }
  1311. // TestHandleHeartbeatResp ensures that we re-send log entries when we get a heartbeat response.
  1312. func TestHandleHeartbeatResp(t *testing.T) {
  1313. storage := NewMemoryStorage()
  1314. storage.Append([]pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}})
  1315. sm := newTestRaft(1, []uint64{1, 2}, 5, 1, storage)
  1316. sm.becomeCandidate()
  1317. sm.becomeLeader()
  1318. sm.raftLog.commitTo(sm.raftLog.lastIndex())
  1319. // A heartbeat response from a node that is behind; re-send MsgApp
  1320. sm.Step(pb.Message{From: 2, Type: pb.MsgHeartbeatResp})
  1321. msgs := sm.readMessages()
  1322. if len(msgs) != 1 {
  1323. t.Fatalf("len(msgs) = %d, want 1", len(msgs))
  1324. }
  1325. if msgs[0].Type != pb.MsgApp {
  1326. t.Errorf("type = %v, want MsgApp", msgs[0].Type)
  1327. }
  1328. // A second heartbeat response generates another MsgApp re-send
  1329. sm.Step(pb.Message{From: 2, Type: pb.MsgHeartbeatResp})
  1330. msgs = sm.readMessages()
  1331. if len(msgs) != 1 {
  1332. t.Fatalf("len(msgs) = %d, want 1", len(msgs))
  1333. }
  1334. if msgs[0].Type != pb.MsgApp {
  1335. t.Errorf("type = %v, want MsgApp", msgs[0].Type)
  1336. }
  1337. // Once we have an MsgAppResp, heartbeats no longer send MsgApp.
  1338. sm.Step(pb.Message{
  1339. From: 2,
  1340. Type: pb.MsgAppResp,
  1341. Index: msgs[0].Index + uint64(len(msgs[0].Entries)),
  1342. })
  1343. // Consume the message sent in response to MsgAppResp
  1344. sm.readMessages()
  1345. sm.Step(pb.Message{From: 2, Type: pb.MsgHeartbeatResp})
  1346. msgs = sm.readMessages()
  1347. if len(msgs) != 0 {
  1348. t.Fatalf("len(msgs) = %d, want 0: %+v", len(msgs), msgs)
  1349. }
  1350. }
  1351. // TestRaftFreesReadOnlyMem ensures raft will free read request from
  1352. // readOnly readIndexQueue and pendingReadIndex map.
  1353. // related issue: https://github.com/etcd-io/etcd/issues/7571
  1354. func TestRaftFreesReadOnlyMem(t *testing.T) {
  1355. sm := newTestRaft(1, []uint64{1, 2}, 5, 1, NewMemoryStorage())
  1356. sm.becomeCandidate()
  1357. sm.becomeLeader()
  1358. sm.raftLog.commitTo(sm.raftLog.lastIndex())
  1359. ctx := []byte("ctx")
  1360. // leader starts linearizable read request.
  1361. // more info: raft dissertation 6.4, step 2.
  1362. sm.Step(pb.Message{From: 2, Type: pb.MsgReadIndex, Entries: []pb.Entry{{Data: ctx}}})
  1363. msgs := sm.readMessages()
  1364. if len(msgs) != 1 {
  1365. t.Fatalf("len(msgs) = %d, want 1", len(msgs))
  1366. }
  1367. if msgs[0].Type != pb.MsgHeartbeat {
  1368. t.Fatalf("type = %v, want MsgHeartbeat", msgs[0].Type)
  1369. }
  1370. if !bytes.Equal(msgs[0].Context, ctx) {
  1371. t.Fatalf("Context = %v, want %v", msgs[0].Context, ctx)
  1372. }
  1373. if len(sm.readOnly.readIndexQueue) != 1 {
  1374. t.Fatalf("len(readIndexQueue) = %v, want 1", len(sm.readOnly.readIndexQueue))
  1375. }
  1376. if len(sm.readOnly.pendingReadIndex) != 1 {
  1377. t.Fatalf("len(pendingReadIndex) = %v, want 1", len(sm.readOnly.pendingReadIndex))
  1378. }
  1379. if _, ok := sm.readOnly.pendingReadIndex[string(ctx)]; !ok {
  1380. t.Fatalf("can't find context %v in pendingReadIndex ", ctx)
  1381. }
  1382. // heartbeat responses from majority of followers (1 in this case)
  1383. // acknowledge the authority of the leader.
  1384. // more info: raft dissertation 6.4, step 3.
  1385. sm.Step(pb.Message{From: 2, Type: pb.MsgHeartbeatResp, Context: ctx})
  1386. if len(sm.readOnly.readIndexQueue) != 0 {
  1387. t.Fatalf("len(readIndexQueue) = %v, want 0", len(sm.readOnly.readIndexQueue))
  1388. }
  1389. if len(sm.readOnly.pendingReadIndex) != 0 {
  1390. t.Fatalf("len(pendingReadIndex) = %v, want 0", len(sm.readOnly.pendingReadIndex))
  1391. }
  1392. if _, ok := sm.readOnly.pendingReadIndex[string(ctx)]; ok {
  1393. t.Fatalf("found context %v in pendingReadIndex, want none", ctx)
  1394. }
  1395. }
  1396. // TestMsgAppRespWaitReset verifies the resume behavior of a leader
  1397. // MsgAppResp.
  1398. func TestMsgAppRespWaitReset(t *testing.T) {
  1399. sm := newTestRaft(1, []uint64{1, 2, 3}, 5, 1, NewMemoryStorage())
  1400. sm.becomeCandidate()
  1401. sm.becomeLeader()
  1402. // The new leader has just emitted a new Term 4 entry; consume those messages
  1403. // from the outgoing queue.
  1404. sm.bcastAppend()
  1405. sm.readMessages()
  1406. // Node 2 acks the first entry, making it committed.
  1407. sm.Step(pb.Message{
  1408. From: 2,
  1409. Type: pb.MsgAppResp,
  1410. Index: 1,
  1411. })
  1412. if sm.raftLog.committed != 1 {
  1413. t.Fatalf("expected committed to be 1, got %d", sm.raftLog.committed)
  1414. }
  1415. // Also consume the MsgApp messages that update Commit on the followers.
  1416. sm.readMessages()
  1417. // A new command is now proposed on node 1.
  1418. sm.Step(pb.Message{
  1419. From: 1,
  1420. Type: pb.MsgProp,
  1421. Entries: []pb.Entry{{}},
  1422. })
  1423. // The command is broadcast to all nodes not in the wait state.
  1424. // Node 2 left the wait state due to its MsgAppResp, but node 3 is still waiting.
  1425. msgs := sm.readMessages()
  1426. if len(msgs) != 1 {
  1427. t.Fatalf("expected 1 message, got %d: %+v", len(msgs), msgs)
  1428. }
  1429. if msgs[0].Type != pb.MsgApp || msgs[0].To != 2 {
  1430. t.Errorf("expected MsgApp to node 2, got %v to %d", msgs[0].Type, msgs[0].To)
  1431. }
  1432. if len(msgs[0].Entries) != 1 || msgs[0].Entries[0].Index != 2 {
  1433. t.Errorf("expected to send entry 2, but got %v", msgs[0].Entries)
  1434. }
  1435. // Now Node 3 acks the first entry. This releases the wait and entry 2 is sent.
  1436. sm.Step(pb.Message{
  1437. From: 3,
  1438. Type: pb.MsgAppResp,
  1439. Index: 1,
  1440. })
  1441. msgs = sm.readMessages()
  1442. if len(msgs) != 1 {
  1443. t.Fatalf("expected 1 message, got %d: %+v", len(msgs), msgs)
  1444. }
  1445. if msgs[0].Type != pb.MsgApp || msgs[0].To != 3 {
  1446. t.Errorf("expected MsgApp to node 3, got %v to %d", msgs[0].Type, msgs[0].To)
  1447. }
  1448. if len(msgs[0].Entries) != 1 || msgs[0].Entries[0].Index != 2 {
  1449. t.Errorf("expected to send entry 2, but got %v", msgs[0].Entries)
  1450. }
  1451. }
  1452. func TestRecvMsgVote(t *testing.T) {
  1453. testRecvMsgVote(t, pb.MsgVote)
  1454. }
  1455. func TestRecvMsgPreVote(t *testing.T) {
  1456. testRecvMsgVote(t, pb.MsgPreVote)
  1457. }
  1458. func testRecvMsgVote(t *testing.T, msgType pb.MessageType) {
  1459. tests := []struct {
  1460. state StateType
  1461. index, logTerm uint64
  1462. voteFor uint64
  1463. wreject bool
  1464. }{
  1465. {StateFollower, 0, 0, None, true},
  1466. {StateFollower, 0, 1, None, true},
  1467. {StateFollower, 0, 2, None, true},
  1468. {StateFollower, 0, 3, None, false},
  1469. {StateFollower, 1, 0, None, true},
  1470. {StateFollower, 1, 1, None, true},
  1471. {StateFollower, 1, 2, None, true},
  1472. {StateFollower, 1, 3, None, false},
  1473. {StateFollower, 2, 0, None, true},
  1474. {StateFollower, 2, 1, None, true},
  1475. {StateFollower, 2, 2, None, false},
  1476. {StateFollower, 2, 3, None, false},
  1477. {StateFollower, 3, 0, None, true},
  1478. {StateFollower, 3, 1, None, true},
  1479. {StateFollower, 3, 2, None, false},
  1480. {StateFollower, 3, 3, None, false},
  1481. {StateFollower, 3, 2, 2, false},
  1482. {StateFollower, 3, 2, 1, true},
  1483. {StateLeader, 3, 3, 1, true},
  1484. {StatePreCandidate, 3, 3, 1, true},
  1485. {StateCandidate, 3, 3, 1, true},
  1486. }
  1487. max := func(a, b uint64) uint64 {
  1488. if a > b {
  1489. return a
  1490. }
  1491. return b
  1492. }
  1493. for i, tt := range tests {
  1494. sm := newTestRaft(1, []uint64{1}, 10, 1, NewMemoryStorage())
  1495. sm.state = tt.state
  1496. switch tt.state {
  1497. case StateFollower:
  1498. sm.step = stepFollower
  1499. case StateCandidate, StatePreCandidate:
  1500. sm.step = stepCandidate
  1501. case StateLeader:
  1502. sm.step = stepLeader
  1503. }
  1504. sm.Vote = tt.voteFor
  1505. sm.raftLog = &raftLog{
  1506. storage: &MemoryStorage{ents: []pb.Entry{{}, {Index: 1, Term: 2}, {Index: 2, Term: 2}}},
  1507. unstable: unstable{offset: 3},
  1508. }
  1509. // raft.Term is greater than or equal to raft.raftLog.lastTerm. In this
  1510. // test we're only testing MsgVote responses when the campaigning node
  1511. // has a different raft log compared to the recipient node.
  1512. // Additionally we're verifying behaviour when the recipient node has
  1513. // already given out its vote for its current term. We're not testing
  1514. // what the recipient node does when receiving a message with a
  1515. // different term number, so we simply initialize both term numbers to
  1516. // be the same.
  1517. term := max(sm.raftLog.lastTerm(), tt.logTerm)
  1518. sm.Term = term
  1519. sm.Step(pb.Message{Type: msgType, Term: term, From: 2, Index: tt.index, LogTerm: tt.logTerm})
  1520. msgs := sm.readMessages()
  1521. if g := len(msgs); g != 1 {
  1522. t.Fatalf("#%d: len(msgs) = %d, want 1", i, g)
  1523. continue
  1524. }
  1525. if g := msgs[0].Type; g != voteRespMsgType(msgType) {
  1526. t.Errorf("#%d, m.Type = %v, want %v", i, g, voteRespMsgType(msgType))
  1527. }
  1528. if g := msgs[0].Reject; g != tt.wreject {
  1529. t.Errorf("#%d, m.Reject = %v, want %v", i, g, tt.wreject)
  1530. }
  1531. }
  1532. }
  1533. func TestStateTransition(t *testing.T) {
  1534. tests := []struct {
  1535. from StateType
  1536. to StateType
  1537. wallow bool
  1538. wterm uint64
  1539. wlead uint64
  1540. }{
  1541. {StateFollower, StateFollower, true, 1, None},
  1542. {StateFollower, StatePreCandidate, true, 0, None},
  1543. {StateFollower, StateCandidate, true, 1, None},
  1544. {StateFollower, StateLeader, false, 0, None},
  1545. {StatePreCandidate, StateFollower, true, 0, None},
  1546. {StatePreCandidate, StatePreCandidate, true, 0, None},
  1547. {StatePreCandidate, StateCandidate, true, 1, None},
  1548. {StatePreCandidate, StateLeader, true, 0, 1},
  1549. {StateCandidate, StateFollower, true, 0, None},
  1550. {StateCandidate, StatePreCandidate, true, 0, None},
  1551. {StateCandidate, StateCandidate, true, 1, None},
  1552. {StateCandidate, StateLeader, true, 0, 1},
  1553. {StateLeader, StateFollower, true, 1, None},
  1554. {StateLeader, StatePreCandidate, false, 0, None},
  1555. {StateLeader, StateCandidate, false, 1, None},
  1556. {StateLeader, StateLeader, true, 0, 1},
  1557. }
  1558. for i, tt := range tests {
  1559. func() {
  1560. defer func() {
  1561. if r := recover(); r != nil {
  1562. if tt.wallow {
  1563. t.Errorf("%d: allow = %v, want %v", i, false, true)
  1564. }
  1565. }
  1566. }()
  1567. sm := newTestRaft(1, []uint64{1}, 10, 1, NewMemoryStorage())
  1568. sm.state = tt.from
  1569. switch tt.to {
  1570. case StateFollower:
  1571. sm.becomeFollower(tt.wterm, tt.wlead)
  1572. case StatePreCandidate:
  1573. sm.becomePreCandidate()
  1574. case StateCandidate:
  1575. sm.becomeCandidate()
  1576. case StateLeader:
  1577. sm.becomeLeader()
  1578. }
  1579. if sm.Term != tt.wterm {
  1580. t.Errorf("%d: term = %d, want %d", i, sm.Term, tt.wterm)
  1581. }
  1582. if sm.lead != tt.wlead {
  1583. t.Errorf("%d: lead = %d, want %d", i, sm.lead, tt.wlead)
  1584. }
  1585. }()
  1586. }
  1587. }
  1588. func TestAllServerStepdown(t *testing.T) {
  1589. tests := []struct {
  1590. state StateType
  1591. wstate StateType
  1592. wterm uint64
  1593. windex uint64
  1594. }{
  1595. {StateFollower, StateFollower, 3, 0},
  1596. {StatePreCandidate, StateFollower, 3, 0},
  1597. {StateCandidate, StateFollower, 3, 0},
  1598. {StateLeader, StateFollower, 3, 1},
  1599. }
  1600. tmsgTypes := [...]pb.MessageType{pb.MsgVote, pb.MsgApp}
  1601. tterm := uint64(3)
  1602. for i, tt := range tests {
  1603. sm := newTestRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  1604. switch tt.state {
  1605. case StateFollower:
  1606. sm.becomeFollower(1, None)
  1607. case StatePreCandidate:
  1608. sm.becomePreCandidate()
  1609. case StateCandidate:
  1610. sm.becomeCandidate()
  1611. case StateLeader:
  1612. sm.becomeCandidate()
  1613. sm.becomeLeader()
  1614. }
  1615. for j, msgType := range tmsgTypes {
  1616. sm.Step(pb.Message{From: 2, Type: msgType, Term: tterm, LogTerm: tterm})
  1617. if sm.state != tt.wstate {
  1618. t.Errorf("#%d.%d state = %v , want %v", i, j, sm.state, tt.wstate)
  1619. }
  1620. if sm.Term != tt.wterm {
  1621. t.Errorf("#%d.%d term = %v , want %v", i, j, sm.Term, tt.wterm)
  1622. }
  1623. if sm.raftLog.lastIndex() != tt.windex {
  1624. t.Errorf("#%d.%d index = %v , want %v", i, j, sm.raftLog.lastIndex(), tt.windex)
  1625. }
  1626. if uint64(len(sm.raftLog.allEntries())) != tt.windex {
  1627. t.Errorf("#%d.%d len(ents) = %v , want %v", i, j, len(sm.raftLog.allEntries()), tt.windex)
  1628. }
  1629. wlead := uint64(2)
  1630. if msgType == pb.MsgVote {
  1631. wlead = None
  1632. }
  1633. if sm.lead != wlead {
  1634. t.Errorf("#%d, sm.lead = %d, want %d", i, sm.lead, None)
  1635. }
  1636. }
  1637. }
  1638. }
  1639. func TestCandidateResetTermMsgHeartbeat(t *testing.T) {
  1640. testCandidateResetTerm(t, pb.MsgHeartbeat)
  1641. }
  1642. func TestCandidateResetTermMsgApp(t *testing.T) {
  1643. testCandidateResetTerm(t, pb.MsgApp)
  1644. }
  1645. // testCandidateResetTerm tests when a candidate receives a
  1646. // MsgHeartbeat or MsgApp from leader, "Step" resets the term
  1647. // with leader's and reverts back to follower.
  1648. func testCandidateResetTerm(t *testing.T, mt pb.MessageType) {
  1649. a := newTestRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  1650. b := newTestRaft(2, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  1651. c := newTestRaft(3, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  1652. nt := newNetwork(a, b, c)
  1653. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  1654. if a.state != StateLeader {
  1655. t.Errorf("state = %s, want %s", a.state, StateLeader)
  1656. }
  1657. if b.state != StateFollower {
  1658. t.Errorf("state = %s, want %s", b.state, StateFollower)
  1659. }
  1660. if c.state != StateFollower {
  1661. t.Errorf("state = %s, want %s", c.state, StateFollower)
  1662. }
  1663. // isolate 3 and increase term in rest
  1664. nt.isolate(3)
  1665. nt.send(pb.Message{From: 2, To: 2, Type: pb.MsgHup})
  1666. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  1667. if a.state != StateLeader {
  1668. t.Errorf("state = %s, want %s", a.state, StateLeader)
  1669. }
  1670. if b.state != StateFollower {
  1671. t.Errorf("state = %s, want %s", b.state, StateFollower)
  1672. }
  1673. // trigger campaign in isolated c
  1674. c.resetRandomizedElectionTimeout()
  1675. for i := 0; i < c.randomizedElectionTimeout; i++ {
  1676. c.tick()
  1677. }
  1678. if c.state != StateCandidate {
  1679. t.Errorf("state = %s, want %s", c.state, StateCandidate)
  1680. }
  1681. nt.recover()
  1682. // leader sends to isolated candidate
  1683. // and expects candidate to revert to follower
  1684. nt.send(pb.Message{From: 1, To: 3, Term: a.Term, Type: mt})
  1685. if c.state != StateFollower {
  1686. t.Errorf("state = %s, want %s", c.state, StateFollower)
  1687. }
  1688. // follower c term is reset with leader's
  1689. if a.Term != c.Term {
  1690. t.Errorf("follower term expected same term as leader's %d, got %d", a.Term, c.Term)
  1691. }
  1692. }
  1693. func TestLeaderStepdownWhenQuorumActive(t *testing.T) {
  1694. sm := newTestRaft(1, []uint64{1, 2, 3}, 5, 1, NewMemoryStorage())
  1695. sm.checkQuorum = true
  1696. sm.becomeCandidate()
  1697. sm.becomeLeader()
  1698. for i := 0; i < sm.electionTimeout+1; i++ {
  1699. sm.Step(pb.Message{From: 2, Type: pb.MsgHeartbeatResp, Term: sm.Term})
  1700. sm.tick()
  1701. }
  1702. if sm.state != StateLeader {
  1703. t.Errorf("state = %v, want %v", sm.state, StateLeader)
  1704. }
  1705. }
  1706. func TestLeaderStepdownWhenQuorumLost(t *testing.T) {
  1707. sm := newTestRaft(1, []uint64{1, 2, 3}, 5, 1, NewMemoryStorage())
  1708. sm.checkQuorum = true
  1709. sm.becomeCandidate()
  1710. sm.becomeLeader()
  1711. for i := 0; i < sm.electionTimeout+1; i++ {
  1712. sm.tick()
  1713. }
  1714. if sm.state != StateFollower {
  1715. t.Errorf("state = %v, want %v", sm.state, StateFollower)
  1716. }
  1717. }
  1718. func TestLeaderSupersedingWithCheckQuorum(t *testing.T) {
  1719. a := newTestRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  1720. b := newTestRaft(2, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  1721. c := newTestRaft(3, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  1722. a.checkQuorum = true
  1723. b.checkQuorum = true
  1724. c.checkQuorum = true
  1725. nt := newNetwork(a, b, c)
  1726. setRandomizedElectionTimeout(b, b.electionTimeout+1)
  1727. for i := 0; i < b.electionTimeout; i++ {
  1728. b.tick()
  1729. }
  1730. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  1731. if a.state != StateLeader {
  1732. t.Errorf("state = %s, want %s", a.state, StateLeader)
  1733. }
  1734. if c.state != StateFollower {
  1735. t.Errorf("state = %s, want %s", c.state, StateFollower)
  1736. }
  1737. nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
  1738. // Peer b rejected c's vote since its electionElapsed had not reached to electionTimeout
  1739. if c.state != StateCandidate {
  1740. t.Errorf("state = %s, want %s", c.state, StateCandidate)
  1741. }
  1742. // Letting b's electionElapsed reach to electionTimeout
  1743. for i := 0; i < b.electionTimeout; i++ {
  1744. b.tick()
  1745. }
  1746. nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
  1747. if c.state != StateLeader {
  1748. t.Errorf("state = %s, want %s", c.state, StateLeader)
  1749. }
  1750. }
  1751. func TestLeaderElectionWithCheckQuorum(t *testing.T) {
  1752. a := newTestRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  1753. b := newTestRaft(2, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  1754. c := newTestRaft(3, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  1755. a.checkQuorum = true
  1756. b.checkQuorum = true
  1757. c.checkQuorum = true
  1758. nt := newNetwork(a, b, c)
  1759. setRandomizedElectionTimeout(a, a.electionTimeout+1)
  1760. setRandomizedElectionTimeout(b, b.electionTimeout+2)
  1761. // Immediately after creation, votes are cast regardless of the
  1762. // election timeout.
  1763. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  1764. if a.state != StateLeader {
  1765. t.Errorf("state = %s, want %s", a.state, StateLeader)
  1766. }
  1767. if c.state != StateFollower {
  1768. t.Errorf("state = %s, want %s", c.state, StateFollower)
  1769. }
  1770. // need to reset randomizedElectionTimeout larger than electionTimeout again,
  1771. // because the value might be reset to electionTimeout since the last state changes
  1772. setRandomizedElectionTimeout(a, a.electionTimeout+1)
  1773. setRandomizedElectionTimeout(b, b.electionTimeout+2)
  1774. for i := 0; i < a.electionTimeout; i++ {
  1775. a.tick()
  1776. }
  1777. for i := 0; i < b.electionTimeout; i++ {
  1778. b.tick()
  1779. }
  1780. nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
  1781. if a.state != StateFollower {
  1782. t.Errorf("state = %s, want %s", a.state, StateFollower)
  1783. }
  1784. if c.state != StateLeader {
  1785. t.Errorf("state = %s, want %s", c.state, StateLeader)
  1786. }
  1787. }
  1788. // TestFreeStuckCandidateWithCheckQuorum ensures that a candidate with a higher term
  1789. // can disrupt the leader even if the leader still "officially" holds the lease, The
  1790. // leader is expected to step down and adopt the candidate's term
  1791. func TestFreeStuckCandidateWithCheckQuorum(t *testing.T) {
  1792. a := newTestRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  1793. b := newTestRaft(2, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  1794. c := newTestRaft(3, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  1795. a.checkQuorum = true
  1796. b.checkQuorum = true
  1797. c.checkQuorum = true
  1798. nt := newNetwork(a, b, c)
  1799. setRandomizedElectionTimeout(b, b.electionTimeout+1)
  1800. for i := 0; i < b.electionTimeout; i++ {
  1801. b.tick()
  1802. }
  1803. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  1804. nt.isolate(1)
  1805. nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
  1806. if b.state != StateFollower {
  1807. t.Errorf("state = %s, want %s", b.state, StateFollower)
  1808. }
  1809. if c.state != StateCandidate {
  1810. t.Errorf("state = %s, want %s", c.state, StateCandidate)
  1811. }
  1812. if c.Term != b.Term+1 {
  1813. t.Errorf("term = %d, want %d", c.Term, b.Term+1)
  1814. }
  1815. // Vote again for safety
  1816. nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
  1817. if b.state != StateFollower {
  1818. t.Errorf("state = %s, want %s", b.state, StateFollower)
  1819. }
  1820. if c.state != StateCandidate {
  1821. t.Errorf("state = %s, want %s", c.state, StateCandidate)
  1822. }
  1823. if c.Term != b.Term+2 {
  1824. t.Errorf("term = %d, want %d", c.Term, b.Term+2)
  1825. }
  1826. nt.recover()
  1827. nt.send(pb.Message{From: 1, To: 3, Type: pb.MsgHeartbeat, Term: a.Term})
  1828. // Disrupt the leader so that the stuck peer is freed
  1829. if a.state != StateFollower {
  1830. t.Errorf("state = %s, want %s", a.state, StateFollower)
  1831. }
  1832. if c.Term != a.Term {
  1833. t.Errorf("term = %d, want %d", c.Term, a.Term)
  1834. }
  1835. // Vote again, should become leader this time
  1836. nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
  1837. if c.state != StateLeader {
  1838. t.Errorf("peer 3 state: %s, want %s", c.state, StateLeader)
  1839. }
  1840. }
  1841. func TestNonPromotableVoterWithCheckQuorum(t *testing.T) {
  1842. a := newTestRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage())
  1843. b := newTestRaft(2, []uint64{1}, 10, 1, NewMemoryStorage())
  1844. a.checkQuorum = true
  1845. b.checkQuorum = true
  1846. nt := newNetwork(a, b)
  1847. setRandomizedElectionTimeout(b, b.electionTimeout+1)
  1848. // Need to remove 2 again to make it a non-promotable node since newNetwork overwritten some internal states
  1849. b.delProgress(2)
  1850. if b.promotable() {
  1851. t.Fatalf("promotable = %v, want false", b.promotable())
  1852. }
  1853. for i := 0; i < b.electionTimeout; i++ {
  1854. b.tick()
  1855. }
  1856. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  1857. if a.state != StateLeader {
  1858. t.Errorf("state = %s, want %s", a.state, StateLeader)
  1859. }
  1860. if b.state != StateFollower {
  1861. t.Errorf("state = %s, want %s", b.state, StateFollower)
  1862. }
  1863. if b.lead != 1 {
  1864. t.Errorf("lead = %d, want 1", b.lead)
  1865. }
  1866. }
  1867. // TestDisruptiveFollower tests isolated follower,
  1868. // with slow network incoming from leader, election times out
  1869. // to become a candidate with an increased term. Then, the
  1870. // candiate's response to late leader heartbeat forces the leader
  1871. // to step down.
  1872. func TestDisruptiveFollower(t *testing.T) {
  1873. n1 := newTestRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  1874. n2 := newTestRaft(2, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  1875. n3 := newTestRaft(3, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  1876. n1.checkQuorum = true
  1877. n2.checkQuorum = true
  1878. n3.checkQuorum = true
  1879. n1.becomeFollower(1, None)
  1880. n2.becomeFollower(1, None)
  1881. n3.becomeFollower(1, None)
  1882. nt := newNetwork(n1, n2, n3)
  1883. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  1884. // check state
  1885. // n1.state == StateLeader
  1886. // n2.state == StateFollower
  1887. // n3.state == StateFollower
  1888. if n1.state != StateLeader {
  1889. t.Fatalf("node 1 state: %s, want %s", n1.state, StateLeader)
  1890. }
  1891. if n2.state != StateFollower {
  1892. t.Fatalf("node 2 state: %s, want %s", n2.state, StateFollower)
  1893. }
  1894. if n3.state != StateFollower {
  1895. t.Fatalf("node 3 state: %s, want %s", n3.state, StateFollower)
  1896. }
  1897. // etcd server "advanceTicksForElection" on restart;
  1898. // this is to expedite campaign trigger when given larger
  1899. // election timeouts (e.g. multi-datacenter deploy)
  1900. // Or leader messages are being delayed while ticks elapse
  1901. setRandomizedElectionTimeout(n3, n3.electionTimeout+2)
  1902. for i := 0; i < n3.randomizedElectionTimeout-1; i++ {
  1903. n3.tick()
  1904. }
  1905. // ideally, before last election tick elapses,
  1906. // the follower n3 receives "pb.MsgApp" or "pb.MsgHeartbeat"
  1907. // from leader n1, and then resets its "electionElapsed"
  1908. // however, last tick may elapse before receiving any
  1909. // messages from leader, thus triggering campaign
  1910. n3.tick()
  1911. // n1 is still leader yet
  1912. // while its heartbeat to candidate n3 is being delayed
  1913. // check state
  1914. // n1.state == StateLeader
  1915. // n2.state == StateFollower
  1916. // n3.state == StateCandidate
  1917. if n1.state != StateLeader {
  1918. t.Fatalf("node 1 state: %s, want %s", n1.state, StateLeader)
  1919. }
  1920. if n2.state != StateFollower {
  1921. t.Fatalf("node 2 state: %s, want %s", n2.state, StateFollower)
  1922. }
  1923. if n3.state != StateCandidate {
  1924. t.Fatalf("node 3 state: %s, want %s", n3.state, StateCandidate)
  1925. }
  1926. // check term
  1927. // n1.Term == 2
  1928. // n2.Term == 2
  1929. // n3.Term == 3
  1930. if n1.Term != 2 {
  1931. t.Fatalf("node 1 term: %d, want %d", n1.Term, 2)
  1932. }
  1933. if n2.Term != 2 {
  1934. t.Fatalf("node 2 term: %d, want %d", n2.Term, 2)
  1935. }
  1936. if n3.Term != 3 {
  1937. t.Fatalf("node 3 term: %d, want %d", n3.Term, 3)
  1938. }
  1939. // while outgoing vote requests are still queued in n3,
  1940. // leader heartbeat finally arrives at candidate n3
  1941. // however, due to delayed network from leader, leader
  1942. // heartbeat was sent with lower term than candidate's
  1943. nt.send(pb.Message{From: 1, To: 3, Term: n1.Term, Type: pb.MsgHeartbeat})
  1944. // then candidate n3 responds with "pb.MsgAppResp" of higher term
  1945. // and leader steps down from a message with higher term
  1946. // this is to disrupt the current leader, so that candidate
  1947. // with higher term can be freed with following election
  1948. // check state
  1949. // n1.state == StateFollower
  1950. // n2.state == StateFollower
  1951. // n3.state == StateCandidate
  1952. if n1.state != StateFollower {
  1953. t.Fatalf("node 1 state: %s, want %s", n1.state, StateFollower)
  1954. }
  1955. if n2.state != StateFollower {
  1956. t.Fatalf("node 2 state: %s, want %s", n2.state, StateFollower)
  1957. }
  1958. if n3.state != StateCandidate {
  1959. t.Fatalf("node 3 state: %s, want %s", n3.state, StateCandidate)
  1960. }
  1961. // check term
  1962. // n1.Term == 3
  1963. // n2.Term == 2
  1964. // n3.Term == 3
  1965. if n1.Term != 3 {
  1966. t.Fatalf("node 1 term: %d, want %d", n1.Term, 3)
  1967. }
  1968. if n2.Term != 2 {
  1969. t.Fatalf("node 2 term: %d, want %d", n2.Term, 2)
  1970. }
  1971. if n3.Term != 3 {
  1972. t.Fatalf("node 3 term: %d, want %d", n3.Term, 3)
  1973. }
  1974. }
  1975. // TestDisruptiveFollowerPreVote tests isolated follower,
  1976. // with slow network incoming from leader, election times out
  1977. // to become a pre-candidate with less log than current leader.
  1978. // Then pre-vote phase prevents this isolated node from forcing
  1979. // current leader to step down, thus less disruptions.
  1980. func TestDisruptiveFollowerPreVote(t *testing.T) {
  1981. n1 := newTestRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  1982. n2 := newTestRaft(2, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  1983. n3 := newTestRaft(3, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  1984. n1.checkQuorum = true
  1985. n2.checkQuorum = true
  1986. n3.checkQuorum = true
  1987. n1.becomeFollower(1, None)
  1988. n2.becomeFollower(1, None)
  1989. n3.becomeFollower(1, None)
  1990. nt := newNetwork(n1, n2, n3)
  1991. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  1992. // check state
  1993. // n1.state == StateLeader
  1994. // n2.state == StateFollower
  1995. // n3.state == StateFollower
  1996. if n1.state != StateLeader {
  1997. t.Fatalf("node 1 state: %s, want %s", n1.state, StateLeader)
  1998. }
  1999. if n2.state != StateFollower {
  2000. t.Fatalf("node 2 state: %s, want %s", n2.state, StateFollower)
  2001. }
  2002. if n3.state != StateFollower {
  2003. t.Fatalf("node 3 state: %s, want %s", n3.state, StateFollower)
  2004. }
  2005. nt.isolate(3)
  2006. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
  2007. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
  2008. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
  2009. n1.preVote = true
  2010. n2.preVote = true
  2011. n3.preVote = true
  2012. nt.recover()
  2013. nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
  2014. // check state
  2015. // n1.state == StateLeader
  2016. // n2.state == StateFollower
  2017. // n3.state == StatePreCandidate
  2018. if n1.state != StateLeader {
  2019. t.Fatalf("node 1 state: %s, want %s", n1.state, StateLeader)
  2020. }
  2021. if n2.state != StateFollower {
  2022. t.Fatalf("node 2 state: %s, want %s", n2.state, StateFollower)
  2023. }
  2024. if n3.state != StatePreCandidate {
  2025. t.Fatalf("node 3 state: %s, want %s", n3.state, StatePreCandidate)
  2026. }
  2027. // check term
  2028. // n1.Term == 2
  2029. // n2.Term == 2
  2030. // n3.Term == 2
  2031. if n1.Term != 2 {
  2032. t.Fatalf("node 1 term: %d, want %d", n1.Term, 2)
  2033. }
  2034. if n2.Term != 2 {
  2035. t.Fatalf("node 2 term: %d, want %d", n2.Term, 2)
  2036. }
  2037. if n3.Term != 2 {
  2038. t.Fatalf("node 2 term: %d, want %d", n3.Term, 2)
  2039. }
  2040. // delayed leader heartbeat does not force current leader to step down
  2041. nt.send(pb.Message{From: 1, To: 3, Term: n1.Term, Type: pb.MsgHeartbeat})
  2042. if n1.state != StateLeader {
  2043. t.Fatalf("node 1 state: %s, want %s", n1.state, StateLeader)
  2044. }
  2045. }
  2046. func TestReadOnlyOptionSafe(t *testing.T) {
  2047. a := newTestRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  2048. b := newTestRaft(2, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  2049. c := newTestRaft(3, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  2050. nt := newNetwork(a, b, c)
  2051. setRandomizedElectionTimeout(b, b.electionTimeout+1)
  2052. for i := 0; i < b.electionTimeout; i++ {
  2053. b.tick()
  2054. }
  2055. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  2056. if a.state != StateLeader {
  2057. t.Fatalf("state = %s, want %s", a.state, StateLeader)
  2058. }
  2059. tests := []struct {
  2060. sm *raft
  2061. proposals int
  2062. wri uint64
  2063. wctx []byte
  2064. }{
  2065. {a, 10, 11, []byte("ctx1")},
  2066. {b, 10, 21, []byte("ctx2")},
  2067. {c, 10, 31, []byte("ctx3")},
  2068. {a, 10, 41, []byte("ctx4")},
  2069. {b, 10, 51, []byte("ctx5")},
  2070. {c, 10, 61, []byte("ctx6")},
  2071. }
  2072. for i, tt := range tests {
  2073. for j := 0; j < tt.proposals; j++ {
  2074. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}})
  2075. }
  2076. nt.send(pb.Message{From: tt.sm.id, To: tt.sm.id, Type: pb.MsgReadIndex, Entries: []pb.Entry{{Data: tt.wctx}}})
  2077. r := tt.sm
  2078. if len(r.readStates) == 0 {
  2079. t.Errorf("#%d: len(readStates) = 0, want non-zero", i)
  2080. }
  2081. rs := r.readStates[0]
  2082. if rs.Index != tt.wri {
  2083. t.Errorf("#%d: readIndex = %d, want %d", i, rs.Index, tt.wri)
  2084. }
  2085. if !bytes.Equal(rs.RequestCtx, tt.wctx) {
  2086. t.Errorf("#%d: requestCtx = %v, want %v", i, rs.RequestCtx, tt.wctx)
  2087. }
  2088. r.readStates = nil
  2089. }
  2090. }
  2091. func TestReadOnlyOptionLease(t *testing.T) {
  2092. a := newTestRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  2093. b := newTestRaft(2, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  2094. c := newTestRaft(3, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  2095. a.readOnly.option = ReadOnlyLeaseBased
  2096. b.readOnly.option = ReadOnlyLeaseBased
  2097. c.readOnly.option = ReadOnlyLeaseBased
  2098. a.checkQuorum = true
  2099. b.checkQuorum = true
  2100. c.checkQuorum = true
  2101. nt := newNetwork(a, b, c)
  2102. setRandomizedElectionTimeout(b, b.electionTimeout+1)
  2103. for i := 0; i < b.electionTimeout; i++ {
  2104. b.tick()
  2105. }
  2106. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  2107. if a.state != StateLeader {
  2108. t.Fatalf("state = %s, want %s", a.state, StateLeader)
  2109. }
  2110. tests := []struct {
  2111. sm *raft
  2112. proposals int
  2113. wri uint64
  2114. wctx []byte
  2115. }{
  2116. {a, 10, 11, []byte("ctx1")},
  2117. {b, 10, 21, []byte("ctx2")},
  2118. {c, 10, 31, []byte("ctx3")},
  2119. {a, 10, 41, []byte("ctx4")},
  2120. {b, 10, 51, []byte("ctx5")},
  2121. {c, 10, 61, []byte("ctx6")},
  2122. }
  2123. for i, tt := range tests {
  2124. for j := 0; j < tt.proposals; j++ {
  2125. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}})
  2126. }
  2127. nt.send(pb.Message{From: tt.sm.id, To: tt.sm.id, Type: pb.MsgReadIndex, Entries: []pb.Entry{{Data: tt.wctx}}})
  2128. r := tt.sm
  2129. rs := r.readStates[0]
  2130. if rs.Index != tt.wri {
  2131. t.Errorf("#%d: readIndex = %d, want %d", i, rs.Index, tt.wri)
  2132. }
  2133. if !bytes.Equal(rs.RequestCtx, tt.wctx) {
  2134. t.Errorf("#%d: requestCtx = %v, want %v", i, rs.RequestCtx, tt.wctx)
  2135. }
  2136. r.readStates = nil
  2137. }
  2138. }
  2139. // TestReadOnlyForNewLeader ensures that a leader only accepts MsgReadIndex message
  2140. // when it commits at least one log entry at it term.
  2141. func TestReadOnlyForNewLeader(t *testing.T) {
  2142. nodeConfigs := []struct {
  2143. id uint64
  2144. committed uint64
  2145. applied uint64
  2146. compactIndex uint64
  2147. }{
  2148. {1, 1, 1, 0},
  2149. {2, 2, 2, 2},
  2150. {3, 2, 2, 2},
  2151. }
  2152. peers := make([]stateMachine, 0)
  2153. for _, c := range nodeConfigs {
  2154. storage := NewMemoryStorage()
  2155. storage.Append([]pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 1}})
  2156. storage.SetHardState(pb.HardState{Term: 1, Commit: c.committed})
  2157. if c.compactIndex != 0 {
  2158. storage.Compact(c.compactIndex)
  2159. }
  2160. cfg := newTestConfig(c.id, []uint64{1, 2, 3}, 10, 1, storage)
  2161. cfg.Applied = c.applied
  2162. raft := newRaft(cfg)
  2163. peers = append(peers, raft)
  2164. }
  2165. nt := newNetwork(peers...)
  2166. // Drop MsgApp to forbid peer a to commit any log entry at its term after it becomes leader.
  2167. nt.ignore(pb.MsgApp)
  2168. // Force peer a to become leader.
  2169. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  2170. sm := nt.peers[1].(*raft)
  2171. if sm.state != StateLeader {
  2172. t.Fatalf("state = %s, want %s", sm.state, StateLeader)
  2173. }
  2174. // Ensure peer a drops read only request.
  2175. var windex uint64 = 4
  2176. wctx := []byte("ctx")
  2177. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgReadIndex, Entries: []pb.Entry{{Data: wctx}}})
  2178. if len(sm.readStates) != 0 {
  2179. t.Fatalf("len(readStates) = %d, want zero", len(sm.readStates))
  2180. }
  2181. nt.recover()
  2182. // Force peer a to commit a log entry at its term
  2183. for i := 0; i < sm.heartbeatTimeout; i++ {
  2184. sm.tick()
  2185. }
  2186. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}})
  2187. if sm.raftLog.committed != 4 {
  2188. t.Fatalf("committed = %d, want 4", sm.raftLog.committed)
  2189. }
  2190. lastLogTerm := sm.raftLog.zeroTermOnErrCompacted(sm.raftLog.term(sm.raftLog.committed))
  2191. if lastLogTerm != sm.Term {
  2192. t.Fatalf("last log term = %d, want %d", lastLogTerm, sm.Term)
  2193. }
  2194. // Ensure peer a accepts read only request after it commits a entry at its term.
  2195. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgReadIndex, Entries: []pb.Entry{{Data: wctx}}})
  2196. if len(sm.readStates) != 1 {
  2197. t.Fatalf("len(readStates) = %d, want 1", len(sm.readStates))
  2198. }
  2199. rs := sm.readStates[0]
  2200. if rs.Index != windex {
  2201. t.Fatalf("readIndex = %d, want %d", rs.Index, windex)
  2202. }
  2203. if !bytes.Equal(rs.RequestCtx, wctx) {
  2204. t.Fatalf("requestCtx = %v, want %v", rs.RequestCtx, wctx)
  2205. }
  2206. }
  2207. func TestLeaderAppResp(t *testing.T) {
  2208. // initial progress: match = 0; next = 3
  2209. tests := []struct {
  2210. index uint64
  2211. reject bool
  2212. // progress
  2213. wmatch uint64
  2214. wnext uint64
  2215. // message
  2216. wmsgNum int
  2217. windex uint64
  2218. wcommitted uint64
  2219. }{
  2220. {3, true, 0, 3, 0, 0, 0}, // stale resp; no replies
  2221. {2, true, 0, 2, 1, 1, 0}, // denied resp; leader does not commit; decrease next and send probing msg
  2222. {2, false, 2, 4, 2, 2, 2}, // accept resp; leader commits; broadcast with commit index
  2223. {0, false, 0, 3, 0, 0, 0}, // ignore heartbeat replies
  2224. }
  2225. for i, tt := range tests {
  2226. // sm term is 1 after it becomes the leader.
  2227. // thus the last log term must be 1 to be committed.
  2228. sm := newTestRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  2229. sm.raftLog = &raftLog{
  2230. storage: &MemoryStorage{ents: []pb.Entry{{}, {Index: 1, Term: 0}, {Index: 2, Term: 1}}},
  2231. unstable: unstable{offset: 3},
  2232. }
  2233. sm.becomeCandidate()
  2234. sm.becomeLeader()
  2235. sm.readMessages()
  2236. sm.Step(pb.Message{From: 2, Type: pb.MsgAppResp, Index: tt.index, Term: sm.Term, Reject: tt.reject, RejectHint: tt.index})
  2237. p := sm.prs[2]
  2238. if p.Match != tt.wmatch {
  2239. t.Errorf("#%d match = %d, want %d", i, p.Match, tt.wmatch)
  2240. }
  2241. if p.Next != tt.wnext {
  2242. t.Errorf("#%d next = %d, want %d", i, p.Next, tt.wnext)
  2243. }
  2244. msgs := sm.readMessages()
  2245. if len(msgs) != tt.wmsgNum {
  2246. t.Errorf("#%d msgNum = %d, want %d", i, len(msgs), tt.wmsgNum)
  2247. }
  2248. for j, msg := range msgs {
  2249. if msg.Index != tt.windex {
  2250. t.Errorf("#%d.%d index = %d, want %d", i, j, msg.Index, tt.windex)
  2251. }
  2252. if msg.Commit != tt.wcommitted {
  2253. t.Errorf("#%d.%d commit = %d, want %d", i, j, msg.Commit, tt.wcommitted)
  2254. }
  2255. }
  2256. }
  2257. }
  2258. // When the leader receives a heartbeat tick, it should
  2259. // send a MsgHeartbeat with m.Index = 0, m.LogTerm=0 and empty entries.
  2260. func TestBcastBeat(t *testing.T) {
  2261. offset := uint64(1000)
  2262. // make a state machine with log.offset = 1000
  2263. s := pb.Snapshot{
  2264. Metadata: pb.SnapshotMetadata{
  2265. Index: offset,
  2266. Term: 1,
  2267. ConfState: pb.ConfState{Nodes: []uint64{1, 2, 3}},
  2268. },
  2269. }
  2270. storage := NewMemoryStorage()
  2271. storage.ApplySnapshot(s)
  2272. sm := newTestRaft(1, nil, 10, 1, storage)
  2273. sm.Term = 1
  2274. sm.becomeCandidate()
  2275. sm.becomeLeader()
  2276. for i := 0; i < 10; i++ {
  2277. mustAppendEntry(sm, pb.Entry{Index: uint64(i) + 1})
  2278. }
  2279. // slow follower
  2280. sm.prs[2].Match, sm.prs[2].Next = 5, 6
  2281. // normal follower
  2282. sm.prs[3].Match, sm.prs[3].Next = sm.raftLog.lastIndex(), sm.raftLog.lastIndex()+1
  2283. sm.Step(pb.Message{Type: pb.MsgBeat})
  2284. msgs := sm.readMessages()
  2285. if len(msgs) != 2 {
  2286. t.Fatalf("len(msgs) = %v, want 2", len(msgs))
  2287. }
  2288. wantCommitMap := map[uint64]uint64{
  2289. 2: min(sm.raftLog.committed, sm.prs[2].Match),
  2290. 3: min(sm.raftLog.committed, sm.prs[3].Match),
  2291. }
  2292. for i, m := range msgs {
  2293. if m.Type != pb.MsgHeartbeat {
  2294. t.Fatalf("#%d: type = %v, want = %v", i, m.Type, pb.MsgHeartbeat)
  2295. }
  2296. if m.Index != 0 {
  2297. t.Fatalf("#%d: prevIndex = %d, want %d", i, m.Index, 0)
  2298. }
  2299. if m.LogTerm != 0 {
  2300. t.Fatalf("#%d: prevTerm = %d, want %d", i, m.LogTerm, 0)
  2301. }
  2302. if wantCommitMap[m.To] == 0 {
  2303. t.Fatalf("#%d: unexpected to %d", i, m.To)
  2304. } else {
  2305. if m.Commit != wantCommitMap[m.To] {
  2306. t.Fatalf("#%d: commit = %d, want %d", i, m.Commit, wantCommitMap[m.To])
  2307. }
  2308. delete(wantCommitMap, m.To)
  2309. }
  2310. if len(m.Entries) != 0 {
  2311. t.Fatalf("#%d: len(entries) = %d, want 0", i, len(m.Entries))
  2312. }
  2313. }
  2314. }
  2315. // tests the output of the state machine when receiving MsgBeat
  2316. func TestRecvMsgBeat(t *testing.T) {
  2317. tests := []struct {
  2318. state StateType
  2319. wMsg int
  2320. }{
  2321. {StateLeader, 2},
  2322. // candidate and follower should ignore MsgBeat
  2323. {StateCandidate, 0},
  2324. {StateFollower, 0},
  2325. }
  2326. for i, tt := range tests {
  2327. sm := newTestRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  2328. sm.raftLog = &raftLog{storage: &MemoryStorage{ents: []pb.Entry{{}, {Index: 1, Term: 0}, {Index: 2, Term: 1}}}}
  2329. sm.Term = 1
  2330. sm.state = tt.state
  2331. switch tt.state {
  2332. case StateFollower:
  2333. sm.step = stepFollower
  2334. case StateCandidate:
  2335. sm.step = stepCandidate
  2336. case StateLeader:
  2337. sm.step = stepLeader
  2338. }
  2339. sm.Step(pb.Message{From: 1, To: 1, Type: pb.MsgBeat})
  2340. msgs := sm.readMessages()
  2341. if len(msgs) != tt.wMsg {
  2342. t.Errorf("%d: len(msgs) = %d, want %d", i, len(msgs), tt.wMsg)
  2343. }
  2344. for _, m := range msgs {
  2345. if m.Type != pb.MsgHeartbeat {
  2346. t.Errorf("%d: msg.type = %v, want %v", i, m.Type, pb.MsgHeartbeat)
  2347. }
  2348. }
  2349. }
  2350. }
  2351. func TestLeaderIncreaseNext(t *testing.T) {
  2352. previousEnts := []pb.Entry{{Term: 1, Index: 1}, {Term: 1, Index: 2}, {Term: 1, Index: 3}}
  2353. tests := []struct {
  2354. // progress
  2355. state ProgressStateType
  2356. next uint64
  2357. wnext uint64
  2358. }{
  2359. // state replicate, optimistically increase next
  2360. // previous entries + noop entry + propose + 1
  2361. {ProgressStateReplicate, 2, uint64(len(previousEnts) + 1 + 1 + 1)},
  2362. // state probe, not optimistically increase next
  2363. {ProgressStateProbe, 2, 2},
  2364. }
  2365. for i, tt := range tests {
  2366. sm := newTestRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage())
  2367. sm.raftLog.append(previousEnts...)
  2368. sm.becomeCandidate()
  2369. sm.becomeLeader()
  2370. sm.prs[2].State = tt.state
  2371. sm.prs[2].Next = tt.next
  2372. sm.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
  2373. p := sm.prs[2]
  2374. if p.Next != tt.wnext {
  2375. t.Errorf("#%d next = %d, want %d", i, p.Next, tt.wnext)
  2376. }
  2377. }
  2378. }
  2379. func TestSendAppendForProgressProbe(t *testing.T) {
  2380. r := newTestRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage())
  2381. r.becomeCandidate()
  2382. r.becomeLeader()
  2383. r.readMessages()
  2384. r.prs[2].becomeProbe()
  2385. // each round is a heartbeat
  2386. for i := 0; i < 3; i++ {
  2387. if i == 0 {
  2388. // we expect that raft will only send out one msgAPP on the first
  2389. // loop. After that, the follower is paused until a heartbeat response is
  2390. // received.
  2391. mustAppendEntry(r, pb.Entry{Data: []byte("somedata")})
  2392. r.sendAppend(2)
  2393. msg := r.readMessages()
  2394. if len(msg) != 1 {
  2395. t.Errorf("len(msg) = %d, want %d", len(msg), 1)
  2396. }
  2397. if msg[0].Index != 0 {
  2398. t.Errorf("index = %d, want %d", msg[0].Index, 0)
  2399. }
  2400. }
  2401. if !r.prs[2].Paused {
  2402. t.Errorf("paused = %v, want true", r.prs[2].Paused)
  2403. }
  2404. for j := 0; j < 10; j++ {
  2405. mustAppendEntry(r, pb.Entry{Data: []byte("somedata")})
  2406. r.sendAppend(2)
  2407. if l := len(r.readMessages()); l != 0 {
  2408. t.Errorf("len(msg) = %d, want %d", l, 0)
  2409. }
  2410. }
  2411. // do a heartbeat
  2412. for j := 0; j < r.heartbeatTimeout; j++ {
  2413. r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgBeat})
  2414. }
  2415. if !r.prs[2].Paused {
  2416. t.Errorf("paused = %v, want true", r.prs[2].Paused)
  2417. }
  2418. // consume the heartbeat
  2419. msg := r.readMessages()
  2420. if len(msg) != 1 {
  2421. t.Errorf("len(msg) = %d, want %d", len(msg), 1)
  2422. }
  2423. if msg[0].Type != pb.MsgHeartbeat {
  2424. t.Errorf("type = %v, want %v", msg[0].Type, pb.MsgHeartbeat)
  2425. }
  2426. }
  2427. // a heartbeat response will allow another message to be sent
  2428. r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgHeartbeatResp})
  2429. msg := r.readMessages()
  2430. if len(msg) != 1 {
  2431. t.Errorf("len(msg) = %d, want %d", len(msg), 1)
  2432. }
  2433. if msg[0].Index != 0 {
  2434. t.Errorf("index = %d, want %d", msg[0].Index, 0)
  2435. }
  2436. if !r.prs[2].Paused {
  2437. t.Errorf("paused = %v, want true", r.prs[2].Paused)
  2438. }
  2439. }
  2440. func TestSendAppendForProgressReplicate(t *testing.T) {
  2441. r := newTestRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage())
  2442. r.becomeCandidate()
  2443. r.becomeLeader()
  2444. r.readMessages()
  2445. r.prs[2].becomeReplicate()
  2446. for i := 0; i < 10; i++ {
  2447. mustAppendEntry(r, pb.Entry{Data: []byte("somedata")})
  2448. r.sendAppend(2)
  2449. msgs := r.readMessages()
  2450. if len(msgs) != 1 {
  2451. t.Errorf("len(msg) = %d, want %d", len(msgs), 1)
  2452. }
  2453. }
  2454. }
  2455. func TestSendAppendForProgressSnapshot(t *testing.T) {
  2456. r := newTestRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage())
  2457. r.becomeCandidate()
  2458. r.becomeLeader()
  2459. r.readMessages()
  2460. r.prs[2].becomeSnapshot(10)
  2461. for i := 0; i < 10; i++ {
  2462. mustAppendEntry(r, pb.Entry{Data: []byte("somedata")})
  2463. r.sendAppend(2)
  2464. msgs := r.readMessages()
  2465. if len(msgs) != 0 {
  2466. t.Errorf("len(msg) = %d, want %d", len(msgs), 0)
  2467. }
  2468. }
  2469. }
  2470. func TestRecvMsgUnreachable(t *testing.T) {
  2471. previousEnts := []pb.Entry{{Term: 1, Index: 1}, {Term: 1, Index: 2}, {Term: 1, Index: 3}}
  2472. s := NewMemoryStorage()
  2473. s.Append(previousEnts)
  2474. r := newTestRaft(1, []uint64{1, 2}, 10, 1, s)
  2475. r.becomeCandidate()
  2476. r.becomeLeader()
  2477. r.readMessages()
  2478. // set node 2 to state replicate
  2479. r.prs[2].Match = 3
  2480. r.prs[2].becomeReplicate()
  2481. r.prs[2].optimisticUpdate(5)
  2482. r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgUnreachable})
  2483. if r.prs[2].State != ProgressStateProbe {
  2484. t.Errorf("state = %s, want %s", r.prs[2].State, ProgressStateProbe)
  2485. }
  2486. if wnext := r.prs[2].Match + 1; r.prs[2].Next != wnext {
  2487. t.Errorf("next = %d, want %d", r.prs[2].Next, wnext)
  2488. }
  2489. }
  2490. func TestRestore(t *testing.T) {
  2491. s := pb.Snapshot{
  2492. Metadata: pb.SnapshotMetadata{
  2493. Index: 11, // magic number
  2494. Term: 11, // magic number
  2495. ConfState: pb.ConfState{Nodes: []uint64{1, 2, 3}},
  2496. },
  2497. }
  2498. storage := NewMemoryStorage()
  2499. sm := newTestRaft(1, []uint64{1, 2}, 10, 1, storage)
  2500. if ok := sm.restore(s); !ok {
  2501. t.Fatal("restore fail, want succeed")
  2502. }
  2503. if sm.raftLog.lastIndex() != s.Metadata.Index {
  2504. t.Errorf("log.lastIndex = %d, want %d", sm.raftLog.lastIndex(), s.Metadata.Index)
  2505. }
  2506. if mustTerm(sm.raftLog.term(s.Metadata.Index)) != s.Metadata.Term {
  2507. t.Errorf("log.lastTerm = %d, want %d", mustTerm(sm.raftLog.term(s.Metadata.Index)), s.Metadata.Term)
  2508. }
  2509. sg := sm.nodes()
  2510. if !reflect.DeepEqual(sg, s.Metadata.ConfState.Nodes) {
  2511. t.Errorf("sm.Nodes = %+v, want %+v", sg, s.Metadata.ConfState.Nodes)
  2512. }
  2513. if ok := sm.restore(s); ok {
  2514. t.Fatal("restore succeed, want fail")
  2515. }
  2516. }
  2517. // TestRestoreWithLearner restores a snapshot which contains learners.
  2518. func TestRestoreWithLearner(t *testing.T) {
  2519. s := pb.Snapshot{
  2520. Metadata: pb.SnapshotMetadata{
  2521. Index: 11, // magic number
  2522. Term: 11, // magic number
  2523. ConfState: pb.ConfState{Nodes: []uint64{1, 2}, Learners: []uint64{3}},
  2524. },
  2525. }
  2526. storage := NewMemoryStorage()
  2527. sm := newTestLearnerRaft(3, []uint64{1, 2}, []uint64{3}, 8, 2, storage)
  2528. if ok := sm.restore(s); !ok {
  2529. t.Error("restore fail, want succeed")
  2530. }
  2531. if sm.raftLog.lastIndex() != s.Metadata.Index {
  2532. t.Errorf("log.lastIndex = %d, want %d", sm.raftLog.lastIndex(), s.Metadata.Index)
  2533. }
  2534. if mustTerm(sm.raftLog.term(s.Metadata.Index)) != s.Metadata.Term {
  2535. t.Errorf("log.lastTerm = %d, want %d", mustTerm(sm.raftLog.term(s.Metadata.Index)), s.Metadata.Term)
  2536. }
  2537. sg := sm.nodes()
  2538. if len(sg) != len(s.Metadata.ConfState.Nodes) {
  2539. t.Errorf("sm.Nodes = %+v, length not equal with %+v", sg, s.Metadata.ConfState.Nodes)
  2540. }
  2541. lns := sm.learnerNodes()
  2542. if len(lns) != len(s.Metadata.ConfState.Learners) {
  2543. t.Errorf("sm.LearnerNodes = %+v, length not equal with %+v", sg, s.Metadata.ConfState.Learners)
  2544. }
  2545. for _, n := range s.Metadata.ConfState.Nodes {
  2546. if sm.prs[n].IsLearner {
  2547. t.Errorf("sm.Node %x isLearner = %s, want %t", n, sm.prs[n], false)
  2548. }
  2549. }
  2550. for _, n := range s.Metadata.ConfState.Learners {
  2551. if !sm.learnerPrs[n].IsLearner {
  2552. t.Errorf("sm.Node %x isLearner = %s, want %t", n, sm.prs[n], true)
  2553. }
  2554. }
  2555. if ok := sm.restore(s); ok {
  2556. t.Error("restore succeed, want fail")
  2557. }
  2558. }
  2559. // TestRestoreInvalidLearner verfies that a normal peer can't become learner again
  2560. // when restores snapshot.
  2561. func TestRestoreInvalidLearner(t *testing.T) {
  2562. s := pb.Snapshot{
  2563. Metadata: pb.SnapshotMetadata{
  2564. Index: 11, // magic number
  2565. Term: 11, // magic number
  2566. ConfState: pb.ConfState{Nodes: []uint64{1, 2}, Learners: []uint64{3}},
  2567. },
  2568. }
  2569. storage := NewMemoryStorage()
  2570. sm := newTestRaft(3, []uint64{1, 2, 3}, 10, 1, storage)
  2571. if sm.isLearner {
  2572. t.Errorf("%x is learner, want not", sm.id)
  2573. }
  2574. if ok := sm.restore(s); ok {
  2575. t.Error("restore succeed, want fail")
  2576. }
  2577. }
  2578. // TestRestoreLearnerPromotion checks that a learner can become to a follower after
  2579. // restoring snapshot.
  2580. func TestRestoreLearnerPromotion(t *testing.T) {
  2581. s := pb.Snapshot{
  2582. Metadata: pb.SnapshotMetadata{
  2583. Index: 11, // magic number
  2584. Term: 11, // magic number
  2585. ConfState: pb.ConfState{Nodes: []uint64{1, 2, 3}},
  2586. },
  2587. }
  2588. storage := NewMemoryStorage()
  2589. sm := newTestLearnerRaft(3, []uint64{1, 2}, []uint64{3}, 10, 1, storage)
  2590. if !sm.isLearner {
  2591. t.Errorf("%x is not learner, want yes", sm.id)
  2592. }
  2593. if ok := sm.restore(s); !ok {
  2594. t.Error("restore fail, want succeed")
  2595. }
  2596. if sm.isLearner {
  2597. t.Errorf("%x is learner, want not", sm.id)
  2598. }
  2599. }
  2600. // TestLearnerReceiveSnapshot tests that a learner can receive a snpahost from leader
  2601. func TestLearnerReceiveSnapshot(t *testing.T) {
  2602. // restore the state machine from a snapshot so it has a compacted log and a snapshot
  2603. s := pb.Snapshot{
  2604. Metadata: pb.SnapshotMetadata{
  2605. Index: 11, // magic number
  2606. Term: 11, // magic number
  2607. ConfState: pb.ConfState{Nodes: []uint64{1}, Learners: []uint64{2}},
  2608. },
  2609. }
  2610. n1 := newTestLearnerRaft(1, []uint64{1}, []uint64{2}, 10, 1, NewMemoryStorage())
  2611. n2 := newTestLearnerRaft(2, []uint64{1}, []uint64{2}, 10, 1, NewMemoryStorage())
  2612. n1.restore(s)
  2613. // Force set n1 appplied index.
  2614. n1.raftLog.appliedTo(n1.raftLog.committed)
  2615. nt := newNetwork(n1, n2)
  2616. setRandomizedElectionTimeout(n1, n1.electionTimeout)
  2617. for i := 0; i < n1.electionTimeout; i++ {
  2618. n1.tick()
  2619. }
  2620. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgBeat})
  2621. if n2.raftLog.committed != n1.raftLog.committed {
  2622. t.Errorf("peer 2 must commit to %d, but %d", n1.raftLog.committed, n2.raftLog.committed)
  2623. }
  2624. }
  2625. func TestRestoreIgnoreSnapshot(t *testing.T) {
  2626. previousEnts := []pb.Entry{{Term: 1, Index: 1}, {Term: 1, Index: 2}, {Term: 1, Index: 3}}
  2627. commit := uint64(1)
  2628. storage := NewMemoryStorage()
  2629. sm := newTestRaft(1, []uint64{1, 2}, 10, 1, storage)
  2630. sm.raftLog.append(previousEnts...)
  2631. sm.raftLog.commitTo(commit)
  2632. s := pb.Snapshot{
  2633. Metadata: pb.SnapshotMetadata{
  2634. Index: commit,
  2635. Term: 1,
  2636. ConfState: pb.ConfState{Nodes: []uint64{1, 2}},
  2637. },
  2638. }
  2639. // ignore snapshot
  2640. if ok := sm.restore(s); ok {
  2641. t.Errorf("restore = %t, want %t", ok, false)
  2642. }
  2643. if sm.raftLog.committed != commit {
  2644. t.Errorf("commit = %d, want %d", sm.raftLog.committed, commit)
  2645. }
  2646. // ignore snapshot and fast forward commit
  2647. s.Metadata.Index = commit + 1
  2648. if ok := sm.restore(s); ok {
  2649. t.Errorf("restore = %t, want %t", ok, false)
  2650. }
  2651. if sm.raftLog.committed != commit+1 {
  2652. t.Errorf("commit = %d, want %d", sm.raftLog.committed, commit+1)
  2653. }
  2654. }
  2655. func TestProvideSnap(t *testing.T) {
  2656. // restore the state machine from a snapshot so it has a compacted log and a snapshot
  2657. s := pb.Snapshot{
  2658. Metadata: pb.SnapshotMetadata{
  2659. Index: 11, // magic number
  2660. Term: 11, // magic number
  2661. ConfState: pb.ConfState{Nodes: []uint64{1, 2}},
  2662. },
  2663. }
  2664. storage := NewMemoryStorage()
  2665. sm := newTestRaft(1, []uint64{1}, 10, 1, storage)
  2666. sm.restore(s)
  2667. sm.becomeCandidate()
  2668. sm.becomeLeader()
  2669. // force set the next of node 2, so that node 2 needs a snapshot
  2670. sm.prs[2].Next = sm.raftLog.firstIndex()
  2671. sm.Step(pb.Message{From: 2, To: 1, Type: pb.MsgAppResp, Index: sm.prs[2].Next - 1, Reject: true})
  2672. msgs := sm.readMessages()
  2673. if len(msgs) != 1 {
  2674. t.Fatalf("len(msgs) = %d, want 1", len(msgs))
  2675. }
  2676. m := msgs[0]
  2677. if m.Type != pb.MsgSnap {
  2678. t.Errorf("m.Type = %v, want %v", m.Type, pb.MsgSnap)
  2679. }
  2680. }
  2681. func TestIgnoreProvidingSnap(t *testing.T) {
  2682. // restore the state machine from a snapshot so it has a compacted log and a snapshot
  2683. s := pb.Snapshot{
  2684. Metadata: pb.SnapshotMetadata{
  2685. Index: 11, // magic number
  2686. Term: 11, // magic number
  2687. ConfState: pb.ConfState{Nodes: []uint64{1, 2}},
  2688. },
  2689. }
  2690. storage := NewMemoryStorage()
  2691. sm := newTestRaft(1, []uint64{1}, 10, 1, storage)
  2692. sm.restore(s)
  2693. sm.becomeCandidate()
  2694. sm.becomeLeader()
  2695. // force set the next of node 2, so that node 2 needs a snapshot
  2696. // change node 2 to be inactive, expect node 1 ignore sending snapshot to 2
  2697. sm.prs[2].Next = sm.raftLog.firstIndex() - 1
  2698. sm.prs[2].RecentActive = false
  2699. sm.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
  2700. msgs := sm.readMessages()
  2701. if len(msgs) != 0 {
  2702. t.Errorf("len(msgs) = %d, want 0", len(msgs))
  2703. }
  2704. }
  2705. func TestRestoreFromSnapMsg(t *testing.T) {
  2706. s := pb.Snapshot{
  2707. Metadata: pb.SnapshotMetadata{
  2708. Index: 11, // magic number
  2709. Term: 11, // magic number
  2710. ConfState: pb.ConfState{Nodes: []uint64{1, 2}},
  2711. },
  2712. }
  2713. m := pb.Message{Type: pb.MsgSnap, From: 1, Term: 2, Snapshot: s}
  2714. sm := newTestRaft(2, []uint64{1, 2}, 10, 1, NewMemoryStorage())
  2715. sm.Step(m)
  2716. if sm.lead != uint64(1) {
  2717. t.Errorf("sm.lead = %d, want 1", sm.lead)
  2718. }
  2719. // TODO(bdarnell): what should this test?
  2720. }
  2721. func TestSlowNodeRestore(t *testing.T) {
  2722. nt := newNetwork(nil, nil, nil)
  2723. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  2724. nt.isolate(3)
  2725. for j := 0; j <= 100; j++ {
  2726. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}})
  2727. }
  2728. lead := nt.peers[1].(*raft)
  2729. nextEnts(lead, nt.storage[1])
  2730. nt.storage[1].CreateSnapshot(lead.raftLog.applied, &pb.ConfState{Nodes: lead.nodes()}, nil)
  2731. nt.storage[1].Compact(lead.raftLog.applied)
  2732. nt.recover()
  2733. // send heartbeats so that the leader can learn everyone is active.
  2734. // node 3 will only be considered as active when node 1 receives a reply from it.
  2735. for {
  2736. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgBeat})
  2737. if lead.prs[3].RecentActive {
  2738. break
  2739. }
  2740. }
  2741. // trigger a snapshot
  2742. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}})
  2743. follower := nt.peers[3].(*raft)
  2744. // trigger a commit
  2745. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}})
  2746. if follower.raftLog.committed != lead.raftLog.committed {
  2747. t.Errorf("follower.committed = %d, want %d", follower.raftLog.committed, lead.raftLog.committed)
  2748. }
  2749. }
  2750. // TestStepConfig tests that when raft step msgProp in EntryConfChange type,
  2751. // it appends the entry to log and sets pendingConf to be true.
  2752. func TestStepConfig(t *testing.T) {
  2753. // a raft that cannot make progress
  2754. r := newTestRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage())
  2755. r.becomeCandidate()
  2756. r.becomeLeader()
  2757. index := r.raftLog.lastIndex()
  2758. r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Type: pb.EntryConfChange}}})
  2759. if g := r.raftLog.lastIndex(); g != index+1 {
  2760. t.Errorf("index = %d, want %d", g, index+1)
  2761. }
  2762. if r.pendingConfIndex != index+1 {
  2763. t.Errorf("pendingConfIndex = %d, want %d", r.pendingConfIndex, index+1)
  2764. }
  2765. }
  2766. // TestStepIgnoreConfig tests that if raft step the second msgProp in
  2767. // EntryConfChange type when the first one is uncommitted, the node will set
  2768. // the proposal to noop and keep its original state.
  2769. func TestStepIgnoreConfig(t *testing.T) {
  2770. // a raft that cannot make progress
  2771. r := newTestRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage())
  2772. r.becomeCandidate()
  2773. r.becomeLeader()
  2774. r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Type: pb.EntryConfChange}}})
  2775. index := r.raftLog.lastIndex()
  2776. pendingConfIndex := r.pendingConfIndex
  2777. r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Type: pb.EntryConfChange}}})
  2778. wents := []pb.Entry{{Type: pb.EntryNormal, Term: 1, Index: 3, Data: nil}}
  2779. ents, err := r.raftLog.entries(index+1, noLimit)
  2780. if err != nil {
  2781. t.Fatalf("unexpected error %v", err)
  2782. }
  2783. if !reflect.DeepEqual(ents, wents) {
  2784. t.Errorf("ents = %+v, want %+v", ents, wents)
  2785. }
  2786. if r.pendingConfIndex != pendingConfIndex {
  2787. t.Errorf("pendingConfIndex = %d, want %d", r.pendingConfIndex, pendingConfIndex)
  2788. }
  2789. }
  2790. // TestNewLeaderPendingConfig tests that new leader sets its pendingConfigIndex
  2791. // based on uncommitted entries.
  2792. func TestNewLeaderPendingConfig(t *testing.T) {
  2793. tests := []struct {
  2794. addEntry bool
  2795. wpendingIndex uint64
  2796. }{
  2797. {false, 0},
  2798. {true, 1},
  2799. }
  2800. for i, tt := range tests {
  2801. r := newTestRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage())
  2802. if tt.addEntry {
  2803. mustAppendEntry(r, pb.Entry{Type: pb.EntryNormal})
  2804. }
  2805. r.becomeCandidate()
  2806. r.becomeLeader()
  2807. if r.pendingConfIndex != tt.wpendingIndex {
  2808. t.Errorf("#%d: pendingConfIndex = %d, want %d",
  2809. i, r.pendingConfIndex, tt.wpendingIndex)
  2810. }
  2811. }
  2812. }
  2813. // TestAddNode tests that addNode could update nodes correctly.
  2814. func TestAddNode(t *testing.T) {
  2815. r := newTestRaft(1, []uint64{1}, 10, 1, NewMemoryStorage())
  2816. r.addNode(2)
  2817. nodes := r.nodes()
  2818. wnodes := []uint64{1, 2}
  2819. if !reflect.DeepEqual(nodes, wnodes) {
  2820. t.Errorf("nodes = %v, want %v", nodes, wnodes)
  2821. }
  2822. }
  2823. // TestAddLearner tests that addLearner could update nodes correctly.
  2824. func TestAddLearner(t *testing.T) {
  2825. r := newTestRaft(1, []uint64{1}, 10, 1, NewMemoryStorage())
  2826. r.addLearner(2)
  2827. nodes := r.learnerNodes()
  2828. wnodes := []uint64{2}
  2829. if !reflect.DeepEqual(nodes, wnodes) {
  2830. t.Errorf("nodes = %v, want %v", nodes, wnodes)
  2831. }
  2832. if !r.learnerPrs[2].IsLearner {
  2833. t.Errorf("node 2 is learner %t, want %t", r.prs[2].IsLearner, true)
  2834. }
  2835. }
  2836. // TestAddNodeCheckQuorum tests that addNode does not trigger a leader election
  2837. // immediately when checkQuorum is set.
  2838. func TestAddNodeCheckQuorum(t *testing.T) {
  2839. r := newTestRaft(1, []uint64{1}, 10, 1, NewMemoryStorage())
  2840. r.checkQuorum = true
  2841. r.becomeCandidate()
  2842. r.becomeLeader()
  2843. for i := 0; i < r.electionTimeout-1; i++ {
  2844. r.tick()
  2845. }
  2846. r.addNode(2)
  2847. // This tick will reach electionTimeout, which triggers a quorum check.
  2848. r.tick()
  2849. // Node 1 should still be the leader after a single tick.
  2850. if r.state != StateLeader {
  2851. t.Errorf("state = %v, want %v", r.state, StateLeader)
  2852. }
  2853. // After another electionTimeout ticks without hearing from node 2,
  2854. // node 1 should step down.
  2855. for i := 0; i < r.electionTimeout; i++ {
  2856. r.tick()
  2857. }
  2858. if r.state != StateFollower {
  2859. t.Errorf("state = %v, want %v", r.state, StateFollower)
  2860. }
  2861. }
  2862. // TestRemoveNode tests that removeNode could update nodes and
  2863. // and removed list correctly.
  2864. func TestRemoveNode(t *testing.T) {
  2865. r := newTestRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage())
  2866. r.removeNode(2)
  2867. w := []uint64{1}
  2868. if g := r.nodes(); !reflect.DeepEqual(g, w) {
  2869. t.Errorf("nodes = %v, want %v", g, w)
  2870. }
  2871. // remove all nodes from cluster
  2872. r.removeNode(1)
  2873. w = []uint64{}
  2874. if g := r.nodes(); !reflect.DeepEqual(g, w) {
  2875. t.Errorf("nodes = %v, want %v", g, w)
  2876. }
  2877. }
  2878. // TestRemoveLearner tests that removeNode could update nodes and
  2879. // and removed list correctly.
  2880. func TestRemoveLearner(t *testing.T) {
  2881. r := newTestLearnerRaft(1, []uint64{1}, []uint64{2}, 10, 1, NewMemoryStorage())
  2882. r.removeNode(2)
  2883. w := []uint64{1}
  2884. if g := r.nodes(); !reflect.DeepEqual(g, w) {
  2885. t.Errorf("nodes = %v, want %v", g, w)
  2886. }
  2887. w = []uint64{}
  2888. if g := r.learnerNodes(); !reflect.DeepEqual(g, w) {
  2889. t.Errorf("nodes = %v, want %v", g, w)
  2890. }
  2891. // remove all nodes from cluster
  2892. r.removeNode(1)
  2893. if g := r.nodes(); !reflect.DeepEqual(g, w) {
  2894. t.Errorf("nodes = %v, want %v", g, w)
  2895. }
  2896. }
  2897. func TestPromotable(t *testing.T) {
  2898. id := uint64(1)
  2899. tests := []struct {
  2900. peers []uint64
  2901. wp bool
  2902. }{
  2903. {[]uint64{1}, true},
  2904. {[]uint64{1, 2, 3}, true},
  2905. {[]uint64{}, false},
  2906. {[]uint64{2, 3}, false},
  2907. }
  2908. for i, tt := range tests {
  2909. r := newTestRaft(id, tt.peers, 5, 1, NewMemoryStorage())
  2910. if g := r.promotable(); g != tt.wp {
  2911. t.Errorf("#%d: promotable = %v, want %v", i, g, tt.wp)
  2912. }
  2913. }
  2914. }
  2915. func TestRaftNodes(t *testing.T) {
  2916. tests := []struct {
  2917. ids []uint64
  2918. wids []uint64
  2919. }{
  2920. {
  2921. []uint64{1, 2, 3},
  2922. []uint64{1, 2, 3},
  2923. },
  2924. {
  2925. []uint64{3, 2, 1},
  2926. []uint64{1, 2, 3},
  2927. },
  2928. }
  2929. for i, tt := range tests {
  2930. r := newTestRaft(1, tt.ids, 10, 1, NewMemoryStorage())
  2931. if !reflect.DeepEqual(r.nodes(), tt.wids) {
  2932. t.Errorf("#%d: nodes = %+v, want %+v", i, r.nodes(), tt.wids)
  2933. }
  2934. }
  2935. }
  2936. func TestCampaignWhileLeader(t *testing.T) {
  2937. testCampaignWhileLeader(t, false)
  2938. }
  2939. func TestPreCampaignWhileLeader(t *testing.T) {
  2940. testCampaignWhileLeader(t, true)
  2941. }
  2942. func testCampaignWhileLeader(t *testing.T, preVote bool) {
  2943. cfg := newTestConfig(1, []uint64{1}, 5, 1, NewMemoryStorage())
  2944. cfg.PreVote = preVote
  2945. r := newRaft(cfg)
  2946. if r.state != StateFollower {
  2947. t.Errorf("expected new node to be follower but got %s", r.state)
  2948. }
  2949. // We don't call campaign() directly because it comes after the check
  2950. // for our current state.
  2951. r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  2952. if r.state != StateLeader {
  2953. t.Errorf("expected single-node election to become leader but got %s", r.state)
  2954. }
  2955. term := r.Term
  2956. r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  2957. if r.state != StateLeader {
  2958. t.Errorf("expected to remain leader but got %s", r.state)
  2959. }
  2960. if r.Term != term {
  2961. t.Errorf("expected to remain in term %v but got %v", term, r.Term)
  2962. }
  2963. }
  2964. // TestCommitAfterRemoveNode verifies that pending commands can become
  2965. // committed when a config change reduces the quorum requirements.
  2966. func TestCommitAfterRemoveNode(t *testing.T) {
  2967. // Create a cluster with two nodes.
  2968. s := NewMemoryStorage()
  2969. r := newTestRaft(1, []uint64{1, 2}, 5, 1, s)
  2970. r.becomeCandidate()
  2971. r.becomeLeader()
  2972. // Begin to remove the second node.
  2973. cc := pb.ConfChange{
  2974. Type: pb.ConfChangeRemoveNode,
  2975. NodeID: 2,
  2976. }
  2977. ccData, err := cc.Marshal()
  2978. if err != nil {
  2979. t.Fatal(err)
  2980. }
  2981. r.Step(pb.Message{
  2982. Type: pb.MsgProp,
  2983. Entries: []pb.Entry{
  2984. {Type: pb.EntryConfChange, Data: ccData},
  2985. },
  2986. })
  2987. // Stabilize the log and make sure nothing is committed yet.
  2988. if ents := nextEnts(r, s); len(ents) > 0 {
  2989. t.Fatalf("unexpected committed entries: %v", ents)
  2990. }
  2991. ccIndex := r.raftLog.lastIndex()
  2992. // While the config change is pending, make another proposal.
  2993. r.Step(pb.Message{
  2994. Type: pb.MsgProp,
  2995. Entries: []pb.Entry{
  2996. {Type: pb.EntryNormal, Data: []byte("hello")},
  2997. },
  2998. })
  2999. // Node 2 acknowledges the config change, committing it.
  3000. r.Step(pb.Message{
  3001. Type: pb.MsgAppResp,
  3002. From: 2,
  3003. Index: ccIndex,
  3004. })
  3005. ents := nextEnts(r, s)
  3006. if len(ents) != 2 {
  3007. t.Fatalf("expected two committed entries, got %v", ents)
  3008. }
  3009. if ents[0].Type != pb.EntryNormal || ents[0].Data != nil {
  3010. t.Fatalf("expected ents[0] to be empty, but got %v", ents[0])
  3011. }
  3012. if ents[1].Type != pb.EntryConfChange {
  3013. t.Fatalf("expected ents[1] to be EntryConfChange, got %v", ents[1])
  3014. }
  3015. // Apply the config change. This reduces quorum requirements so the
  3016. // pending command can now commit.
  3017. r.removeNode(2)
  3018. ents = nextEnts(r, s)
  3019. if len(ents) != 1 || ents[0].Type != pb.EntryNormal ||
  3020. string(ents[0].Data) != "hello" {
  3021. t.Fatalf("expected one committed EntryNormal, got %v", ents)
  3022. }
  3023. }
  3024. // TestLeaderTransferToUpToDateNode verifies transferring should succeed
  3025. // if the transferee has the most up-to-date log entries when transfer starts.
  3026. func TestLeaderTransferToUpToDateNode(t *testing.T) {
  3027. nt := newNetwork(nil, nil, nil)
  3028. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  3029. lead := nt.peers[1].(*raft)
  3030. if lead.lead != 1 {
  3031. t.Fatalf("after election leader is %x, want 1", lead.lead)
  3032. }
  3033. // Transfer leadership to 2.
  3034. nt.send(pb.Message{From: 2, To: 1, Type: pb.MsgTransferLeader})
  3035. checkLeaderTransferState(t, lead, StateFollower, 2)
  3036. // After some log replication, transfer leadership back to 1.
  3037. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}})
  3038. nt.send(pb.Message{From: 1, To: 2, Type: pb.MsgTransferLeader})
  3039. checkLeaderTransferState(t, lead, StateLeader, 1)
  3040. }
  3041. // TestLeaderTransferToUpToDateNodeFromFollower verifies transferring should succeed
  3042. // if the transferee has the most up-to-date log entries when transfer starts.
  3043. // Not like TestLeaderTransferToUpToDateNode, where the leader transfer message
  3044. // is sent to the leader, in this test case every leader transfer message is sent
  3045. // to the follower.
  3046. func TestLeaderTransferToUpToDateNodeFromFollower(t *testing.T) {
  3047. nt := newNetwork(nil, nil, nil)
  3048. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  3049. lead := nt.peers[1].(*raft)
  3050. if lead.lead != 1 {
  3051. t.Fatalf("after election leader is %x, want 1", lead.lead)
  3052. }
  3053. // Transfer leadership to 2.
  3054. nt.send(pb.Message{From: 2, To: 2, Type: pb.MsgTransferLeader})
  3055. checkLeaderTransferState(t, lead, StateFollower, 2)
  3056. // After some log replication, transfer leadership back to 1.
  3057. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}})
  3058. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgTransferLeader})
  3059. checkLeaderTransferState(t, lead, StateLeader, 1)
  3060. }
  3061. // TestLeaderTransferWithCheckQuorum ensures transferring leader still works
  3062. // even the current leader is still under its leader lease
  3063. func TestLeaderTransferWithCheckQuorum(t *testing.T) {
  3064. nt := newNetwork(nil, nil, nil)
  3065. for i := 1; i < 4; i++ {
  3066. r := nt.peers[uint64(i)].(*raft)
  3067. r.checkQuorum = true
  3068. setRandomizedElectionTimeout(r, r.electionTimeout+i)
  3069. }
  3070. // Letting peer 2 electionElapsed reach to timeout so that it can vote for peer 1
  3071. f := nt.peers[2].(*raft)
  3072. for i := 0; i < f.electionTimeout; i++ {
  3073. f.tick()
  3074. }
  3075. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  3076. lead := nt.peers[1].(*raft)
  3077. if lead.lead != 1 {
  3078. t.Fatalf("after election leader is %x, want 1", lead.lead)
  3079. }
  3080. // Transfer leadership to 2.
  3081. nt.send(pb.Message{From: 2, To: 1, Type: pb.MsgTransferLeader})
  3082. checkLeaderTransferState(t, lead, StateFollower, 2)
  3083. // After some log replication, transfer leadership back to 1.
  3084. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}})
  3085. nt.send(pb.Message{From: 1, To: 2, Type: pb.MsgTransferLeader})
  3086. checkLeaderTransferState(t, lead, StateLeader, 1)
  3087. }
  3088. func TestLeaderTransferToSlowFollower(t *testing.T) {
  3089. defaultLogger.EnableDebug()
  3090. nt := newNetwork(nil, nil, nil)
  3091. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  3092. nt.isolate(3)
  3093. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}})
  3094. nt.recover()
  3095. lead := nt.peers[1].(*raft)
  3096. if lead.prs[3].Match != 1 {
  3097. t.Fatalf("node 1 has match %x for node 3, want %x", lead.prs[3].Match, 1)
  3098. }
  3099. // Transfer leadership to 3 when node 3 is lack of log.
  3100. nt.send(pb.Message{From: 3, To: 1, Type: pb.MsgTransferLeader})
  3101. checkLeaderTransferState(t, lead, StateFollower, 3)
  3102. }
  3103. func TestLeaderTransferAfterSnapshot(t *testing.T) {
  3104. nt := newNetwork(nil, nil, nil)
  3105. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  3106. nt.isolate(3)
  3107. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}})
  3108. lead := nt.peers[1].(*raft)
  3109. nextEnts(lead, nt.storage[1])
  3110. nt.storage[1].CreateSnapshot(lead.raftLog.applied, &pb.ConfState{Nodes: lead.nodes()}, nil)
  3111. nt.storage[1].Compact(lead.raftLog.applied)
  3112. nt.recover()
  3113. if lead.prs[3].Match != 1 {
  3114. t.Fatalf("node 1 has match %x for node 3, want %x", lead.prs[3].Match, 1)
  3115. }
  3116. // Transfer leadership to 3 when node 3 is lack of snapshot.
  3117. nt.send(pb.Message{From: 3, To: 1, Type: pb.MsgTransferLeader})
  3118. // Send pb.MsgHeartbeatResp to leader to trigger a snapshot for node 3.
  3119. nt.send(pb.Message{From: 3, To: 1, Type: pb.MsgHeartbeatResp})
  3120. checkLeaderTransferState(t, lead, StateFollower, 3)
  3121. }
  3122. func TestLeaderTransferToSelf(t *testing.T) {
  3123. nt := newNetwork(nil, nil, nil)
  3124. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  3125. lead := nt.peers[1].(*raft)
  3126. // Transfer leadership to self, there will be noop.
  3127. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgTransferLeader})
  3128. checkLeaderTransferState(t, lead, StateLeader, 1)
  3129. }
  3130. func TestLeaderTransferToNonExistingNode(t *testing.T) {
  3131. nt := newNetwork(nil, nil, nil)
  3132. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  3133. lead := nt.peers[1].(*raft)
  3134. // Transfer leadership to non-existing node, there will be noop.
  3135. nt.send(pb.Message{From: 4, To: 1, Type: pb.MsgTransferLeader})
  3136. checkLeaderTransferState(t, lead, StateLeader, 1)
  3137. }
  3138. func TestLeaderTransferTimeout(t *testing.T) {
  3139. nt := newNetwork(nil, nil, nil)
  3140. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  3141. nt.isolate(3)
  3142. lead := nt.peers[1].(*raft)
  3143. // Transfer leadership to isolated node, wait for timeout.
  3144. nt.send(pb.Message{From: 3, To: 1, Type: pb.MsgTransferLeader})
  3145. if lead.leadTransferee != 3 {
  3146. t.Fatalf("wait transferring, leadTransferee = %v, want %v", lead.leadTransferee, 3)
  3147. }
  3148. for i := 0; i < lead.heartbeatTimeout; i++ {
  3149. lead.tick()
  3150. }
  3151. if lead.leadTransferee != 3 {
  3152. t.Fatalf("wait transferring, leadTransferee = %v, want %v", lead.leadTransferee, 3)
  3153. }
  3154. for i := 0; i < lead.electionTimeout-lead.heartbeatTimeout; i++ {
  3155. lead.tick()
  3156. }
  3157. checkLeaderTransferState(t, lead, StateLeader, 1)
  3158. }
  3159. func TestLeaderTransferIgnoreProposal(t *testing.T) {
  3160. nt := newNetwork(nil, nil, nil)
  3161. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  3162. nt.isolate(3)
  3163. lead := nt.peers[1].(*raft)
  3164. // Transfer leadership to isolated node to let transfer pending, then send proposal.
  3165. nt.send(pb.Message{From: 3, To: 1, Type: pb.MsgTransferLeader})
  3166. if lead.leadTransferee != 3 {
  3167. t.Fatalf("wait transferring, leadTransferee = %v, want %v", lead.leadTransferee, 3)
  3168. }
  3169. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}})
  3170. err := lead.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}})
  3171. if err != ErrProposalDropped {
  3172. t.Fatalf("should return drop proposal error while transferring")
  3173. }
  3174. if lead.prs[1].Match != 1 {
  3175. t.Fatalf("node 1 has match %x, want %x", lead.prs[1].Match, 1)
  3176. }
  3177. }
  3178. func TestLeaderTransferReceiveHigherTermVote(t *testing.T) {
  3179. nt := newNetwork(nil, nil, nil)
  3180. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  3181. nt.isolate(3)
  3182. lead := nt.peers[1].(*raft)
  3183. // Transfer leadership to isolated node to let transfer pending.
  3184. nt.send(pb.Message{From: 3, To: 1, Type: pb.MsgTransferLeader})
  3185. if lead.leadTransferee != 3 {
  3186. t.Fatalf("wait transferring, leadTransferee = %v, want %v", lead.leadTransferee, 3)
  3187. }
  3188. nt.send(pb.Message{From: 2, To: 2, Type: pb.MsgHup, Index: 1, Term: 2})
  3189. checkLeaderTransferState(t, lead, StateFollower, 2)
  3190. }
  3191. func TestLeaderTransferRemoveNode(t *testing.T) {
  3192. nt := newNetwork(nil, nil, nil)
  3193. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  3194. nt.ignore(pb.MsgTimeoutNow)
  3195. lead := nt.peers[1].(*raft)
  3196. // The leadTransferee is removed when leadship transferring.
  3197. nt.send(pb.Message{From: 3, To: 1, Type: pb.MsgTransferLeader})
  3198. if lead.leadTransferee != 3 {
  3199. t.Fatalf("wait transferring, leadTransferee = %v, want %v", lead.leadTransferee, 3)
  3200. }
  3201. lead.removeNode(3)
  3202. checkLeaderTransferState(t, lead, StateLeader, 1)
  3203. }
  3204. // TestLeaderTransferBack verifies leadership can transfer back to self when last transfer is pending.
  3205. func TestLeaderTransferBack(t *testing.T) {
  3206. nt := newNetwork(nil, nil, nil)
  3207. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  3208. nt.isolate(3)
  3209. lead := nt.peers[1].(*raft)
  3210. nt.send(pb.Message{From: 3, To: 1, Type: pb.MsgTransferLeader})
  3211. if lead.leadTransferee != 3 {
  3212. t.Fatalf("wait transferring, leadTransferee = %v, want %v", lead.leadTransferee, 3)
  3213. }
  3214. // Transfer leadership back to self.
  3215. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgTransferLeader})
  3216. checkLeaderTransferState(t, lead, StateLeader, 1)
  3217. }
  3218. // TestLeaderTransferSecondTransferToAnotherNode verifies leader can transfer to another node
  3219. // when last transfer is pending.
  3220. func TestLeaderTransferSecondTransferToAnotherNode(t *testing.T) {
  3221. nt := newNetwork(nil, nil, nil)
  3222. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  3223. nt.isolate(3)
  3224. lead := nt.peers[1].(*raft)
  3225. nt.send(pb.Message{From: 3, To: 1, Type: pb.MsgTransferLeader})
  3226. if lead.leadTransferee != 3 {
  3227. t.Fatalf("wait transferring, leadTransferee = %v, want %v", lead.leadTransferee, 3)
  3228. }
  3229. // Transfer leadership to another node.
  3230. nt.send(pb.Message{From: 2, To: 1, Type: pb.MsgTransferLeader})
  3231. checkLeaderTransferState(t, lead, StateFollower, 2)
  3232. }
  3233. // TestLeaderTransferSecondTransferToSameNode verifies second transfer leader request
  3234. // to the same node should not extend the timeout while the first one is pending.
  3235. func TestLeaderTransferSecondTransferToSameNode(t *testing.T) {
  3236. nt := newNetwork(nil, nil, nil)
  3237. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  3238. nt.isolate(3)
  3239. lead := nt.peers[1].(*raft)
  3240. nt.send(pb.Message{From: 3, To: 1, Type: pb.MsgTransferLeader})
  3241. if lead.leadTransferee != 3 {
  3242. t.Fatalf("wait transferring, leadTransferee = %v, want %v", lead.leadTransferee, 3)
  3243. }
  3244. for i := 0; i < lead.heartbeatTimeout; i++ {
  3245. lead.tick()
  3246. }
  3247. // Second transfer leadership request to the same node.
  3248. nt.send(pb.Message{From: 3, To: 1, Type: pb.MsgTransferLeader})
  3249. for i := 0; i < lead.electionTimeout-lead.heartbeatTimeout; i++ {
  3250. lead.tick()
  3251. }
  3252. checkLeaderTransferState(t, lead, StateLeader, 1)
  3253. }
  3254. func checkLeaderTransferState(t *testing.T, r *raft, state StateType, lead uint64) {
  3255. if r.state != state || r.lead != lead {
  3256. t.Fatalf("after transferring, node has state %v lead %v, want state %v lead %v", r.state, r.lead, state, lead)
  3257. }
  3258. if r.leadTransferee != None {
  3259. t.Fatalf("after transferring, node has leadTransferee %v, want leadTransferee %v", r.leadTransferee, None)
  3260. }
  3261. }
  3262. // TestTransferNonMember verifies that when a MsgTimeoutNow arrives at
  3263. // a node that has been removed from the group, nothing happens.
  3264. // (previously, if the node also got votes, it would panic as it
  3265. // transitioned to StateLeader)
  3266. func TestTransferNonMember(t *testing.T) {
  3267. r := newTestRaft(1, []uint64{2, 3, 4}, 5, 1, NewMemoryStorage())
  3268. r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgTimeoutNow})
  3269. r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgVoteResp})
  3270. r.Step(pb.Message{From: 3, To: 1, Type: pb.MsgVoteResp})
  3271. if r.state != StateFollower {
  3272. t.Fatalf("state is %s, want StateFollower", r.state)
  3273. }
  3274. }
  3275. // TestNodeWithSmallerTermCanCompleteElection tests the scenario where a node
  3276. // that has been partitioned away (and fallen behind) rejoins the cluster at
  3277. // about the same time the leader node gets partitioned away.
  3278. // Previously the cluster would come to a standstill when run with PreVote
  3279. // enabled.
  3280. func TestNodeWithSmallerTermCanCompleteElection(t *testing.T) {
  3281. n1 := newTestRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  3282. n2 := newTestRaft(2, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  3283. n3 := newTestRaft(3, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  3284. n1.becomeFollower(1, None)
  3285. n2.becomeFollower(1, None)
  3286. n3.becomeFollower(1, None)
  3287. n1.preVote = true
  3288. n2.preVote = true
  3289. n3.preVote = true
  3290. // cause a network partition to isolate node 3
  3291. nt := newNetwork(n1, n2, n3)
  3292. nt.cut(1, 3)
  3293. nt.cut(2, 3)
  3294. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  3295. sm := nt.peers[1].(*raft)
  3296. if sm.state != StateLeader {
  3297. t.Errorf("peer 1 state: %s, want %s", sm.state, StateLeader)
  3298. }
  3299. sm = nt.peers[2].(*raft)
  3300. if sm.state != StateFollower {
  3301. t.Errorf("peer 2 state: %s, want %s", sm.state, StateFollower)
  3302. }
  3303. nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
  3304. sm = nt.peers[3].(*raft)
  3305. if sm.state != StatePreCandidate {
  3306. t.Errorf("peer 3 state: %s, want %s", sm.state, StatePreCandidate)
  3307. }
  3308. nt.send(pb.Message{From: 2, To: 2, Type: pb.MsgHup})
  3309. // check whether the term values are expected
  3310. // a.Term == 3
  3311. // b.Term == 3
  3312. // c.Term == 1
  3313. sm = nt.peers[1].(*raft)
  3314. if sm.Term != 3 {
  3315. t.Errorf("peer 1 term: %d, want %d", sm.Term, 3)
  3316. }
  3317. sm = nt.peers[2].(*raft)
  3318. if sm.Term != 3 {
  3319. t.Errorf("peer 2 term: %d, want %d", sm.Term, 3)
  3320. }
  3321. sm = nt.peers[3].(*raft)
  3322. if sm.Term != 1 {
  3323. t.Errorf("peer 3 term: %d, want %d", sm.Term, 1)
  3324. }
  3325. // check state
  3326. // a == follower
  3327. // b == leader
  3328. // c == pre-candidate
  3329. sm = nt.peers[1].(*raft)
  3330. if sm.state != StateFollower {
  3331. t.Errorf("peer 1 state: %s, want %s", sm.state, StateFollower)
  3332. }
  3333. sm = nt.peers[2].(*raft)
  3334. if sm.state != StateLeader {
  3335. t.Errorf("peer 2 state: %s, want %s", sm.state, StateLeader)
  3336. }
  3337. sm = nt.peers[3].(*raft)
  3338. if sm.state != StatePreCandidate {
  3339. t.Errorf("peer 3 state: %s, want %s", sm.state, StatePreCandidate)
  3340. }
  3341. sm.logger.Infof("going to bring back peer 3 and kill peer 2")
  3342. // recover the network then immediately isolate b which is currently
  3343. // the leader, this is to emulate the crash of b.
  3344. nt.recover()
  3345. nt.cut(2, 1)
  3346. nt.cut(2, 3)
  3347. // call for election
  3348. nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
  3349. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  3350. // do we have a leader?
  3351. sma := nt.peers[1].(*raft)
  3352. smb := nt.peers[3].(*raft)
  3353. if sma.state != StateLeader && smb.state != StateLeader {
  3354. t.Errorf("no leader")
  3355. }
  3356. }
  3357. // TestPreVoteWithSplitVote verifies that after split vote, cluster can complete
  3358. // election in next round.
  3359. func TestPreVoteWithSplitVote(t *testing.T) {
  3360. n1 := newTestRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  3361. n2 := newTestRaft(2, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  3362. n3 := newTestRaft(3, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  3363. n1.becomeFollower(1, None)
  3364. n2.becomeFollower(1, None)
  3365. n3.becomeFollower(1, None)
  3366. n1.preVote = true
  3367. n2.preVote = true
  3368. n3.preVote = true
  3369. nt := newNetwork(n1, n2, n3)
  3370. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  3371. // simulate leader down. followers start split vote.
  3372. nt.isolate(1)
  3373. nt.send([]pb.Message{
  3374. {From: 2, To: 2, Type: pb.MsgHup},
  3375. {From: 3, To: 3, Type: pb.MsgHup},
  3376. }...)
  3377. // check whether the term values are expected
  3378. // n2.Term == 3
  3379. // n3.Term == 3
  3380. sm := nt.peers[2].(*raft)
  3381. if sm.Term != 3 {
  3382. t.Errorf("peer 2 term: %d, want %d", sm.Term, 3)
  3383. }
  3384. sm = nt.peers[3].(*raft)
  3385. if sm.Term != 3 {
  3386. t.Errorf("peer 3 term: %d, want %d", sm.Term, 3)
  3387. }
  3388. // check state
  3389. // n2 == candidate
  3390. // n3 == candidate
  3391. sm = nt.peers[2].(*raft)
  3392. if sm.state != StateCandidate {
  3393. t.Errorf("peer 2 state: %s, want %s", sm.state, StateCandidate)
  3394. }
  3395. sm = nt.peers[3].(*raft)
  3396. if sm.state != StateCandidate {
  3397. t.Errorf("peer 3 state: %s, want %s", sm.state, StateCandidate)
  3398. }
  3399. // node 2 election timeout first
  3400. nt.send(pb.Message{From: 2, To: 2, Type: pb.MsgHup})
  3401. // check whether the term values are expected
  3402. // n2.Term == 4
  3403. // n3.Term == 4
  3404. sm = nt.peers[2].(*raft)
  3405. if sm.Term != 4 {
  3406. t.Errorf("peer 2 term: %d, want %d", sm.Term, 4)
  3407. }
  3408. sm = nt.peers[3].(*raft)
  3409. if sm.Term != 4 {
  3410. t.Errorf("peer 3 term: %d, want %d", sm.Term, 4)
  3411. }
  3412. // check state
  3413. // n2 == leader
  3414. // n3 == follower
  3415. sm = nt.peers[2].(*raft)
  3416. if sm.state != StateLeader {
  3417. t.Errorf("peer 2 state: %s, want %s", sm.state, StateLeader)
  3418. }
  3419. sm = nt.peers[3].(*raft)
  3420. if sm.state != StateFollower {
  3421. t.Errorf("peer 3 state: %s, want %s", sm.state, StateFollower)
  3422. }
  3423. }
  3424. // simulate rolling update a cluster for Pre-Vote. cluster has 3 nodes [n1, n2, n3].
  3425. // n1 is leader with term 2
  3426. // n2 is follower with term 2
  3427. // n3 is partitioned, with term 4 and less log, state is candidate
  3428. func newPreVoteMigrationCluster(t *testing.T) *network {
  3429. n1 := newTestRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  3430. n2 := newTestRaft(2, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  3431. n3 := newTestRaft(3, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
  3432. n1.becomeFollower(1, None)
  3433. n2.becomeFollower(1, None)
  3434. n3.becomeFollower(1, None)
  3435. n1.preVote = true
  3436. n2.preVote = true
  3437. // We intentionally do not enable PreVote for n3, this is done so in order
  3438. // to simulate a rolling restart process where it's possible to have a mixed
  3439. // version cluster with replicas with PreVote enabled, and replicas without.
  3440. nt := newNetwork(n1, n2, n3)
  3441. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
  3442. // Cause a network partition to isolate n3.
  3443. nt.isolate(3)
  3444. nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}})
  3445. nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
  3446. nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
  3447. // check state
  3448. // n1.state == StateLeader
  3449. // n2.state == StateFollower
  3450. // n3.state == StateCandidate
  3451. if n1.state != StateLeader {
  3452. t.Fatalf("node 1 state: %s, want %s", n1.state, StateLeader)
  3453. }
  3454. if n2.state != StateFollower {
  3455. t.Fatalf("node 2 state: %s, want %s", n2.state, StateFollower)
  3456. }
  3457. if n3.state != StateCandidate {
  3458. t.Fatalf("node 3 state: %s, want %s", n3.state, StateCandidate)
  3459. }
  3460. // check term
  3461. // n1.Term == 2
  3462. // n2.Term == 2
  3463. // n3.Term == 4
  3464. if n1.Term != 2 {
  3465. t.Fatalf("node 1 term: %d, want %d", n1.Term, 2)
  3466. }
  3467. if n2.Term != 2 {
  3468. t.Fatalf("node 2 term: %d, want %d", n2.Term, 2)
  3469. }
  3470. if n3.Term != 4 {
  3471. t.Fatalf("node 3 term: %d, want %d", n3.Term, 4)
  3472. }
  3473. // Enable prevote on n3, then recover the network
  3474. n3.preVote = true
  3475. nt.recover()
  3476. return nt
  3477. }
  3478. func TestPreVoteMigrationCanCompleteElection(t *testing.T) {
  3479. nt := newPreVoteMigrationCluster(t)
  3480. // n1 is leader with term 2
  3481. // n2 is follower with term 2
  3482. // n3 is pre-candidate with term 4, and less log
  3483. n2 := nt.peers[2].(*raft)
  3484. n3 := nt.peers[3].(*raft)
  3485. // simulate leader down
  3486. nt.isolate(1)
  3487. // Call for elections from both n2 and n3.
  3488. nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
  3489. nt.send(pb.Message{From: 2, To: 2, Type: pb.MsgHup})
  3490. // check state
  3491. // n2.state == Follower
  3492. // n3.state == PreCandidate
  3493. if n2.state != StateFollower {
  3494. t.Errorf("node 2 state: %s, want %s", n2.state, StateFollower)
  3495. }
  3496. if n3.state != StatePreCandidate {
  3497. t.Errorf("node 3 state: %s, want %s", n3.state, StatePreCandidate)
  3498. }
  3499. nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
  3500. nt.send(pb.Message{From: 2, To: 2, Type: pb.MsgHup})
  3501. // Do we have a leader?
  3502. if n2.state != StateLeader && n3.state != StateFollower {
  3503. t.Errorf("no leader")
  3504. }
  3505. }
  3506. func TestPreVoteMigrationWithFreeStuckPreCandidate(t *testing.T) {
  3507. nt := newPreVoteMigrationCluster(t)
  3508. // n1 is leader with term 2
  3509. // n2 is follower with term 2
  3510. // n3 is pre-candidate with term 4, and less log
  3511. n1 := nt.peers[1].(*raft)
  3512. n2 := nt.peers[2].(*raft)
  3513. n3 := nt.peers[3].(*raft)
  3514. nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
  3515. if n1.state != StateLeader {
  3516. t.Errorf("node 1 state: %s, want %s", n1.state, StateLeader)
  3517. }
  3518. if n2.state != StateFollower {
  3519. t.Errorf("node 2 state: %s, want %s", n2.state, StateFollower)
  3520. }
  3521. if n3.state != StatePreCandidate {
  3522. t.Errorf("node 3 state: %s, want %s", n3.state, StatePreCandidate)
  3523. }
  3524. // Pre-Vote again for safety
  3525. nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
  3526. if n1.state != StateLeader {
  3527. t.Errorf("node 1 state: %s, want %s", n1.state, StateLeader)
  3528. }
  3529. if n2.state != StateFollower {
  3530. t.Errorf("node 2 state: %s, want %s", n2.state, StateFollower)
  3531. }
  3532. if n3.state != StatePreCandidate {
  3533. t.Errorf("node 3 state: %s, want %s", n3.state, StatePreCandidate)
  3534. }
  3535. nt.send(pb.Message{From: 1, To: 3, Type: pb.MsgHeartbeat, Term: n1.Term})
  3536. // Disrupt the leader so that the stuck peer is freed
  3537. if n1.state != StateFollower {
  3538. t.Errorf("state = %s, want %s", n1.state, StateFollower)
  3539. }
  3540. if n3.Term != n1.Term {
  3541. t.Errorf("term = %d, want %d", n3.Term, n1.Term)
  3542. }
  3543. }
  3544. func entsWithConfig(configFunc func(*Config), terms ...uint64) *raft {
  3545. storage := NewMemoryStorage()
  3546. for i, term := range terms {
  3547. storage.Append([]pb.Entry{{Index: uint64(i + 1), Term: term}})
  3548. }
  3549. cfg := newTestConfig(1, []uint64{}, 5, 1, storage)
  3550. if configFunc != nil {
  3551. configFunc(cfg)
  3552. }
  3553. sm := newRaft(cfg)
  3554. sm.reset(terms[len(terms)-1])
  3555. return sm
  3556. }
  3557. // votedWithConfig creates a raft state machine with Vote and Term set
  3558. // to the given value but no log entries (indicating that it voted in
  3559. // the given term but has not received any logs).
  3560. func votedWithConfig(configFunc func(*Config), vote, term uint64) *raft {
  3561. storage := NewMemoryStorage()
  3562. storage.SetHardState(pb.HardState{Vote: vote, Term: term})
  3563. cfg := newTestConfig(1, []uint64{}, 5, 1, storage)
  3564. if configFunc != nil {
  3565. configFunc(cfg)
  3566. }
  3567. sm := newRaft(cfg)
  3568. sm.reset(term)
  3569. return sm
  3570. }
  3571. type network struct {
  3572. peers map[uint64]stateMachine
  3573. storage map[uint64]*MemoryStorage
  3574. dropm map[connem]float64
  3575. ignorem map[pb.MessageType]bool
  3576. // msgHook is called for each message sent. It may inspect the
  3577. // message and return true to send it or false to drop it.
  3578. msgHook func(pb.Message) bool
  3579. }
  3580. // newNetwork initializes a network from peers.
  3581. // A nil node will be replaced with a new *stateMachine.
  3582. // A *stateMachine will get its k, id.
  3583. // When using stateMachine, the address list is always [1, n].
  3584. func newNetwork(peers ...stateMachine) *network {
  3585. return newNetworkWithConfig(nil, peers...)
  3586. }
  3587. // newNetworkWithConfig is like newNetwork but calls the given func to
  3588. // modify the configuration of any state machines it creates.
  3589. func newNetworkWithConfig(configFunc func(*Config), peers ...stateMachine) *network {
  3590. size := len(peers)
  3591. peerAddrs := idsBySize(size)
  3592. npeers := make(map[uint64]stateMachine, size)
  3593. nstorage := make(map[uint64]*MemoryStorage, size)
  3594. for j, p := range peers {
  3595. id := peerAddrs[j]
  3596. switch v := p.(type) {
  3597. case nil:
  3598. nstorage[id] = NewMemoryStorage()
  3599. cfg := newTestConfig(id, peerAddrs, 10, 1, nstorage[id])
  3600. if configFunc != nil {
  3601. configFunc(cfg)
  3602. }
  3603. sm := newRaft(cfg)
  3604. npeers[id] = sm
  3605. case *raft:
  3606. learners := make(map[uint64]bool, len(v.learnerPrs))
  3607. for i := range v.learnerPrs {
  3608. learners[i] = true
  3609. }
  3610. v.id = id
  3611. v.prs = make(map[uint64]*Progress)
  3612. v.learnerPrs = make(map[uint64]*Progress)
  3613. for i := 0; i < size; i++ {
  3614. if _, ok := learners[peerAddrs[i]]; ok {
  3615. v.learnerPrs[peerAddrs[i]] = &Progress{IsLearner: true}
  3616. } else {
  3617. v.prs[peerAddrs[i]] = &Progress{}
  3618. }
  3619. }
  3620. v.reset(v.Term)
  3621. npeers[id] = v
  3622. case *blackHole:
  3623. npeers[id] = v
  3624. default:
  3625. panic(fmt.Sprintf("unexpected state machine type: %T", p))
  3626. }
  3627. }
  3628. return &network{
  3629. peers: npeers,
  3630. storage: nstorage,
  3631. dropm: make(map[connem]float64),
  3632. ignorem: make(map[pb.MessageType]bool),
  3633. }
  3634. }
  3635. func preVoteConfig(c *Config) {
  3636. c.PreVote = true
  3637. }
  3638. func (nw *network) send(msgs ...pb.Message) {
  3639. for len(msgs) > 0 {
  3640. m := msgs[0]
  3641. p := nw.peers[m.To]
  3642. p.Step(m)
  3643. msgs = append(msgs[1:], nw.filter(p.readMessages())...)
  3644. }
  3645. }
  3646. func (nw *network) drop(from, to uint64, perc float64) {
  3647. nw.dropm[connem{from, to}] = perc
  3648. }
  3649. func (nw *network) cut(one, other uint64) {
  3650. nw.drop(one, other, 2.0) // always drop
  3651. nw.drop(other, one, 2.0) // always drop
  3652. }
  3653. func (nw *network) isolate(id uint64) {
  3654. for i := 0; i < len(nw.peers); i++ {
  3655. nid := uint64(i) + 1
  3656. if nid != id {
  3657. nw.drop(id, nid, 1.0) // always drop
  3658. nw.drop(nid, id, 1.0) // always drop
  3659. }
  3660. }
  3661. }
  3662. func (nw *network) ignore(t pb.MessageType) {
  3663. nw.ignorem[t] = true
  3664. }
  3665. func (nw *network) recover() {
  3666. nw.dropm = make(map[connem]float64)
  3667. nw.ignorem = make(map[pb.MessageType]bool)
  3668. }
  3669. func (nw *network) filter(msgs []pb.Message) []pb.Message {
  3670. mm := []pb.Message{}
  3671. for _, m := range msgs {
  3672. if nw.ignorem[m.Type] {
  3673. continue
  3674. }
  3675. switch m.Type {
  3676. case pb.MsgHup:
  3677. // hups never go over the network, so don't drop them but panic
  3678. panic("unexpected msgHup")
  3679. default:
  3680. perc := nw.dropm[connem{m.From, m.To}]
  3681. if n := rand.Float64(); n < perc {
  3682. continue
  3683. }
  3684. }
  3685. if nw.msgHook != nil {
  3686. if !nw.msgHook(m) {
  3687. continue
  3688. }
  3689. }
  3690. mm = append(mm, m)
  3691. }
  3692. return mm
  3693. }
  3694. type connem struct {
  3695. from, to uint64
  3696. }
  3697. type blackHole struct{}
  3698. func (blackHole) Step(pb.Message) error { return nil }
  3699. func (blackHole) readMessages() []pb.Message { return nil }
  3700. var nopStepper = &blackHole{}
  3701. func idsBySize(size int) []uint64 {
  3702. ids := make([]uint64, size)
  3703. for i := 0; i < size; i++ {
  3704. ids[i] = 1 + uint64(i)
  3705. }
  3706. return ids
  3707. }
  3708. // setRandomizedElectionTimeout set up the value by caller instead of choosing
  3709. // by system, in some test scenario we need to fill in some expected value to
  3710. // ensure the certainty
  3711. func setRandomizedElectionTimeout(r *raft, v int) {
  3712. r.randomizedElectionTimeout = v
  3713. }
  3714. func newTestConfig(id uint64, peers []uint64, election, heartbeat int, storage Storage) *Config {
  3715. return &Config{
  3716. ID: id,
  3717. peers: peers,
  3718. ElectionTick: election,
  3719. HeartbeatTick: heartbeat,
  3720. Storage: storage,
  3721. MaxSizePerMsg: noLimit,
  3722. MaxInflightMsgs: 256,
  3723. }
  3724. }
  3725. func newTestRaft(id uint64, peers []uint64, election, heartbeat int, storage Storage) *raft {
  3726. return newRaft(newTestConfig(id, peers, election, heartbeat, storage))
  3727. }
  3728. func newTestLearnerRaft(id uint64, peers []uint64, learners []uint64, election, heartbeat int, storage Storage) *raft {
  3729. cfg := newTestConfig(id, peers, election, heartbeat, storage)
  3730. cfg.learners = learners
  3731. return newRaft(cfg)
  3732. }