raft_test.go 128 KB

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