raft_test.go 122 KB

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