├── .gitattributes ├── .gitignore ├── LICENSE ├── README.md └── src ├── go.mod ├── go.sum ├── labgob ├── labgob.go └── test_test.go ├── labrpc ├── labrpc.go └── test_test.go └── raft ├── append_entry.go ├── config.go ├── go-test-many.sh ├── leader_election.go ├── log.go ├── persister.go ├── raft.go ├── request_vote.go ├── test.sh ├── test_test.go └── util.go /.gitattributes: -------------------------------------------------------------------------------- 1 | # Auto detect text files and perform LF normalization 2 | * text=auto 3 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | # Binaries for programs and plugins 2 | *.exe 3 | *.exe~ 4 | *.dll 5 | *.so 6 | *.dylib 7 | 8 | # Test binary, build with `go test -c` 9 | *.test 10 | 11 | # Output of the go coverage tool, specifically when used with LiteIDE 12 | *.out 13 | .idea 14 | .DS_Store 15 | test.txt 16 | test* 17 | *.log 18 | 19 | pkg/ 20 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | MIT License 2 | 3 | Copyright (c) 2021 zhangshiqiu 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in all 13 | copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 21 | SOFTWARE. 22 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | 本文将介绍6.824 Lab2(测试用例2021/2020版 2A + 2B + 2C部分)的具体实现。代码通过5000次测试,大致上应该没有问题。2021版的测试还有一个2D的部分,并没有包含在本文中。2D部分是关于Raft Snapshot,过早的实现2D可能会掩盖一些隐藏的bug。比如2C的一些test其实会产生超长的歧义链,这个时候就需要实现fast rollback优化,但是如果过早实现了snapshot就可以通过发送snapshot的方式直接修正歧义链。 2 | 3 | ## Raft的结构 4 | 5 | ```go 6 | type Raft struct { 7 | mu sync.Mutex 8 | peers []*labrpc.ClientEnd 9 | persister *Persister 10 | me int 11 | dead int32 12 | 13 | state RaftState 14 | appendEntryCh chan *Entry 15 | heartBeat time.Duration 16 | electionTime time.Time 17 | 18 | currentTerm int 19 | votedFor int 20 | log Log 21 | 22 | commitIndex int 23 | lastApplied int 24 | 25 | nextIndex []int 26 | matchIndex []int 27 | 28 | applyCh chan ApplyMsg 29 | applyCond *sync.Cond 30 | } 31 | ``` 32 | 33 | Raft的结构有一部分已经给出,剩下的大部分可以根据Figure 2补全。 34 | 35 | ```go 36 | func Make(peers []*labrpc.ClientEnd, me int, 37 | persister *Persister, applyCh chan ApplyMsg) *Raft { 38 | rf := &Raft{} 39 | rf.peers = peers 40 | rf.persister = persister 41 | rf.me = me 42 | 43 | rf.state = Follower 44 | rf.currentTerm = 0 45 | rf.votedFor = -1 46 | rf.heartBeat = 100 * time.Millisecond 47 | rf.resetElectionTimer() 48 | 49 | rf.log = makeEmptyLog() 50 | rf.log.append(Entry{-1, 0, 0}) 51 | rf.commitIndex = 0 52 | rf.lastApplied = 0 53 | rf.nextIndex = make([]int, len(rf.peers)) 54 | rf.matchIndex = make([]int, len(rf.peers)) 55 | 56 | rf.applyCh = applyCh 57 | rf.applyCond = sync.NewCond(&rf.mu) 58 | 59 | rf.readPersist(persister.ReadRaftState()) 60 | 61 | go rf.ticker() 62 | 63 | go rf.applier() 64 | return rf 65 | } 66 | ``` 67 | 68 | 初始化Raft的时候,除了给raft做基本的赋值之外,还要额外启动两个goroutine。作业要求中提到不要使用Go内置的timer,在2021版的代码里新增了一个ticker函数,作用也很简单,计时并且按时间触发leader election或者append entry。而applier则是负责将command应用到state machine,这一点和论文中一致。 69 | 70 | 先看ticker()函数 71 | 72 | ## ticker 73 | 74 | ```go 75 | func (rf *Raft) ticker() { 76 | for rf.killed() == false { 77 | time.Sleep(rf.heartBeat) 78 | rf.mu.Lock() 79 | if rf.state == Leader { 80 | rf.appendEntries(true) 81 | } 82 | if time.Now().After(rf.electionTime) { 83 | rf.leaderElection() 84 | } 85 | rf.mu.Unlock() 86 | } 87 | } 88 | ``` 89 | 90 | ticker会以心跳为周期不断检查状态。如果当前是Leader就会发送心跳包,而心跳包是靠appendEntries()发送空log,而不是额外的函数,这一点在论文和student guide都有强调。 91 | 92 | 如果发现选举超时,这时候就会出发新一轮leader election。先看leader election的实现 93 | 94 | ## leader election 95 | ```go 96 | func (rf *Raft) leaderElection() { 97 | rf.currentTerm++ 98 | rf.state = Candidate 99 | rf.votedFor = rf.me 100 | rf.persist() 101 | rf.resetElectionTimer() 102 | term := rf.currentTerm 103 | voteCounter := 1 104 | lastLog := rf.log.lastLog() 105 | args := RequestVoteArgs{ 106 | Term: term, 107 | CandidateId: rf.me, 108 | LastLogIndex: lastLog.Index, 109 | LastLogTerm: lastLog.Term, 110 | } 111 | 112 | var becomeLeader sync.Once 113 | for serverId, _ := range rf.peers { 114 | if serverId != rf.me { 115 | go rf.candidateRequestVote(serverId, &args, &voteCounter, &becomeLeader) 116 | } 117 | } 118 | } 119 | ``` 120 | 121 | 启动新一轮leader election时,首先要将自己转为candidate状态,并且给自己投一票。然后向所有peer请求投票。RequestVote RPC的参数和返回值需要按照Figure 2实现。 122 | 123 | ```go 124 | func (rf *Raft) candidateRequestVote(serverId int, args *RequestVoteArgs, voteCounter *int, becomeLeader *sync.Once) { 125 | reply := RequestVoteReply{} 126 | ok := rf.sendRequestVote(serverId, args, &reply) 127 | if !ok { 128 | return 129 | } 130 | rf.mu.Lock() 131 | defer rf.mu.Unlock() 132 | if reply.Term > args.Term { 133 | rf.setNewTerm(reply.Term) 134 | return 135 | } 136 | if reply.Term < args.Term { 137 | return 138 | } 139 | if !reply.VoteGranted { 140 | return 141 | } 142 | 143 | *voteCounter++ 144 | if *voteCounter > len(rf.peers)/2 && 145 | rf.currentTerm == args.Term && 146 | rf.state == Candidate { 147 | becomeLeader.Do(func() { 148 | rf.state = Leader 149 | lastLogIndex := rf.log.lastLog().Index 150 | for i, _ := range rf.peers { 151 | rf.nextIndex[i] = lastLogIndex + 1 152 | rf.matchIndex[i] = 0 153 | } 154 | rf.appendEntries(true) 155 | }) 156 | } 157 | } 158 | ``` 159 | 除了要满足论文的Figure 2中*Rules for Servers*的要求之外,要注意当candidate收到半数以上投票之后就可以进入leader状态,而这个状态转变会更新nextIndex[]和matchIndex[],并且再成为leader之后要立刻发送一次心跳。我们希望状态转变只发生一次,这里我使用了go的sync.Once。简单的使用bool flag也同样可以达成目的,只不过可读性没有这么直观。 160 | 161 | ## RequestVote 162 | 163 | 另一方面,任何服务器收到RequestVote RPC之后,要实现Figure 2中*RequestVote RPC Receiver implementation*的逻辑,同时也要满足*Rules for Servers* 164 | 165 | ```go 166 | func (rf *Raft) RequestVote(args *RequestVoteArgs, reply *RequestVoteReply) { 167 | rf.mu.Lock() 168 | defer rf.mu.Unlock() 169 | 170 | // rules for servers 171 | // all servers 2 172 | if args.Term > rf.currentTerm { 173 | rf.setNewTerm(args.Term) 174 | } 175 | 176 | // request vote rpc receiver 1 177 | if args.Term < rf.currentTerm { 178 | reply.Term = rf.currentTerm 179 | reply.VoteGranted = false 180 | return 181 | } 182 | 183 | // request vote rpc receiver 2 184 | myLastLog := rf.log.lastLog() 185 | upToDate := args.LastLogTerm > myLastLog.Term || 186 | (args.LastLogTerm == myLastLog.Term && args.LastLogIndex >= myLastLog.Index) 187 | if (rf.votedFor == -1 || rf.votedFor == args.CandidateId) && upToDate { 188 | reply.VoteGranted = true 189 | rf.votedFor = args.CandidateId 190 | rf.persist() 191 | rf.resetElectionTimer() 192 | } else { 193 | reply.VoteGranted = false 194 | } 195 | reply.Term = rf.currentTerm 196 | } 197 | ``` 198 | 199 | 论文5.2 & 5.4节详细解释了这部分逻辑的来源。 200 | 201 | ## AppendEntry 202 | 203 | 完成了leader election之后,leader会立刻触发一次心跳包,随后在每个心跳周期发送心跳包,来阻止新一轮leader election。 204 | Figure 2中*Rules for Servers*的*Leaders*部分将心跳称为`initial empty AppendEntries RPCs (heartbeat)`,将包含log的RPC称为`AppendEntries RPC with log entries starting at nextIndex`。这种描述听起来像是用了两段不同的代码。 205 | 而实际上因为这里的心跳有两种理解:每个心跳周期,发送一次AppendEntries RPC,当这个RPC不包含log时,这个包被称为心跳包。所以也有可能发生这么一种情况:触发了一次心跳,但是带有log(即心跳周期到了,触发了一次AppendEntries RPC,但是由于follower落后了,所以这个RPC带有一段log,此时这个包就不能称为心跳包)。 206 | 207 | 实践中,我在每个心跳周期和收到新的command之后各会触发一次AppendEntries RPC。然而仔细读论文后发现,论文中并没有只说了心跳会触发AppendEntries RPC,并没有说收到客户端的指令之后应该触发AppendEntries RPC。 208 | 209 | 我甚至认为在理论上AppendEntries可以完全交给heartbeat周期来触发,即收到command后,并不立刻发送AppendEntries,而是等待下一个心跳。这种方法可以减少RPC的数量,并且通过了连续1000次测试。但是代价就是每条command的提交周期变长。 210 | 211 | ```go 212 | func (rf *Raft) appendEntries(heartbeat bool) { 213 | lastLog := rf.log.lastLog() 214 | for peer, _ := range rf.peers { 215 | if peer == rf.me { 216 | rf.resetElectionTimer() 217 | continue 218 | } 219 | // rules for leader 3 220 | if lastLog.Index >= rf.nextIndex[peer] || heartbeat { 221 | nextIndex := rf.nextIndex[peer] 222 | if nextIndex <= 0 { 223 | nextIndex = 1 224 | } 225 | if lastLog.Index+1 < nextIndex { 226 | nextIndex = lastLog.Index 227 | } 228 | prevLog := rf.log.at(nextIndex - 1) 229 | args := AppendEntriesArgs{ 230 | Term: rf.currentTerm, 231 | LeaderId: rf.me, 232 | PrevLogIndex: prevLog.Index, 233 | PrevLogTerm: prevLog.Term, 234 | Entries: make([]Entry, lastLog.Index-nextIndex+1), 235 | LeaderCommit: rf.commitIndex, 236 | } 237 | copy(args.Entries, rf.log.slice(nextIndex)) 238 | go rf.leaderSendEntries(peer, &args) 239 | } 240 | } 241 | } 242 | ``` 243 | Leader在AppendEntries中会并行地给所有server发送消息,然后根据返回的消息更新nextIndex和matchIndex,这部分需要按照论文5.3节来实现。 244 | 但是同样在5.3节,作者提到了fast rollback优化。Morris的讲座上,实现这种优化需要在返回消息中额外加入XTerm, XIndex, XLen三个字段。 245 | ```go 246 | type AppendEntriesReply struct { 247 | Term int 248 | Success bool 249 | Conflict bool 250 | XTerm int 251 | XIndex int 252 | XLen int 253 | } 254 | ``` 255 | 原作的说法上,这种优化可能不是必须的,所以并不作为raft核心算法的一部分。实际上,我感觉如果直接在raft-core的代码上实现,有可能会引入一个小bug,不影响运行但可能会拖效率。然而这点我也不好证明,只能说里面多半有一部分冗余代码,但是我也不敢删,所以就留着…… 256 | 257 | ```go 258 | func (rf *Raft) leaderSendEntries(serverId int, args *AppendEntriesArgs) { 259 | var reply AppendEntriesReply 260 | ok := rf.sendAppendEntries(serverId, args, &reply) 261 | if !ok { 262 | return 263 | } 264 | rf.mu.Lock() 265 | defer rf.mu.Unlock() 266 | if reply.Term > rf.currentTerm { 267 | rf.setNewTerm(reply.Term) 268 | return 269 | } 270 | if args.Term == rf.currentTerm { 271 | // rules for leader 3.1 272 | if reply.Success { 273 | match := args.PrevLogIndex + len(args.Entries) 274 | next := match + 1 275 | rf.nextIndex[serverId] = max(rf.nextIndex[serverId], next) 276 | rf.matchIndex[serverId] = max(rf.matchIndex[serverId], match) 277 | } else if reply.Conflict { 278 | if reply.XTerm == -1 { 279 | rf.nextIndex[serverId] = reply.XLen 280 | } else { 281 | lastLogInXTerm := rf.findLastLogInTerm(reply.XTerm) 282 | if lastLogInXTerm > 0 { 283 | rf.nextIndex[serverId] = lastLogInXTerm 284 | } else { 285 | rf.nextIndex[serverId] = reply.XIndex 286 | } 287 | } 288 | } else if rf.nextIndex[serverId] > 1 { 289 | rf.nextIndex[serverId]-- 290 | } 291 | rf.leaderCommitRule() 292 | } 293 | } 294 | ``` 295 | 当peer收到AppendEntry RPC的时候,需要实现Figure 2中*AppendEntry RPC Receiver implementation* + *Rules for Servers*。具体哪些相关,我已经加在注释里了。论文里的步骤必须严格遵守,不要自由发挥。这一点想必大家在debug的时候都深有体会…… 296 | 297 | ```go 298 | func (rf *Raft) AppendEntries(args *AppendEntriesArgs, reply *AppendEntriesReply) { 299 | rf.mu.Lock() 300 | defer rf.mu.Unlock() 301 | // rules for servers 302 | // all servers 2 303 | reply.Success = false 304 | reply.Term = rf.currentTerm 305 | if args.Term > rf.currentTerm { 306 | rf.setNewTerm(args.Term) 307 | return 308 | } 309 | 310 | // append entries rpc 1 311 | if args.Term < rf.currentTerm { 312 | return 313 | } 314 | rf.resetElectionTimer() 315 | 316 | // candidate rule 3 317 | if rf.state == Candidate { 318 | rf.state = Follower 319 | } 320 | // append entries rpc 2 321 | if rf.log.lastLog().Index < args.PrevLogIndex { 322 | reply.Conflict = true 323 | reply.XTerm = -1 324 | reply.XIndex = -1 325 | reply.XLen = rf.log.len() 326 | return 327 | } 328 | if rf.log.at(args.PrevLogIndex).Term != args.PrevLogTerm { 329 | reply.Conflict = true 330 | xTerm := rf.log.at(args.PrevLogIndex).Term 331 | for xIndex := args.PrevLogIndex; xIndex > 0; xIndex-- { 332 | if rf.log.at(xIndex-1).Term != xTerm { 333 | reply.XIndex = xIndex 334 | break 335 | } 336 | } 337 | reply.XTerm = xTerm 338 | reply.XLen = rf.log.len() 339 | return 340 | } 341 | 342 | for idx, entry := range args.Entries { 343 | // append entries rpc 3 344 | if entry.Index <= rf.log.lastLog().Index && rf.log.at(entry.Index).Term != entry.Term { 345 | rf.log.truncate(entry.Index) 346 | rf.persist() 347 | } 348 | // append entries rpc 4 349 | if entry.Index > rf.log.lastLog().Index { 350 | rf.log.append(args.Entries[idx:]...) 351 | rf.persist() 352 | break 353 | } 354 | } 355 | 356 | // append entries rpc 5 357 | if args.LeaderCommit > rf.commitIndex { 358 | rf.commitIndex = min(args.LeaderCommit, rf.log.lastLog().Index) 359 | rf.apply() 360 | } 361 | reply.Success = true 362 | } 363 | ``` 364 | 完成AppendEntry RPC之后,Leader需要提交已有的日志条目,这一点在论文5.3 & 5.4有文字叙述。但是具体在什么时候提交,需要自己去把握。仔细看Figure 2的话,其实这部分对应*Rules for Servers*中Leader部分的最后一小段 365 | 366 | ```go 367 | func (rf *Raft) leaderCommitRule() { 368 | // leader rule 4 369 | if rf.state != Leader { 370 | return 371 | } 372 | 373 | for n := rf.commitIndex + 1; n <= rf.log.lastLog().Index; n++ { 374 | if rf.log.at(n).Term != rf.currentTerm { 375 | continue 376 | } 377 | counter := 1 378 | for serverId := 0; serverId < len(rf.peers); serverId++ { 379 | if serverId != rf.me && rf.matchIndex[serverId] >= n { 380 | counter++ 381 | } 382 | if counter > len(rf.peers)/2 { 383 | rf.commitIndex = n 384 | rf.apply() 385 | break 386 | } 387 | } 388 | } 389 | } 390 | ``` 391 | 392 | ## applier 393 | 394 | student guide中提到应该使用一个`a dedicated “applier”`来专门处理日志commit的事情。所以按TA说的来,并且按照作业要求使用applyCond。这里可能会触发student guide所说的`The four-way deadlock`,不过guide中也给出了解决方案。不重复赘述,文末有中文版的链接,自己去读。 395 | 396 | ```go 397 | func (rf *Raft) apply() { 398 | rf.applyCond.Broadcast() 399 | } 400 | 401 | func (rf *Raft) applier() { 402 | rf.mu.Lock() 403 | defer rf.mu.Unlock() 404 | 405 | for !rf.killed() { 406 | if rf.commitIndex > rf.lastApplied && rf.log.lastLog().Index > rf.lastApplied { 407 | rf.lastApplied++ 408 | applyMsg := ApplyMsg{ 409 | CommandValid: true, 410 | Command: rf.log.at(rf.lastApplied).Command, 411 | CommandIndex: rf.lastApplied, 412 | } 413 | rf.mu.Unlock() 414 | rf.applyCh <- applyMsg 415 | rf.mu.Lock() 416 | } else { 417 | rf.applyCond.Wait() 418 | } 419 | } 420 | } 421 | ``` 422 | 423 | ## Start 424 | 425 | 最后是start函数,它会接受客户端的command,并且应用raft算法。前面也说了,每次start并不一定要立刻触发AppendEntry。理论上如果每次都触发AppendEntry,而start被调用的频率又超高,Leader就会疯狂发送RPC。如果不主动触发,而被动的依赖心跳周期,反而可以造成batch operation的效果,将QPS固定成一个相对较小的值。当中的trade-off需要根据使用场景自己衡量。 426 | 427 | ```go 428 | func (rf *Raft) Start(command interface{}) (int, int, bool) { 429 | rf.mu.Lock() 430 | defer rf.mu.Unlock() 431 | if rf.state != Leader { 432 | return -1, rf.currentTerm, false 433 | } 434 | index := rf.log.lastLog().Index + 1 435 | term := rf.currentTerm 436 | 437 | log := Entry{ 438 | Command: command, 439 | Index: index, 440 | Term: term, 441 | } 442 | rf.log.append(log) 443 | rf.persist() 444 | rf.appendEntries(false) 445 | 446 | return index, term, true 447 | } 448 | ``` 449 | 450 | ## 总结 451 | 452 | 1. 一定要按照论文+student guide来实现,完全按照论文确实可以完美复现。但是话说回来,都做到这份上了,为啥不直接给个伪代码版本。。。 453 | 2. 千万不要过早优化。直接使用函数粒度的锁,细粒度的锁在提升性能的同时,会增加复杂度,尤其debug的难度,并且这个难度在复杂的高并发+不可靠的网络背景下可以无限上升。等待debug难度过大,就只能删掉重构了。 454 | 3. 通过单次测试只是第一步,真正的考验才刚刚开始。很多bug出现的概率不高(话说统计课上将概率低于5%叫做小概率事件,然而这种bug到处都是…… 455 | 4. 所以debug的log一定要写详细点,向我单跑一次TestFigure8Unreliable2C能打出两万条log 456 | 5. 接上条,早点写个log可视化的脚本来处理。Python写了一下,大约30多行,可以把45s左右的test过程,变成一个5分钟左右的动画,能看到每个server的append、commit等过程 457 | 6. 论文+student guide需要反复看,所以早点把重点摘出来写成笔记放在手边。我在微信上发了中文版的翻译 458 | 459 | ## Acknowledge 460 | + 感谢 @chentaiyue 提出的的issue,非常细心! 461 | 462 | 463 | 464 | 465 | 466 | 467 | 468 | 469 | 470 | 471 | 472 | 473 | 474 | -------------------------------------------------------------------------------- /src/go.mod: -------------------------------------------------------------------------------- 1 | module "6.824" 2 | 3 | go 1.19 4 | -------------------------------------------------------------------------------- /src/go.sum: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/s09g/raft-go/0b9750ced3a518cbf8c00d02803c553cc8d952ca/src/go.sum -------------------------------------------------------------------------------- /src/labgob/labgob.go: -------------------------------------------------------------------------------- 1 | package labgob 2 | 3 | // 4 | // trying to send non-capitalized fields over RPC produces a range of 5 | // misbehavior, including both mysterious incorrect computation and 6 | // outright crashes. so this wrapper around Go's encoding/gob warns 7 | // about non-capitalized field names. 8 | // 9 | 10 | import "encoding/gob" 11 | import "io" 12 | import "reflect" 13 | import "fmt" 14 | import "sync" 15 | import "unicode" 16 | import "unicode/utf8" 17 | 18 | var mu sync.Mutex 19 | var errorCount int // for TestCapital 20 | var checked map[reflect.Type]bool 21 | 22 | type LabEncoder struct { 23 | gob *gob.Encoder 24 | } 25 | 26 | func NewEncoder(w io.Writer) *LabEncoder { 27 | enc := &LabEncoder{} 28 | enc.gob = gob.NewEncoder(w) 29 | return enc 30 | } 31 | 32 | func (enc *LabEncoder) Encode(e interface{}) error { 33 | checkValue(e) 34 | return enc.gob.Encode(e) 35 | } 36 | 37 | func (enc *LabEncoder) EncodeValue(value reflect.Value) error { 38 | checkValue(value.Interface()) 39 | return enc.gob.EncodeValue(value) 40 | } 41 | 42 | type LabDecoder struct { 43 | gob *gob.Decoder 44 | } 45 | 46 | func NewDecoder(r io.Reader) *LabDecoder { 47 | dec := &LabDecoder{} 48 | dec.gob = gob.NewDecoder(r) 49 | return dec 50 | } 51 | 52 | func (dec *LabDecoder) Decode(e interface{}) error { 53 | checkValue(e) 54 | checkDefault(e) 55 | return dec.gob.Decode(e) 56 | } 57 | 58 | func Register(value interface{}) { 59 | checkValue(value) 60 | gob.Register(value) 61 | } 62 | 63 | func RegisterName(name string, value interface{}) { 64 | checkValue(value) 65 | gob.RegisterName(name, value) 66 | } 67 | 68 | func checkValue(value interface{}) { 69 | checkType(reflect.TypeOf(value)) 70 | } 71 | 72 | func checkType(t reflect.Type) { 73 | k := t.Kind() 74 | 75 | mu.Lock() 76 | // only complain once, and avoid recursion. 77 | if checked == nil { 78 | checked = map[reflect.Type]bool{} 79 | } 80 | if checked[t] { 81 | mu.Unlock() 82 | return 83 | } 84 | checked[t] = true 85 | mu.Unlock() 86 | 87 | switch k { 88 | case reflect.Struct: 89 | for i := 0; i < t.NumField(); i++ { 90 | f := t.Field(i) 91 | rune, _ := utf8.DecodeRuneInString(f.Name) 92 | if unicode.IsUpper(rune) == false { 93 | // ta da 94 | fmt.Printf("labgob error: lower-case field %v of %v in RPC or persist/snapshot will break your Raft\n", 95 | f.Name, t.Name()) 96 | mu.Lock() 97 | errorCount += 1 98 | mu.Unlock() 99 | } 100 | checkType(f.Type) 101 | } 102 | return 103 | case reflect.Slice, reflect.Array, reflect.Ptr: 104 | checkType(t.Elem()) 105 | return 106 | case reflect.Map: 107 | checkType(t.Elem()) 108 | checkType(t.Key()) 109 | return 110 | default: 111 | return 112 | } 113 | } 114 | 115 | // 116 | // warn if the value contains non-default values, 117 | // as it would if one sent an RPC but the reply 118 | // struct was already modified. if the RPC reply 119 | // contains default values, GOB won't overwrite 120 | // the non-default value. 121 | // 122 | func checkDefault(value interface{}) { 123 | if value == nil { 124 | return 125 | } 126 | checkDefault1(reflect.ValueOf(value), 1, "") 127 | } 128 | 129 | func checkDefault1(value reflect.Value, depth int, name string) { 130 | if depth > 3 { 131 | return 132 | } 133 | 134 | t := value.Type() 135 | k := t.Kind() 136 | 137 | switch k { 138 | case reflect.Struct: 139 | for i := 0; i < t.NumField(); i++ { 140 | vv := value.Field(i) 141 | name1 := t.Field(i).Name 142 | if name != "" { 143 | name1 = name + "." + name1 144 | } 145 | checkDefault1(vv, depth+1, name1) 146 | } 147 | return 148 | case reflect.Ptr: 149 | if value.IsNil() { 150 | return 151 | } 152 | checkDefault1(value.Elem(), depth+1, name) 153 | return 154 | case reflect.Bool, 155 | reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64, 156 | reflect.Uint, reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64, 157 | reflect.Uintptr, reflect.Float32, reflect.Float64, 158 | reflect.String: 159 | if reflect.DeepEqual(reflect.Zero(t).Interface(), value.Interface()) == false { 160 | mu.Lock() 161 | if errorCount < 1 { 162 | what := name 163 | if what == "" { 164 | what = t.Name() 165 | } 166 | // this warning typically arises if code re-uses the same RPC reply 167 | // variable for multiple RPC calls, or if code restores persisted 168 | // state into variable that already have non-default values. 169 | fmt.Printf("labgob warning: Decoding into a non-default variable/field %v may not work\n", 170 | what) 171 | } 172 | errorCount += 1 173 | mu.Unlock() 174 | } 175 | return 176 | } 177 | } 178 | -------------------------------------------------------------------------------- /src/labgob/test_test.go: -------------------------------------------------------------------------------- 1 | package labgob 2 | 3 | import "testing" 4 | 5 | import "bytes" 6 | 7 | type T1 struct { 8 | T1int0 int 9 | T1int1 int 10 | T1string0 string 11 | T1string1 string 12 | } 13 | 14 | type T2 struct { 15 | T2slice []T1 16 | T2map map[int]*T1 17 | T2t3 interface{} 18 | } 19 | 20 | type T3 struct { 21 | T3int999 int 22 | } 23 | 24 | // 25 | // test that we didn't break GOB. 26 | // 27 | func TestGOB(t *testing.T) { 28 | e0 := errorCount 29 | 30 | w := new(bytes.Buffer) 31 | 32 | Register(T3{}) 33 | 34 | { 35 | x0 := 0 36 | x1 := 1 37 | t1 := T1{} 38 | t1.T1int1 = 1 39 | t1.T1string1 = "6.824" 40 | t2 := T2{} 41 | t2.T2slice = []T1{T1{}, t1} 42 | t2.T2map = map[int]*T1{} 43 | t2.T2map[99] = &T1{1, 2, "x", "y"} 44 | t2.T2t3 = T3{999} 45 | 46 | e := NewEncoder(w) 47 | e.Encode(x0) 48 | e.Encode(x1) 49 | e.Encode(t1) 50 | e.Encode(t2) 51 | } 52 | data := w.Bytes() 53 | 54 | { 55 | var x0 int 56 | var x1 int 57 | var t1 T1 58 | var t2 T2 59 | 60 | r := bytes.NewBuffer(data) 61 | d := NewDecoder(r) 62 | if d.Decode(&x0) != nil || 63 | d.Decode(&x1) != nil || 64 | d.Decode(&t1) != nil || 65 | d.Decode(&t2) != nil { 66 | t.Fatalf("Decode failed") 67 | } 68 | 69 | if x0 != 0 { 70 | t.Fatalf("wrong x0 %v\n", x0) 71 | } 72 | if x1 != 1 { 73 | t.Fatalf("wrong x1 %v\n", x1) 74 | } 75 | if t1.T1int0 != 0 { 76 | t.Fatalf("wrong t1.T1int0 %v\n", t1.T1int0) 77 | } 78 | if t1.T1int1 != 1 { 79 | t.Fatalf("wrong t1.T1int1 %v\n", t1.T1int1) 80 | } 81 | if t1.T1string0 != "" { 82 | t.Fatalf("wrong t1.T1string0 %v\n", t1.T1string0) 83 | } 84 | if t1.T1string1 != "6.824" { 85 | t.Fatalf("wrong t1.T1string1 %v\n", t1.T1string1) 86 | } 87 | if len(t2.T2slice) != 2 { 88 | t.Fatalf("wrong t2.T2slice len %v\n", len(t2.T2slice)) 89 | } 90 | if t2.T2slice[1].T1int1 != 1 { 91 | t.Fatalf("wrong slice value\n") 92 | } 93 | if len(t2.T2map) != 1 { 94 | t.Fatalf("wrong t2.T2map len %v\n", len(t2.T2map)) 95 | } 96 | if t2.T2map[99].T1string1 != "y" { 97 | t.Fatalf("wrong map value\n") 98 | } 99 | t3 := (t2.T2t3).(T3) 100 | if t3.T3int999 != 999 { 101 | t.Fatalf("wrong t2.T2t3.T3int999\n") 102 | } 103 | } 104 | 105 | if errorCount != e0 { 106 | t.Fatalf("there were errors, but should not have been") 107 | } 108 | } 109 | 110 | type T4 struct { 111 | Yes int 112 | no int 113 | } 114 | 115 | // 116 | // make sure we check capitalization 117 | // labgob prints one warning during this test. 118 | // 119 | func TestCapital(t *testing.T) { 120 | e0 := errorCount 121 | 122 | v := []map[*T4]int{} 123 | 124 | w := new(bytes.Buffer) 125 | e := NewEncoder(w) 126 | e.Encode(v) 127 | data := w.Bytes() 128 | 129 | var v1 []map[T4]int 130 | r := bytes.NewBuffer(data) 131 | d := NewDecoder(r) 132 | d.Decode(&v1) 133 | 134 | if errorCount != e0+1 { 135 | t.Fatalf("failed to warn about lower-case field") 136 | } 137 | } 138 | 139 | // 140 | // check that we warn when someone sends a default value over 141 | // RPC but the target into which we're decoding holds a non-default 142 | // value, which GOB seems not to overwrite as you'd expect. 143 | // 144 | // labgob does not print a warning. 145 | // 146 | func TestDefault(t *testing.T) { 147 | e0 := errorCount 148 | 149 | type DD struct { 150 | X int 151 | } 152 | 153 | // send a default value... 154 | dd1 := DD{} 155 | 156 | w := new(bytes.Buffer) 157 | e := NewEncoder(w) 158 | e.Encode(dd1) 159 | data := w.Bytes() 160 | 161 | // and receive it into memory that already 162 | // holds non-default values. 163 | reply := DD{99} 164 | 165 | r := bytes.NewBuffer(data) 166 | d := NewDecoder(r) 167 | d.Decode(&reply) 168 | 169 | if errorCount != e0+1 { 170 | t.Fatalf("failed to warn about decoding into non-default value") 171 | } 172 | } 173 | -------------------------------------------------------------------------------- /src/labrpc/labrpc.go: -------------------------------------------------------------------------------- 1 | package labrpc 2 | 3 | // 4 | // channel-based RPC, for 824 labs. 5 | // 6 | // simulates a network that can lose requests, lose replies, 7 | // delay messages, and entirely disconnect particular hosts. 8 | // 9 | // we will use the original labrpc.go to test your code for grading. 10 | // so, while you can modify this code to help you debug, please 11 | // test against the original before submitting. 12 | // 13 | // adapted from Go net/rpc/server.go. 14 | // 15 | // sends labgob-encoded values to ensure that RPCs 16 | // don't include references to program objects. 17 | // 18 | // net := MakeNetwork() -- holds network, clients, servers. 19 | // end := net.MakeEnd(endname) -- create a client end-point, to talk to one server. 20 | // net.AddServer(servername, server) -- adds a named server to network. 21 | // net.DeleteServer(servername) -- eliminate the named server. 22 | // net.Connect(endname, servername) -- connect a client to a server. 23 | // net.Enable(endname, enabled) -- enable/disable a client. 24 | // net.Reliable(bool) -- false means drop/delay messages 25 | // 26 | // end.Call("Raft.AppendEntries", &args, &reply) -- send an RPC, wait for reply. 27 | // the "Raft" is the name of the server struct to be called. 28 | // the "AppendEntries" is the name of the method to be called. 29 | // Call() returns true to indicate that the server executed the request 30 | // and the reply is valid. 31 | // Call() returns false if the network lost the request or reply 32 | // or the server is down. 33 | // It is OK to have multiple Call()s in progress at the same time on the 34 | // same ClientEnd. 35 | // Concurrent calls to Call() may be delivered to the server out of order, 36 | // since the network may re-order messages. 37 | // Call() is guaranteed to return (perhaps after a delay) *except* if the 38 | // handler function on the server side does not return. 39 | // the server RPC handler function must declare its args and reply arguments 40 | // as pointers, so that their types exactly match the types of the arguments 41 | // to Call(). 42 | // 43 | // srv := MakeServer() 44 | // srv.AddService(svc) -- a server can have multiple services, e.g. Raft and k/v 45 | // pass srv to net.AddServer() 46 | // 47 | // svc := MakeService(receiverObject) -- obj's methods will handle RPCs 48 | // much like Go's rpcs.Register() 49 | // pass svc to srv.AddService() 50 | // 51 | 52 | import "6.824/labgob" 53 | import "bytes" 54 | import "reflect" 55 | import "sync" 56 | import "log" 57 | import "strings" 58 | import "math/rand" 59 | import "time" 60 | import "sync/atomic" 61 | 62 | type reqMsg struct { 63 | endname interface{} // name of sending ClientEnd 64 | svcMeth string // e.g. "Raft.AppendEntries" 65 | argsType reflect.Type 66 | args []byte 67 | replyCh chan replyMsg 68 | } 69 | 70 | type replyMsg struct { 71 | ok bool 72 | reply []byte 73 | } 74 | 75 | type ClientEnd struct { 76 | endname interface{} // this end-point's name 77 | ch chan reqMsg // copy of Network.endCh 78 | done chan struct{} // closed when Network is cleaned up 79 | } 80 | 81 | // send an RPC, wait for the reply. 82 | // the return value indicates success; false means that 83 | // no reply was received from the server. 84 | func (e *ClientEnd) Call(svcMeth string, args interface{}, reply interface{}) bool { 85 | req := reqMsg{} 86 | req.endname = e.endname 87 | req.svcMeth = svcMeth 88 | req.argsType = reflect.TypeOf(args) 89 | req.replyCh = make(chan replyMsg) 90 | 91 | qb := new(bytes.Buffer) 92 | qe := labgob.NewEncoder(qb) 93 | if err := qe.Encode(args); err != nil { 94 | panic(err) 95 | } 96 | req.args = qb.Bytes() 97 | 98 | // 99 | // send the request. 100 | // 101 | select { 102 | case e.ch <- req: 103 | // the request has been sent. 104 | case <-e.done: 105 | // entire Network has been destroyed. 106 | return false 107 | } 108 | 109 | // 110 | // wait for the reply. 111 | // 112 | rep := <-req.replyCh 113 | if rep.ok { 114 | rb := bytes.NewBuffer(rep.reply) 115 | rd := labgob.NewDecoder(rb) 116 | if err := rd.Decode(reply); err != nil { 117 | log.Fatalf("ClientEnd.Call(): decode reply: %v\n", err) 118 | } 119 | return true 120 | } else { 121 | return false 122 | } 123 | } 124 | 125 | type Network struct { 126 | mu sync.Mutex 127 | reliable bool 128 | longDelays bool // pause a long time on send on disabled connection 129 | longReordering bool // sometimes delay replies a long time 130 | ends map[interface{}]*ClientEnd // ends, by name 131 | enabled map[interface{}]bool // by end name 132 | servers map[interface{}]*Server // servers, by name 133 | connections map[interface{}]interface{} // endname -> servername 134 | endCh chan reqMsg 135 | done chan struct{} // closed when Network is cleaned up 136 | count int32 // total RPC count, for statistics 137 | bytes int64 // total bytes send, for statistics 138 | } 139 | 140 | func MakeNetwork() *Network { 141 | rn := &Network{} 142 | rn.reliable = true 143 | rn.ends = map[interface{}]*ClientEnd{} 144 | rn.enabled = map[interface{}]bool{} 145 | rn.servers = map[interface{}]*Server{} 146 | rn.connections = map[interface{}](interface{}){} 147 | rn.endCh = make(chan reqMsg) 148 | rn.done = make(chan struct{}) 149 | 150 | // single goroutine to handle all ClientEnd.Call()s 151 | go func() { 152 | for { 153 | select { 154 | case xreq := <-rn.endCh: 155 | atomic.AddInt32(&rn.count, 1) 156 | atomic.AddInt64(&rn.bytes, int64(len(xreq.args))) 157 | go rn.processReq(xreq) 158 | case <-rn.done: 159 | return 160 | } 161 | } 162 | }() 163 | 164 | return rn 165 | } 166 | 167 | func (rn *Network) Cleanup() { 168 | close(rn.done) 169 | } 170 | 171 | func (rn *Network) Reliable(yes bool) { 172 | rn.mu.Lock() 173 | defer rn.mu.Unlock() 174 | 175 | rn.reliable = yes 176 | } 177 | 178 | func (rn *Network) LongReordering(yes bool) { 179 | rn.mu.Lock() 180 | defer rn.mu.Unlock() 181 | 182 | rn.longReordering = yes 183 | } 184 | 185 | func (rn *Network) LongDelays(yes bool) { 186 | rn.mu.Lock() 187 | defer rn.mu.Unlock() 188 | 189 | rn.longDelays = yes 190 | } 191 | 192 | func (rn *Network) readEndnameInfo(endname interface{}) (enabled bool, 193 | servername interface{}, server *Server, reliable bool, longreordering bool, 194 | ) { 195 | rn.mu.Lock() 196 | defer rn.mu.Unlock() 197 | 198 | enabled = rn.enabled[endname] 199 | servername = rn.connections[endname] 200 | if servername != nil { 201 | server = rn.servers[servername] 202 | } 203 | reliable = rn.reliable 204 | longreordering = rn.longReordering 205 | return 206 | } 207 | 208 | func (rn *Network) isServerDead(endname interface{}, servername interface{}, server *Server) bool { 209 | rn.mu.Lock() 210 | defer rn.mu.Unlock() 211 | 212 | if rn.enabled[endname] == false || rn.servers[servername] != server { 213 | return true 214 | } 215 | return false 216 | } 217 | 218 | func (rn *Network) processReq(req reqMsg) { 219 | enabled, servername, server, reliable, longreordering := rn.readEndnameInfo(req.endname) 220 | 221 | if enabled && servername != nil && server != nil { 222 | if reliable == false { 223 | // short delay 224 | ms := (rand.Int() % 27) 225 | time.Sleep(time.Duration(ms) * time.Millisecond) 226 | } 227 | 228 | if reliable == false && (rand.Int()%1000) < 100 { 229 | // drop the request, return as if timeout 230 | req.replyCh <- replyMsg{false, nil} 231 | return 232 | } 233 | 234 | // execute the request (call the RPC handler). 235 | // in a separate thread so that we can periodically check 236 | // if the server has been killed and the RPC should get a 237 | // failure reply. 238 | ech := make(chan replyMsg) 239 | go func() { 240 | r := server.dispatch(req) 241 | ech <- r 242 | }() 243 | 244 | // wait for handler to return, 245 | // but stop waiting if DeleteServer() has been called, 246 | // and return an error. 247 | var reply replyMsg 248 | replyOK := false 249 | serverDead := false 250 | for replyOK == false && serverDead == false { 251 | select { 252 | case reply = <-ech: 253 | replyOK = true 254 | case <-time.After(100 * time.Millisecond): 255 | serverDead = rn.isServerDead(req.endname, servername, server) 256 | if serverDead { 257 | go func() { 258 | <-ech // drain channel to let the goroutine created earlier terminate 259 | }() 260 | } 261 | } 262 | } 263 | 264 | // do not reply if DeleteServer() has been called, i.e. 265 | // the server has been killed. this is needed to avoid 266 | // situation in which a client gets a positive reply 267 | // to an Append, but the server persisted the update 268 | // into the old Persister. config.go is careful to call 269 | // DeleteServer() before superseding the Persister. 270 | serverDead = rn.isServerDead(req.endname, servername, server) 271 | 272 | if replyOK == false || serverDead == true { 273 | // server was killed while we were waiting; return error. 274 | req.replyCh <- replyMsg{false, nil} 275 | } else if reliable == false && (rand.Int()%1000) < 100 { 276 | // drop the reply, return as if timeout 277 | req.replyCh <- replyMsg{false, nil} 278 | } else if longreordering == true && rand.Intn(900) < 600 { 279 | // delay the response for a while 280 | ms := 200 + rand.Intn(1+rand.Intn(2000)) 281 | // Russ points out that this timer arrangement will decrease 282 | // the number of goroutines, so that the race 283 | // detector is less likely to get upset. 284 | time.AfterFunc(time.Duration(ms)*time.Millisecond, func() { 285 | atomic.AddInt64(&rn.bytes, int64(len(reply.reply))) 286 | req.replyCh <- reply 287 | }) 288 | } else { 289 | atomic.AddInt64(&rn.bytes, int64(len(reply.reply))) 290 | req.replyCh <- reply 291 | } 292 | } else { 293 | // simulate no reply and eventual timeout. 294 | ms := 0 295 | if rn.longDelays { 296 | // let Raft tests check that leader doesn't send 297 | // RPCs synchronously. 298 | ms = (rand.Int() % 7000) 299 | } else { 300 | // many kv tests require the client to try each 301 | // server in fairly rapid succession. 302 | ms = (rand.Int() % 100) 303 | } 304 | time.AfterFunc(time.Duration(ms)*time.Millisecond, func() { 305 | req.replyCh <- replyMsg{false, nil} 306 | }) 307 | } 308 | 309 | } 310 | 311 | // create a client end-point. 312 | // start the thread that listens and delivers. 313 | func (rn *Network) MakeEnd(endname interface{}) *ClientEnd { 314 | rn.mu.Lock() 315 | defer rn.mu.Unlock() 316 | 317 | if _, ok := rn.ends[endname]; ok { 318 | log.Fatalf("MakeEnd: %v already exists\n", endname) 319 | } 320 | 321 | e := &ClientEnd{} 322 | e.endname = endname 323 | e.ch = rn.endCh 324 | e.done = rn.done 325 | rn.ends[endname] = e 326 | rn.enabled[endname] = false 327 | rn.connections[endname] = nil 328 | 329 | return e 330 | } 331 | 332 | func (rn *Network) AddServer(servername interface{}, rs *Server) { 333 | rn.mu.Lock() 334 | defer rn.mu.Unlock() 335 | 336 | rn.servers[servername] = rs 337 | } 338 | 339 | func (rn *Network) DeleteServer(servername interface{}) { 340 | rn.mu.Lock() 341 | defer rn.mu.Unlock() 342 | 343 | rn.servers[servername] = nil 344 | } 345 | 346 | // connect a ClientEnd to a server. 347 | // a ClientEnd can only be connected once in its lifetime. 348 | func (rn *Network) Connect(endname interface{}, servername interface{}) { 349 | rn.mu.Lock() 350 | defer rn.mu.Unlock() 351 | 352 | rn.connections[endname] = servername 353 | } 354 | 355 | // enable/disable a ClientEnd. 356 | func (rn *Network) Enable(endname interface{}, enabled bool) { 357 | rn.mu.Lock() 358 | defer rn.mu.Unlock() 359 | 360 | rn.enabled[endname] = enabled 361 | } 362 | 363 | // get a server's count of incoming RPCs. 364 | func (rn *Network) GetCount(servername interface{}) int { 365 | rn.mu.Lock() 366 | defer rn.mu.Unlock() 367 | 368 | svr := rn.servers[servername] 369 | return svr.GetCount() 370 | } 371 | 372 | func (rn *Network) GetTotalCount() int { 373 | x := atomic.LoadInt32(&rn.count) 374 | return int(x) 375 | } 376 | 377 | func (rn *Network) GetTotalBytes() int64 { 378 | x := atomic.LoadInt64(&rn.bytes) 379 | return x 380 | } 381 | 382 | // a server is a collection of services, all sharing 383 | // the same rpc dispatcher. so that e.g. both a Raft 384 | // and a k/v server can listen to the same rpc endpoint. 385 | type Server struct { 386 | mu sync.Mutex 387 | services map[string]*Service 388 | count int // incoming RPCs 389 | } 390 | 391 | func MakeServer() *Server { 392 | rs := &Server{} 393 | rs.services = map[string]*Service{} 394 | return rs 395 | } 396 | 397 | func (rs *Server) AddService(svc *Service) { 398 | rs.mu.Lock() 399 | defer rs.mu.Unlock() 400 | rs.services[svc.name] = svc 401 | } 402 | 403 | func (rs *Server) dispatch(req reqMsg) replyMsg { 404 | rs.mu.Lock() 405 | 406 | rs.count += 1 407 | 408 | // split Raft.AppendEntries into service and method 409 | dot := strings.LastIndex(req.svcMeth, ".") 410 | serviceName := req.svcMeth[:dot] 411 | methodName := req.svcMeth[dot+1:] 412 | 413 | service, ok := rs.services[serviceName] 414 | 415 | rs.mu.Unlock() 416 | 417 | if ok { 418 | return service.dispatch(methodName, req) 419 | } else { 420 | choices := []string{} 421 | for k, _ := range rs.services { 422 | choices = append(choices, k) 423 | } 424 | log.Fatalf("labrpc.Server.dispatch(): unknown service %v in %v.%v; expecting one of %v\n", 425 | serviceName, serviceName, methodName, choices) 426 | return replyMsg{false, nil} 427 | } 428 | } 429 | 430 | func (rs *Server) GetCount() int { 431 | rs.mu.Lock() 432 | defer rs.mu.Unlock() 433 | return rs.count 434 | } 435 | 436 | // an object with methods that can be called via RPC. 437 | // a single server may have more than one Service. 438 | type Service struct { 439 | name string 440 | rcvr reflect.Value 441 | typ reflect.Type 442 | methods map[string]reflect.Method 443 | } 444 | 445 | func MakeService(rcvr interface{}) *Service { 446 | svc := &Service{} 447 | svc.typ = reflect.TypeOf(rcvr) 448 | svc.rcvr = reflect.ValueOf(rcvr) 449 | svc.name = reflect.Indirect(svc.rcvr).Type().Name() 450 | svc.methods = map[string]reflect.Method{} 451 | 452 | for m := 0; m < svc.typ.NumMethod(); m++ { 453 | method := svc.typ.Method(m) 454 | mtype := method.Type 455 | mname := method.Name 456 | 457 | //fmt.Printf("%v pp %v ni %v 1k %v 2k %v no %v\n", 458 | // mname, method.PkgPath, mtype.NumIn(), mtype.In(1).Kind(), mtype.In(2).Kind(), mtype.NumOut()) 459 | 460 | if method.PkgPath != "" || // capitalized? 461 | mtype.NumIn() != 3 || 462 | //mtype.In(1).Kind() != reflect.Ptr || 463 | mtype.In(2).Kind() != reflect.Ptr || 464 | mtype.NumOut() != 0 { 465 | // the method is not suitable for a handler 466 | //fmt.Printf("bad method: %v\n", mname) 467 | } else { 468 | // the method looks like a handler 469 | svc.methods[mname] = method 470 | } 471 | } 472 | 473 | return svc 474 | } 475 | 476 | func (svc *Service) dispatch(methname string, req reqMsg) replyMsg { 477 | if method, ok := svc.methods[methname]; ok { 478 | // prepare space into which to read the argument. 479 | // the Value's type will be a pointer to req.argsType. 480 | args := reflect.New(req.argsType) 481 | 482 | // decode the argument. 483 | ab := bytes.NewBuffer(req.args) 484 | ad := labgob.NewDecoder(ab) 485 | ad.Decode(args.Interface()) 486 | 487 | // allocate space for the reply. 488 | replyType := method.Type.In(2) 489 | replyType = replyType.Elem() 490 | replyv := reflect.New(replyType) 491 | 492 | // call the method. 493 | function := method.Func 494 | function.Call([]reflect.Value{svc.rcvr, args.Elem(), replyv}) 495 | 496 | // encode the reply. 497 | rb := new(bytes.Buffer) 498 | re := labgob.NewEncoder(rb) 499 | re.EncodeValue(replyv) 500 | 501 | return replyMsg{true, rb.Bytes()} 502 | } else { 503 | choices := []string{} 504 | for k, _ := range svc.methods { 505 | choices = append(choices, k) 506 | } 507 | log.Fatalf("labrpc.Service.dispatch(): unknown method %v in %v; expecting one of %v\n", 508 | methname, req.svcMeth, choices) 509 | return replyMsg{false, nil} 510 | } 511 | } 512 | -------------------------------------------------------------------------------- /src/labrpc/test_test.go: -------------------------------------------------------------------------------- 1 | package labrpc 2 | 3 | import "testing" 4 | import "strconv" 5 | import "sync" 6 | import "runtime" 7 | import "time" 8 | import "fmt" 9 | 10 | type JunkArgs struct { 11 | X int 12 | } 13 | type JunkReply struct { 14 | X string 15 | } 16 | 17 | type JunkServer struct { 18 | mu sync.Mutex 19 | log1 []string 20 | log2 []int 21 | } 22 | 23 | func (js *JunkServer) Handler1(args string, reply *int) { 24 | js.mu.Lock() 25 | defer js.mu.Unlock() 26 | js.log1 = append(js.log1, args) 27 | *reply, _ = strconv.Atoi(args) 28 | } 29 | 30 | func (js *JunkServer) Handler2(args int, reply *string) { 31 | js.mu.Lock() 32 | defer js.mu.Unlock() 33 | js.log2 = append(js.log2, args) 34 | *reply = "handler2-" + strconv.Itoa(args) 35 | } 36 | 37 | func (js *JunkServer) Handler3(args int, reply *int) { 38 | js.mu.Lock() 39 | defer js.mu.Unlock() 40 | time.Sleep(20 * time.Second) 41 | *reply = -args 42 | } 43 | 44 | // args is a pointer 45 | func (js *JunkServer) Handler4(args *JunkArgs, reply *JunkReply) { 46 | reply.X = "pointer" 47 | } 48 | 49 | // args is a not pointer 50 | func (js *JunkServer) Handler5(args JunkArgs, reply *JunkReply) { 51 | reply.X = "no pointer" 52 | } 53 | 54 | func (js *JunkServer) Handler6(args string, reply *int) { 55 | js.mu.Lock() 56 | defer js.mu.Unlock() 57 | *reply = len(args) 58 | } 59 | 60 | func (js *JunkServer) Handler7(args int, reply *string) { 61 | js.mu.Lock() 62 | defer js.mu.Unlock() 63 | *reply = "" 64 | for i := 0; i < args; i++ { 65 | *reply = *reply + "y" 66 | } 67 | } 68 | 69 | func TestBasic(t *testing.T) { 70 | runtime.GOMAXPROCS(4) 71 | 72 | rn := MakeNetwork() 73 | defer rn.Cleanup() 74 | 75 | e := rn.MakeEnd("end1-99") 76 | 77 | js := &JunkServer{} 78 | svc := MakeService(js) 79 | 80 | rs := MakeServer() 81 | rs.AddService(svc) 82 | rn.AddServer("server99", rs) 83 | 84 | rn.Connect("end1-99", "server99") 85 | rn.Enable("end1-99", true) 86 | 87 | { 88 | reply := "" 89 | e.Call("JunkServer.Handler2", 111, &reply) 90 | if reply != "handler2-111" { 91 | t.Fatalf("wrong reply from Handler2") 92 | } 93 | } 94 | 95 | { 96 | reply := 0 97 | e.Call("JunkServer.Handler1", "9099", &reply) 98 | if reply != 9099 { 99 | t.Fatalf("wrong reply from Handler1") 100 | } 101 | } 102 | } 103 | 104 | func TestTypes(t *testing.T) { 105 | runtime.GOMAXPROCS(4) 106 | 107 | rn := MakeNetwork() 108 | defer rn.Cleanup() 109 | 110 | e := rn.MakeEnd("end1-99") 111 | 112 | js := &JunkServer{} 113 | svc := MakeService(js) 114 | 115 | rs := MakeServer() 116 | rs.AddService(svc) 117 | rn.AddServer("server99", rs) 118 | 119 | rn.Connect("end1-99", "server99") 120 | rn.Enable("end1-99", true) 121 | 122 | { 123 | var args JunkArgs 124 | var reply JunkReply 125 | // args must match type (pointer or not) of handler. 126 | e.Call("JunkServer.Handler4", &args, &reply) 127 | if reply.X != "pointer" { 128 | t.Fatalf("wrong reply from Handler4") 129 | } 130 | } 131 | 132 | { 133 | var args JunkArgs 134 | var reply JunkReply 135 | // args must match type (pointer or not) of handler. 136 | e.Call("JunkServer.Handler5", args, &reply) 137 | if reply.X != "no pointer" { 138 | t.Fatalf("wrong reply from Handler5") 139 | } 140 | } 141 | } 142 | 143 | // 144 | // does net.Enable(endname, false) really disconnect a client? 145 | // 146 | func TestDisconnect(t *testing.T) { 147 | runtime.GOMAXPROCS(4) 148 | 149 | rn := MakeNetwork() 150 | defer rn.Cleanup() 151 | 152 | e := rn.MakeEnd("end1-99") 153 | 154 | js := &JunkServer{} 155 | svc := MakeService(js) 156 | 157 | rs := MakeServer() 158 | rs.AddService(svc) 159 | rn.AddServer("server99", rs) 160 | 161 | rn.Connect("end1-99", "server99") 162 | 163 | { 164 | reply := "" 165 | e.Call("JunkServer.Handler2", 111, &reply) 166 | if reply != "" { 167 | t.Fatalf("unexpected reply from Handler2") 168 | } 169 | } 170 | 171 | rn.Enable("end1-99", true) 172 | 173 | { 174 | reply := 0 175 | e.Call("JunkServer.Handler1", "9099", &reply) 176 | if reply != 9099 { 177 | t.Fatalf("wrong reply from Handler1") 178 | } 179 | } 180 | } 181 | 182 | // 183 | // test net.GetCount() 184 | // 185 | func TestCounts(t *testing.T) { 186 | runtime.GOMAXPROCS(4) 187 | 188 | rn := MakeNetwork() 189 | defer rn.Cleanup() 190 | 191 | e := rn.MakeEnd("end1-99") 192 | 193 | js := &JunkServer{} 194 | svc := MakeService(js) 195 | 196 | rs := MakeServer() 197 | rs.AddService(svc) 198 | rn.AddServer(99, rs) 199 | 200 | rn.Connect("end1-99", 99) 201 | rn.Enable("end1-99", true) 202 | 203 | for i := 0; i < 17; i++ { 204 | reply := "" 205 | e.Call("JunkServer.Handler2", i, &reply) 206 | wanted := "handler2-" + strconv.Itoa(i) 207 | if reply != wanted { 208 | t.Fatalf("wrong reply %v from Handler1, expecting %v", reply, wanted) 209 | } 210 | } 211 | 212 | n := rn.GetCount(99) 213 | if n != 17 { 214 | t.Fatalf("wrong GetCount() %v, expected 17\n", n) 215 | } 216 | } 217 | 218 | // 219 | // test net.GetTotalBytes() 220 | // 221 | func TestBytes(t *testing.T) { 222 | runtime.GOMAXPROCS(4) 223 | 224 | rn := MakeNetwork() 225 | defer rn.Cleanup() 226 | 227 | e := rn.MakeEnd("end1-99") 228 | 229 | js := &JunkServer{} 230 | svc := MakeService(js) 231 | 232 | rs := MakeServer() 233 | rs.AddService(svc) 234 | rn.AddServer(99, rs) 235 | 236 | rn.Connect("end1-99", 99) 237 | rn.Enable("end1-99", true) 238 | 239 | for i := 0; i < 17; i++ { 240 | args := "xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxx" 241 | args = args + args 242 | args = args + args 243 | reply := 0 244 | e.Call("JunkServer.Handler6", args, &reply) 245 | wanted := len(args) 246 | if reply != wanted { 247 | t.Fatalf("wrong reply %v from Handler6, expecting %v", reply, wanted) 248 | } 249 | } 250 | 251 | n := rn.GetTotalBytes() 252 | if n < 4828 || n > 6000 { 253 | t.Fatalf("wrong GetTotalBytes() %v, expected about 5000\n", n) 254 | } 255 | 256 | for i := 0; i < 17; i++ { 257 | args := 107 258 | reply := "" 259 | e.Call("JunkServer.Handler7", args, &reply) 260 | wanted := args 261 | if len(reply) != wanted { 262 | t.Fatalf("wrong reply len=%v from Handler6, expecting %v", len(reply), wanted) 263 | } 264 | } 265 | 266 | nn := rn.GetTotalBytes() - n 267 | if nn < 1800 || nn > 2500 { 268 | t.Fatalf("wrong GetTotalBytes() %v, expected about 2000\n", nn) 269 | } 270 | } 271 | 272 | // 273 | // test RPCs from concurrent ClientEnds 274 | // 275 | func TestConcurrentMany(t *testing.T) { 276 | runtime.GOMAXPROCS(4) 277 | 278 | rn := MakeNetwork() 279 | defer rn.Cleanup() 280 | 281 | js := &JunkServer{} 282 | svc := MakeService(js) 283 | 284 | rs := MakeServer() 285 | rs.AddService(svc) 286 | rn.AddServer(1000, rs) 287 | 288 | ch := make(chan int) 289 | 290 | nclients := 20 291 | nrpcs := 10 292 | for ii := 0; ii < nclients; ii++ { 293 | go func(i int) { 294 | n := 0 295 | defer func() { ch <- n }() 296 | 297 | e := rn.MakeEnd(i) 298 | rn.Connect(i, 1000) 299 | rn.Enable(i, true) 300 | 301 | for j := 0; j < nrpcs; j++ { 302 | arg := i*100 + j 303 | reply := "" 304 | e.Call("JunkServer.Handler2", arg, &reply) 305 | wanted := "handler2-" + strconv.Itoa(arg) 306 | if reply != wanted { 307 | t.Fatalf("wrong reply %v from Handler1, expecting %v", reply, wanted) 308 | } 309 | n += 1 310 | } 311 | }(ii) 312 | } 313 | 314 | total := 0 315 | for ii := 0; ii < nclients; ii++ { 316 | x := <-ch 317 | total += x 318 | } 319 | 320 | if total != nclients*nrpcs { 321 | t.Fatalf("wrong number of RPCs completed, got %v, expected %v", total, nclients*nrpcs) 322 | } 323 | 324 | n := rn.GetCount(1000) 325 | if n != total { 326 | t.Fatalf("wrong GetCount() %v, expected %v\n", n, total) 327 | } 328 | } 329 | 330 | // 331 | // test unreliable 332 | // 333 | func TestUnreliable(t *testing.T) { 334 | runtime.GOMAXPROCS(4) 335 | 336 | rn := MakeNetwork() 337 | defer rn.Cleanup() 338 | rn.Reliable(false) 339 | 340 | js := &JunkServer{} 341 | svc := MakeService(js) 342 | 343 | rs := MakeServer() 344 | rs.AddService(svc) 345 | rn.AddServer(1000, rs) 346 | 347 | ch := make(chan int) 348 | 349 | nclients := 300 350 | for ii := 0; ii < nclients; ii++ { 351 | go func(i int) { 352 | n := 0 353 | defer func() { ch <- n }() 354 | 355 | e := rn.MakeEnd(i) 356 | rn.Connect(i, 1000) 357 | rn.Enable(i, true) 358 | 359 | arg := i * 100 360 | reply := "" 361 | ok := e.Call("JunkServer.Handler2", arg, &reply) 362 | if ok { 363 | wanted := "handler2-" + strconv.Itoa(arg) 364 | if reply != wanted { 365 | t.Fatalf("wrong reply %v from Handler1, expecting %v", reply, wanted) 366 | } 367 | n += 1 368 | } 369 | }(ii) 370 | } 371 | 372 | total := 0 373 | for ii := 0; ii < nclients; ii++ { 374 | x := <-ch 375 | total += x 376 | } 377 | 378 | if total == nclients || total == 0 { 379 | t.Fatalf("all RPCs succeeded despite unreliable") 380 | } 381 | } 382 | 383 | // 384 | // test concurrent RPCs from a single ClientEnd 385 | // 386 | func TestConcurrentOne(t *testing.T) { 387 | runtime.GOMAXPROCS(4) 388 | 389 | rn := MakeNetwork() 390 | defer rn.Cleanup() 391 | 392 | js := &JunkServer{} 393 | svc := MakeService(js) 394 | 395 | rs := MakeServer() 396 | rs.AddService(svc) 397 | rn.AddServer(1000, rs) 398 | 399 | e := rn.MakeEnd("c") 400 | rn.Connect("c", 1000) 401 | rn.Enable("c", true) 402 | 403 | ch := make(chan int) 404 | 405 | nrpcs := 20 406 | for ii := 0; ii < nrpcs; ii++ { 407 | go func(i int) { 408 | n := 0 409 | defer func() { ch <- n }() 410 | 411 | arg := 100 + i 412 | reply := "" 413 | e.Call("JunkServer.Handler2", arg, &reply) 414 | wanted := "handler2-" + strconv.Itoa(arg) 415 | if reply != wanted { 416 | t.Fatalf("wrong reply %v from Handler2, expecting %v", reply, wanted) 417 | } 418 | n += 1 419 | }(ii) 420 | } 421 | 422 | total := 0 423 | for ii := 0; ii < nrpcs; ii++ { 424 | x := <-ch 425 | total += x 426 | } 427 | 428 | if total != nrpcs { 429 | t.Fatalf("wrong number of RPCs completed, got %v, expected %v", total, nrpcs) 430 | } 431 | 432 | js.mu.Lock() 433 | defer js.mu.Unlock() 434 | if len(js.log2) != nrpcs { 435 | t.Fatalf("wrong number of RPCs delivered") 436 | } 437 | 438 | n := rn.GetCount(1000) 439 | if n != total { 440 | t.Fatalf("wrong GetCount() %v, expected %v\n", n, total) 441 | } 442 | } 443 | 444 | // 445 | // regression: an RPC that's delayed during Enabled=false 446 | // should not delay subsequent RPCs (e.g. after Enabled=true). 447 | // 448 | func TestRegression1(t *testing.T) { 449 | runtime.GOMAXPROCS(4) 450 | 451 | rn := MakeNetwork() 452 | defer rn.Cleanup() 453 | 454 | js := &JunkServer{} 455 | svc := MakeService(js) 456 | 457 | rs := MakeServer() 458 | rs.AddService(svc) 459 | rn.AddServer(1000, rs) 460 | 461 | e := rn.MakeEnd("c") 462 | rn.Connect("c", 1000) 463 | 464 | // start some RPCs while the ClientEnd is disabled. 465 | // they'll be delayed. 466 | rn.Enable("c", false) 467 | ch := make(chan bool) 468 | nrpcs := 20 469 | for ii := 0; ii < nrpcs; ii++ { 470 | go func(i int) { 471 | ok := false 472 | defer func() { ch <- ok }() 473 | 474 | arg := 100 + i 475 | reply := "" 476 | // this call ought to return false. 477 | e.Call("JunkServer.Handler2", arg, &reply) 478 | ok = true 479 | }(ii) 480 | } 481 | 482 | time.Sleep(100 * time.Millisecond) 483 | 484 | // now enable the ClientEnd and check that an RPC completes quickly. 485 | t0 := time.Now() 486 | rn.Enable("c", true) 487 | { 488 | arg := 99 489 | reply := "" 490 | e.Call("JunkServer.Handler2", arg, &reply) 491 | wanted := "handler2-" + strconv.Itoa(arg) 492 | if reply != wanted { 493 | t.Fatalf("wrong reply %v from Handler2, expecting %v", reply, wanted) 494 | } 495 | } 496 | dur := time.Since(t0).Seconds() 497 | 498 | if dur > 0.03 { 499 | t.Fatalf("RPC took too long (%v) after Enable", dur) 500 | } 501 | 502 | for ii := 0; ii < nrpcs; ii++ { 503 | <-ch 504 | } 505 | 506 | js.mu.Lock() 507 | defer js.mu.Unlock() 508 | if len(js.log2) != 1 { 509 | t.Fatalf("wrong number (%v) of RPCs delivered, expected 1", len(js.log2)) 510 | } 511 | 512 | n := rn.GetCount(1000) 513 | if n != 1 { 514 | t.Fatalf("wrong GetCount() %v, expected %v\n", n, 1) 515 | } 516 | } 517 | 518 | // 519 | // if an RPC is stuck in a server, and the server 520 | // is killed with DeleteServer(), does the RPC 521 | // get un-stuck? 522 | // 523 | func TestKilled(t *testing.T) { 524 | runtime.GOMAXPROCS(4) 525 | 526 | rn := MakeNetwork() 527 | defer rn.Cleanup() 528 | 529 | e := rn.MakeEnd("end1-99") 530 | 531 | js := &JunkServer{} 532 | svc := MakeService(js) 533 | 534 | rs := MakeServer() 535 | rs.AddService(svc) 536 | rn.AddServer("server99", rs) 537 | 538 | rn.Connect("end1-99", "server99") 539 | rn.Enable("end1-99", true) 540 | 541 | doneCh := make(chan bool) 542 | go func() { 543 | reply := 0 544 | ok := e.Call("JunkServer.Handler3", 99, &reply) 545 | doneCh <- ok 546 | }() 547 | 548 | time.Sleep(1000 * time.Millisecond) 549 | 550 | select { 551 | case <-doneCh: 552 | t.Fatalf("Handler3 should not have returned yet") 553 | case <-time.After(100 * time.Millisecond): 554 | } 555 | 556 | rn.DeleteServer("server99") 557 | 558 | select { 559 | case x := <-doneCh: 560 | if x != false { 561 | t.Fatalf("Handler3 returned successfully despite DeleteServer()") 562 | } 563 | case <-time.After(100 * time.Millisecond): 564 | t.Fatalf("Handler3 should return after DeleteServer()") 565 | } 566 | } 567 | 568 | func TestBenchmark(t *testing.T) { 569 | runtime.GOMAXPROCS(4) 570 | 571 | rn := MakeNetwork() 572 | defer rn.Cleanup() 573 | 574 | e := rn.MakeEnd("end1-99") 575 | 576 | js := &JunkServer{} 577 | svc := MakeService(js) 578 | 579 | rs := MakeServer() 580 | rs.AddService(svc) 581 | rn.AddServer("server99", rs) 582 | 583 | rn.Connect("end1-99", "server99") 584 | rn.Enable("end1-99", true) 585 | 586 | t0 := time.Now() 587 | n := 100000 588 | for iters := 0; iters < n; iters++ { 589 | reply := "" 590 | e.Call("JunkServer.Handler2", 111, &reply) 591 | if reply != "handler2-111" { 592 | t.Fatalf("wrong reply from Handler2") 593 | } 594 | } 595 | fmt.Printf("%v for %v\n", time.Since(t0), n) 596 | // march 2016, rtm laptop, 22 microseconds per RPC 597 | } 598 | -------------------------------------------------------------------------------- /src/raft/append_entry.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | type AppendEntriesArgs struct { 4 | Term int 5 | LeaderId int 6 | PrevLogIndex int 7 | PrevLogTerm int 8 | Entries []Entry 9 | LeaderCommit int 10 | } 11 | 12 | type AppendEntriesReply struct { 13 | Term int 14 | Success bool 15 | Conflict bool 16 | XTerm int 17 | XIndex int 18 | XLen int 19 | } 20 | 21 | func (rf *Raft) appendEntries(heartbeat bool) { 22 | lastLog := rf.log.lastLog() 23 | for peer, _ := range rf.peers { 24 | if peer == rf.me { 25 | rf.resetElectionTimer() 26 | continue 27 | } 28 | // rules for leader 3 29 | if lastLog.Index >= rf.nextIndex[peer] || heartbeat { 30 | nextIndex := rf.nextIndex[peer] 31 | if nextIndex <= 0 { 32 | nextIndex = 1 33 | } 34 | if lastLog.Index+1 < nextIndex { 35 | nextIndex = lastLog.Index 36 | } 37 | prevLog := rf.log.at(nextIndex - 1) 38 | args := AppendEntriesArgs{ 39 | Term: rf.currentTerm, 40 | LeaderId: rf.me, 41 | PrevLogIndex: prevLog.Index, 42 | PrevLogTerm: prevLog.Term, 43 | Entries: make([]Entry, lastLog.Index-nextIndex+1), 44 | LeaderCommit: rf.commitIndex, 45 | } 46 | copy(args.Entries, rf.log.slice(nextIndex)) 47 | go rf.leaderSendEntries(peer, &args) 48 | } 49 | } 50 | } 51 | 52 | func (rf *Raft) leaderSendEntries(serverId int, args *AppendEntriesArgs) { 53 | var reply AppendEntriesReply 54 | ok := rf.sendAppendEntries(serverId, args, &reply) 55 | if !ok { 56 | return 57 | } 58 | rf.mu.Lock() 59 | defer rf.mu.Unlock() 60 | if reply.Term > rf.currentTerm { 61 | rf.setNewTerm(reply.Term) 62 | return 63 | } 64 | if args.Term == rf.currentTerm { 65 | // rules for leader 3.1 66 | if reply.Success { 67 | match := args.PrevLogIndex + len(args.Entries) 68 | next := match + 1 69 | rf.nextIndex[serverId] = max(rf.nextIndex[serverId], next) 70 | rf.matchIndex[serverId] = max(rf.matchIndex[serverId], match) 71 | DPrintf("[%v]: %v append success next %v match %v", rf.me, serverId, rf.nextIndex[serverId], rf.matchIndex[serverId]) 72 | } else if reply.Conflict { 73 | DPrintf("[%v]: Conflict from %v %#v", rf.me, serverId, reply) 74 | if reply.XTerm == -1 { 75 | rf.nextIndex[serverId] = reply.XLen 76 | } else { 77 | lastLogInXTerm := rf.findLastLogInTerm(reply.XTerm) 78 | DPrintf("[%v]: lastLogInXTerm %v", rf.me, lastLogInXTerm) 79 | if lastLogInXTerm > 0 { 80 | rf.nextIndex[serverId] = lastLogInXTerm 81 | } else { 82 | rf.nextIndex[serverId] = reply.XIndex 83 | } 84 | } 85 | 86 | DPrintf("[%v]: leader nextIndex[%v] %v", rf.me, serverId, rf.nextIndex[serverId]) 87 | } else if rf.nextIndex[serverId] > 1 { 88 | rf.nextIndex[serverId]-- 89 | } 90 | rf.leaderCommitRule() 91 | } 92 | } 93 | 94 | func (rf *Raft) findLastLogInTerm(x int) int { 95 | for i := rf.log.lastLog().Index; i > 0; i-- { 96 | term := rf.log.at(i).Term 97 | if term == x { 98 | return i 99 | } else if term < x { 100 | break 101 | } 102 | } 103 | return -1 104 | } 105 | 106 | func (rf *Raft) leaderCommitRule() { 107 | // leader rule 4 108 | if rf.state != Leader { 109 | return 110 | } 111 | 112 | for n := rf.commitIndex + 1; n <= rf.log.lastLog().Index; n++ { 113 | if rf.log.at(n).Term != rf.currentTerm { 114 | continue 115 | } 116 | counter := 1 117 | for serverId := 0; serverId < len(rf.peers); serverId++ { 118 | if serverId != rf.me && rf.matchIndex[serverId] >= n { 119 | counter++ 120 | } 121 | if counter > len(rf.peers)/2 { 122 | rf.commitIndex = n 123 | DPrintf("[%v] leader尝试提交 index %v", rf.me, rf.commitIndex) 124 | rf.apply() 125 | break 126 | } 127 | } 128 | } 129 | } 130 | 131 | func (rf *Raft) AppendEntries(args *AppendEntriesArgs, reply *AppendEntriesReply) { 132 | rf.mu.Lock() 133 | defer rf.mu.Unlock() 134 | DPrintf("[%d]: (term %d) follower 收到 [%v] AppendEntries %v, prevIndex %v, prevTerm %v", rf.me, rf.currentTerm, args.LeaderId, args.Entries, args.PrevLogIndex, args.PrevLogTerm) 135 | // rules for servers 136 | // all servers 2 137 | reply.Success = false 138 | reply.Term = rf.currentTerm 139 | if args.Term > rf.currentTerm { 140 | rf.setNewTerm(args.Term) 141 | return 142 | } 143 | 144 | // append entries rpc 1 145 | if args.Term < rf.currentTerm { 146 | return 147 | } 148 | rf.resetElectionTimer() 149 | 150 | // candidate rule 3 151 | if rf.state == Candidate { 152 | rf.state = Follower 153 | } 154 | // append entries rpc 2 155 | if rf.log.lastLog().Index < args.PrevLogIndex { 156 | reply.Conflict = true 157 | reply.XTerm = -1 158 | reply.XIndex = -1 159 | reply.XLen = rf.log.len() 160 | DPrintf("[%v]: Conflict XTerm %v, XIndex %v, XLen %v", rf.me, reply.XTerm, reply.XIndex, reply.XLen) 161 | return 162 | } 163 | if rf.log.at(args.PrevLogIndex).Term != args.PrevLogTerm { 164 | reply.Conflict = true 165 | xTerm := rf.log.at(args.PrevLogIndex).Term 166 | for xIndex := args.PrevLogIndex; xIndex > 0; xIndex-- { 167 | if rf.log.at(xIndex-1).Term != xTerm { 168 | reply.XIndex = xIndex 169 | break 170 | } 171 | } 172 | reply.XTerm = xTerm 173 | reply.XLen = rf.log.len() 174 | DPrintf("[%v]: Conflict XTerm %v, XIndex %v, XLen %v", rf.me, reply.XTerm, reply.XIndex, reply.XLen) 175 | return 176 | } 177 | 178 | for idx, entry := range args.Entries { 179 | // append entries rpc 3 180 | if entry.Index <= rf.log.lastLog().Index && rf.log.at(entry.Index).Term != entry.Term { 181 | rf.log.truncate(entry.Index) 182 | rf.persist() 183 | } 184 | // append entries rpc 4 185 | if entry.Index > rf.log.lastLog().Index { 186 | rf.log.append(args.Entries[idx:]...) 187 | DPrintf("[%d]: follower append [%v]", rf.me, args.Entries[idx:]) 188 | rf.persist() 189 | break 190 | } 191 | } 192 | 193 | // append entries rpc 5 194 | if args.LeaderCommit > rf.commitIndex { 195 | rf.commitIndex = min(args.LeaderCommit, rf.log.lastLog().Index) 196 | rf.apply() 197 | } 198 | reply.Success = true 199 | } 200 | 201 | func (rf *Raft) sendAppendEntries(server int, args *AppendEntriesArgs, reply *AppendEntriesReply) bool { 202 | ok := rf.peers[server].Call("Raft.AppendEntries", args, reply) 203 | return ok 204 | } 205 | -------------------------------------------------------------------------------- /src/raft/config.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | // 4 | // support for Raft tester. 5 | // 6 | // we will use the original config.go to test your code for grading. 7 | // so, while you can modify this code to help you debug, please 8 | // test with the original before submitting. 9 | // 10 | 11 | import "6.824/labgob" 12 | import "6.824/labrpc" 13 | import "bytes" 14 | import "log" 15 | import "sync" 16 | import "testing" 17 | import "runtime" 18 | import "math/rand" 19 | import crand "crypto/rand" 20 | import "math/big" 21 | import "encoding/base64" 22 | import "time" 23 | import "fmt" 24 | 25 | func randstring(n int) string { 26 | b := make([]byte, 2*n) 27 | crand.Read(b) 28 | s := base64.URLEncoding.EncodeToString(b) 29 | return s[0:n] 30 | } 31 | 32 | func makeSeed() int64 { 33 | max := big.NewInt(int64(1) << 62) 34 | bigx, _ := crand.Int(crand.Reader, max) 35 | x := bigx.Int64() 36 | return x 37 | } 38 | 39 | type config struct { 40 | mu sync.Mutex 41 | t *testing.T 42 | net *labrpc.Network 43 | n int 44 | rafts []*Raft 45 | applyErr []string // from apply channel readers 46 | connected []bool // whether each server is on the net 47 | saved []*Persister 48 | endnames [][]string // the port file names each sends to 49 | logs []map[int]interface{} // copy of each server's committed entries 50 | start time.Time // time at which make_config() was called 51 | // begin()/end() statistics 52 | t0 time.Time // time at which test_test.go called cfg.begin() 53 | rpcs0 int // rpcTotal() at start of test 54 | cmds0 int // number of agreements 55 | bytes0 int64 56 | maxIndex int 57 | maxIndex0 int 58 | } 59 | 60 | var ncpu_once sync.Once 61 | 62 | func make_config(t *testing.T, n int, unreliable bool, snapshot bool) *config { 63 | ncpu_once.Do(func() { 64 | if runtime.NumCPU() < 2 { 65 | fmt.Printf("warning: only one CPU, which may conceal locking bugs\n") 66 | } 67 | rand.Seed(makeSeed()) 68 | }) 69 | runtime.GOMAXPROCS(4) 70 | cfg := &config{} 71 | cfg.t = t 72 | cfg.net = labrpc.MakeNetwork() 73 | cfg.n = n 74 | cfg.applyErr = make([]string, cfg.n) 75 | cfg.rafts = make([]*Raft, cfg.n) 76 | cfg.connected = make([]bool, cfg.n) 77 | cfg.saved = make([]*Persister, cfg.n) 78 | cfg.endnames = make([][]string, cfg.n) 79 | cfg.logs = make([]map[int]interface{}, cfg.n) 80 | cfg.start = time.Now() 81 | 82 | cfg.setunreliable(unreliable) 83 | 84 | cfg.net.LongDelays(true) 85 | 86 | applier := cfg.applier 87 | if snapshot { 88 | applier = cfg.applierSnap 89 | } 90 | // create a full set of Rafts. 91 | for i := 0; i < cfg.n; i++ { 92 | cfg.logs[i] = map[int]interface{}{} 93 | cfg.start1(i, applier) 94 | } 95 | 96 | // connect everyone 97 | for i := 0; i < cfg.n; i++ { 98 | cfg.connect(i) 99 | } 100 | 101 | return cfg 102 | } 103 | 104 | // shut down a Raft server but save its persistent state. 105 | func (cfg *config) crash1(i int) { 106 | cfg.disconnect(i) 107 | cfg.net.DeleteServer(i) // disable client connections to the server. 108 | 109 | cfg.mu.Lock() 110 | defer cfg.mu.Unlock() 111 | 112 | // a fresh persister, in case old instance 113 | // continues to update the Persister. 114 | // but copy old persister's content so that we always 115 | // pass Make() the last persisted state. 116 | if cfg.saved[i] != nil { 117 | cfg.saved[i] = cfg.saved[i].Copy() 118 | } 119 | 120 | rf := cfg.rafts[i] 121 | if rf != nil { 122 | cfg.mu.Unlock() 123 | rf.Kill() 124 | cfg.mu.Lock() 125 | cfg.rafts[i] = nil 126 | } 127 | 128 | if cfg.saved[i] != nil { 129 | raftlog := cfg.saved[i].ReadRaftState() 130 | snapshot := cfg.saved[i].ReadSnapshot() 131 | cfg.saved[i] = &Persister{} 132 | cfg.saved[i].SaveStateAndSnapshot(raftlog, snapshot) 133 | } 134 | } 135 | 136 | func (cfg *config) checkLogs(i int, m ApplyMsg) (string, bool) { 137 | err_msg := "" 138 | v := m.Command 139 | for j := 0; j < len(cfg.logs); j++ { 140 | if old, oldok := cfg.logs[j][m.CommandIndex]; oldok && old != v { 141 | log.Printf("%v: log %v; server %v\n", i, cfg.logs[i], cfg.logs[j]) 142 | // some server has already committed a different value for this entry! 143 | err_msg = fmt.Sprintf("commit index=%v server=%v %v != server=%v %v", 144 | m.CommandIndex, i, m.Command, j, old) 145 | } 146 | } 147 | _, prevok := cfg.logs[i][m.CommandIndex-1] 148 | cfg.logs[i][m.CommandIndex] = v 149 | if m.CommandIndex > cfg.maxIndex { 150 | cfg.maxIndex = m.CommandIndex 151 | } 152 | return err_msg, prevok 153 | } 154 | 155 | // applier reads message from apply ch and checks that they match the log 156 | // contents 157 | func (cfg *config) applier(i int, applyCh chan ApplyMsg) { 158 | for m := range applyCh { 159 | if m.CommandValid == false { 160 | // ignore other types of ApplyMsg 161 | } else { 162 | cfg.mu.Lock() 163 | err_msg, prevok := cfg.checkLogs(i, m) 164 | cfg.mu.Unlock() 165 | if m.CommandIndex > 1 && prevok == false { 166 | err_msg = fmt.Sprintf("server %v apply out of order %v", i, m.CommandIndex) 167 | } 168 | if err_msg != "" { 169 | log.Fatalf("apply error: %v\n", err_msg) 170 | cfg.applyErr[i] = err_msg 171 | // keep reading after error so that Raft doesn't block 172 | // holding locks... 173 | } 174 | } 175 | } 176 | } 177 | 178 | const SnapShotInterval = 10 179 | 180 | // periodically snapshot raft state 181 | func (cfg *config) applierSnap(i int, applyCh chan ApplyMsg) { 182 | lastApplied := 0 183 | for m := range applyCh { 184 | if m.SnapshotValid { 185 | //DPrintf("Installsnapshot %v %v\n", m.SnapshotIndex, lastApplied) 186 | cfg.mu.Lock() 187 | if cfg.rafts[i].CondInstallSnapshot(m.SnapshotTerm, 188 | m.SnapshotIndex, m.Snapshot) { 189 | cfg.logs[i] = make(map[int]interface{}) 190 | r := bytes.NewBuffer(m.Snapshot) 191 | d := labgob.NewDecoder(r) 192 | var v int 193 | if d.Decode(&v) != nil { 194 | log.Fatalf("decode error\n") 195 | } 196 | cfg.logs[i][m.SnapshotIndex] = v 197 | lastApplied = m.SnapshotIndex 198 | } 199 | cfg.mu.Unlock() 200 | } else if m.CommandValid && m.CommandIndex > lastApplied { 201 | //DPrintf("apply %v lastApplied %v\n", m.CommandIndex, lastApplied) 202 | cfg.mu.Lock() 203 | err_msg, prevok := cfg.checkLogs(i, m) 204 | cfg.mu.Unlock() 205 | if m.CommandIndex > 1 && prevok == false { 206 | err_msg = fmt.Sprintf("server %v apply out of order %v", i, m.CommandIndex) 207 | } 208 | if err_msg != "" { 209 | log.Fatalf("apply error: %v\n", err_msg) 210 | cfg.applyErr[i] = err_msg 211 | // keep reading after error so that Raft doesn't block 212 | // holding locks... 213 | } 214 | lastApplied = m.CommandIndex 215 | if (m.CommandIndex+1)%SnapShotInterval == 0 { 216 | w := new(bytes.Buffer) 217 | e := labgob.NewEncoder(w) 218 | v := m.Command 219 | e.Encode(v) 220 | cfg.rafts[i].Snapshot(m.CommandIndex, w.Bytes()) 221 | } 222 | } else { 223 | // Ignore other types of ApplyMsg or old 224 | // commands. Old command may never happen, 225 | // depending on the Raft implementation, but 226 | // just in case. 227 | // DPrintf("Ignore: Index %v lastApplied %v\n", m.CommandIndex, lastApplied) 228 | 229 | } 230 | } 231 | } 232 | 233 | // start or re-start a Raft. 234 | // if one already exists, "kill" it first. 235 | // allocate new outgoing port file names, and a new 236 | // state persister, to isolate previous instance of 237 | // this server. since we cannot really kill it. 238 | func (cfg *config) start1(i int, applier func(int, chan ApplyMsg)) { 239 | cfg.crash1(i) 240 | 241 | // a fresh set of outgoing ClientEnd names. 242 | // so that old crashed instance's ClientEnds can't send. 243 | cfg.endnames[i] = make([]string, cfg.n) 244 | for j := 0; j < cfg.n; j++ { 245 | cfg.endnames[i][j] = randstring(20) 246 | } 247 | 248 | // a fresh set of ClientEnds. 249 | ends := make([]*labrpc.ClientEnd, cfg.n) 250 | for j := 0; j < cfg.n; j++ { 251 | ends[j] = cfg.net.MakeEnd(cfg.endnames[i][j]) 252 | cfg.net.Connect(cfg.endnames[i][j], j) 253 | } 254 | 255 | cfg.mu.Lock() 256 | 257 | // a fresh persister, so old instance doesn't overwrite 258 | // new instance's persisted state. 259 | // but copy old persister's content so that we always 260 | // pass Make() the last persisted state. 261 | if cfg.saved[i] != nil { 262 | cfg.saved[i] = cfg.saved[i].Copy() 263 | } else { 264 | cfg.saved[i] = MakePersister() 265 | } 266 | 267 | cfg.mu.Unlock() 268 | 269 | applyCh := make(chan ApplyMsg) 270 | 271 | rf := Make(ends, i, cfg.saved[i], applyCh) 272 | 273 | cfg.mu.Lock() 274 | cfg.rafts[i] = rf 275 | cfg.mu.Unlock() 276 | 277 | go applier(i, applyCh) 278 | 279 | svc := labrpc.MakeService(rf) 280 | srv := labrpc.MakeServer() 281 | srv.AddService(svc) 282 | cfg.net.AddServer(i, srv) 283 | } 284 | 285 | func (cfg *config) checkTimeout() { 286 | // enforce a two minute real-time limit on each test 287 | if !cfg.t.Failed() && time.Since(cfg.start) > 120*time.Second { 288 | cfg.t.Fatal("test took longer than 120 seconds") 289 | } 290 | } 291 | 292 | func (cfg *config) cleanup() { 293 | for i := 0; i < len(cfg.rafts); i++ { 294 | if cfg.rafts[i] != nil { 295 | cfg.rafts[i].Kill() 296 | } 297 | } 298 | cfg.net.Cleanup() 299 | cfg.checkTimeout() 300 | } 301 | 302 | // attach server i to the net. 303 | func (cfg *config) connect(i int) { 304 | // fmt.Printf("connect(%d)\n", i) 305 | 306 | cfg.connected[i] = true 307 | 308 | // outgoing ClientEnds 309 | for j := 0; j < cfg.n; j++ { 310 | if cfg.connected[j] { 311 | endname := cfg.endnames[i][j] 312 | cfg.net.Enable(endname, true) 313 | } 314 | } 315 | 316 | // incoming ClientEnds 317 | for j := 0; j < cfg.n; j++ { 318 | if cfg.connected[j] { 319 | endname := cfg.endnames[j][i] 320 | cfg.net.Enable(endname, true) 321 | } 322 | } 323 | } 324 | 325 | // detach server i from the net. 326 | func (cfg *config) disconnect(i int) { 327 | // fmt.Printf("disconnect(%d)\n", i) 328 | 329 | cfg.connected[i] = false 330 | 331 | // outgoing ClientEnds 332 | for j := 0; j < cfg.n; j++ { 333 | if cfg.endnames[i] != nil { 334 | endname := cfg.endnames[i][j] 335 | cfg.net.Enable(endname, false) 336 | } 337 | } 338 | 339 | // incoming ClientEnds 340 | for j := 0; j < cfg.n; j++ { 341 | if cfg.endnames[j] != nil { 342 | endname := cfg.endnames[j][i] 343 | cfg.net.Enable(endname, false) 344 | } 345 | } 346 | } 347 | 348 | func (cfg *config) rpcCount(server int) int { 349 | return cfg.net.GetCount(server) 350 | } 351 | 352 | func (cfg *config) rpcTotal() int { 353 | return cfg.net.GetTotalCount() 354 | } 355 | 356 | func (cfg *config) setunreliable(unrel bool) { 357 | cfg.net.Reliable(!unrel) 358 | } 359 | 360 | func (cfg *config) bytesTotal() int64 { 361 | return cfg.net.GetTotalBytes() 362 | } 363 | 364 | func (cfg *config) setlongreordering(longrel bool) { 365 | cfg.net.LongReordering(longrel) 366 | } 367 | 368 | // check that there's exactly one leader. 369 | // try a few times in case re-elections are needed. 370 | func (cfg *config) checkOneLeader() int { 371 | for iters := 0; iters < 10; iters++ { 372 | ms := 450 + (rand.Int63() % 100) 373 | time.Sleep(time.Duration(ms) * time.Millisecond) 374 | 375 | leaders := make(map[int][]int) 376 | for i := 0; i < cfg.n; i++ { 377 | if cfg.connected[i] { 378 | if term, leader := cfg.rafts[i].GetState(); leader { 379 | leaders[term] = append(leaders[term], i) 380 | } 381 | } 382 | } 383 | 384 | lastTermWithLeader := -1 385 | for term, leaders := range leaders { 386 | if len(leaders) > 1 { 387 | cfg.t.Fatalf("term %d has %d (>1) leaders", term, len(leaders)) 388 | } 389 | if term > lastTermWithLeader { 390 | lastTermWithLeader = term 391 | } 392 | } 393 | 394 | if len(leaders) != 0 { 395 | return leaders[lastTermWithLeader][0] 396 | } 397 | } 398 | cfg.t.Fatalf("expected one leader, got none") 399 | return -1 400 | } 401 | 402 | // check that everyone agrees on the term. 403 | func (cfg *config) checkTerms() int { 404 | term := -1 405 | for i := 0; i < cfg.n; i++ { 406 | if cfg.connected[i] { 407 | xterm, _ := cfg.rafts[i].GetState() 408 | if term == -1 { 409 | term = xterm 410 | } else if term != xterm { 411 | cfg.t.Fatalf("servers disagree on term") 412 | } 413 | } 414 | } 415 | return term 416 | } 417 | 418 | // check that there's no leader 419 | func (cfg *config) checkNoLeader() { 420 | for i := 0; i < cfg.n; i++ { 421 | if cfg.connected[i] { 422 | _, is_leader := cfg.rafts[i].GetState() 423 | if is_leader { 424 | cfg.t.Fatalf("expected no leader, but %v claims to be leader", i) 425 | } 426 | } 427 | } 428 | } 429 | 430 | // how many servers think a log entry is committed? 431 | func (cfg *config) nCommitted(index int) (int, interface{}) { 432 | count := 0 433 | var cmd interface{} = nil 434 | for i := 0; i < len(cfg.rafts); i++ { 435 | if cfg.applyErr[i] != "" { 436 | cfg.t.Fatal(cfg.applyErr[i]) 437 | } 438 | 439 | cfg.mu.Lock() 440 | cmd1, ok := cfg.logs[i][index] 441 | cfg.mu.Unlock() 442 | 443 | if ok { 444 | if count > 0 && cmd != cmd1 { 445 | cfg.t.Fatalf("committed values do not match: index %v, %v, %v\n", 446 | index, cmd, cmd1) 447 | } 448 | count += 1 449 | cmd = cmd1 450 | } 451 | } 452 | return count, cmd 453 | } 454 | 455 | // wait for at least n servers to commit. 456 | // but don't wait forever. 457 | func (cfg *config) wait(index int, n int, startTerm int) interface{} { 458 | to := 10 * time.Millisecond 459 | for iters := 0; iters < 30; iters++ { 460 | nd, _ := cfg.nCommitted(index) 461 | if nd >= n { 462 | break 463 | } 464 | time.Sleep(to) 465 | if to < time.Second { 466 | to *= 2 467 | } 468 | if startTerm > -1 { 469 | for _, r := range cfg.rafts { 470 | if t, _ := r.GetState(); t > startTerm { 471 | // someone has moved on 472 | // can no longer guarantee that we'll "win" 473 | return -1 474 | } 475 | } 476 | } 477 | } 478 | nd, cmd := cfg.nCommitted(index) 479 | if nd < n { 480 | cfg.t.Fatalf("only %d decided for index %d; wanted %d\n", 481 | nd, index, n) 482 | } 483 | return cmd 484 | } 485 | 486 | // do a complete agreement. 487 | // it might choose the wrong leader initially, 488 | // and have to re-submit after giving up. 489 | // entirely gives up after about 10 seconds. 490 | // indirectly checks that the servers agree on the 491 | // same value, since nCommitted() checks this, 492 | // as do the threads that read from applyCh. 493 | // returns index. 494 | // if retry==true, may submit the command multiple 495 | // times, in case a leader fails just after Start(). 496 | // if retry==false, calls Start() only once, in order 497 | // to simplify the early Lab 2B tests. 498 | func (cfg *config) one(cmd interface{}, expectedServers int, retry bool) int { 499 | t0 := time.Now() 500 | starts := 0 501 | for time.Since(t0).Seconds() < 10 { 502 | // try all the servers, maybe one is the leader. 503 | index := -1 504 | for si := 0; si < cfg.n; si++ { 505 | starts = (starts + 1) % cfg.n 506 | var rf *Raft 507 | cfg.mu.Lock() 508 | if cfg.connected[starts] { 509 | rf = cfg.rafts[starts] 510 | } 511 | cfg.mu.Unlock() 512 | if rf != nil { 513 | index1, _, ok := rf.Start(cmd) 514 | if ok { 515 | index = index1 516 | break 517 | } 518 | } 519 | } 520 | 521 | if index != -1 { 522 | // somebody claimed to be the leader and to have 523 | // submitted our command; wait a while for agreement. 524 | t1 := time.Now() 525 | for time.Since(t1).Seconds() < 2 { 526 | nd, cmd1 := cfg.nCommitted(index) 527 | if nd > 0 && nd >= expectedServers { 528 | // committed 529 | if cmd1 == cmd { 530 | // and it was the command we submitted. 531 | return index 532 | } 533 | } 534 | time.Sleep(20 * time.Millisecond) 535 | } 536 | if retry == false { 537 | cfg.t.Fatalf("one(%v) failed to reach agreement", cmd) 538 | } 539 | } else { 540 | time.Sleep(50 * time.Millisecond) 541 | } 542 | } 543 | cfg.t.Fatalf("one(%v) failed to reach agreement", cmd) 544 | return -1 545 | } 546 | 547 | // start a Test. 548 | // print the Test message. 549 | // e.g. cfg.begin("Test (2B): RPC counts aren't too high") 550 | func (cfg *config) begin(description string) { 551 | fmt.Printf("%s ...\n", description) 552 | cfg.t0 = time.Now() 553 | cfg.rpcs0 = cfg.rpcTotal() 554 | cfg.bytes0 = cfg.bytesTotal() 555 | cfg.cmds0 = 0 556 | cfg.maxIndex0 = cfg.maxIndex 557 | } 558 | 559 | // end a Test -- the fact that we got here means there 560 | // was no failure. 561 | // print the Passed message, 562 | // and some performance numbers. 563 | func (cfg *config) end() { 564 | cfg.checkTimeout() 565 | if cfg.t.Failed() == false { 566 | cfg.mu.Lock() 567 | t := time.Since(cfg.t0).Seconds() // real time 568 | npeers := cfg.n // number of Raft peers 569 | nrpc := cfg.rpcTotal() - cfg.rpcs0 // number of RPC sends 570 | nbytes := cfg.bytesTotal() - cfg.bytes0 // number of bytes 571 | ncmds := cfg.maxIndex - cfg.maxIndex0 // number of Raft agreements reported 572 | cfg.mu.Unlock() 573 | 574 | fmt.Printf(" ... Passed --") 575 | fmt.Printf(" %4.1f %d %4d %7d %4d\n", t, npeers, nrpc, nbytes, ncmds) 576 | } 577 | } 578 | 579 | // Maximum log size across all servers 580 | func (cfg *config) LogSize() int { 581 | logsize := 0 582 | for i := 0; i < cfg.n; i++ { 583 | n := cfg.saved[i].RaftStateSize() 584 | if n > logsize { 585 | logsize = n 586 | } 587 | } 588 | return logsize 589 | } 590 | -------------------------------------------------------------------------------- /src/raft/go-test-many.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | # 3 | # Script for running `go test` a bunch of times, in parallel, storing the test 4 | # output as you go, and showing a nice status output telling you how you're 5 | # doing. 6 | # 7 | # Normally, you should be able to execute this script with 8 | # 9 | # ./go-test-many.sh 10 | # 11 | # and it should do The Right Thing(tm) by default. However, it does take some 12 | # arguments so that you can tweak it for your testing setup. To understand 13 | # them, we should first go quickly through what exactly this script does. 14 | # 15 | # First, it compiles your Go program (using go test -c) to ensure that all the 16 | # tests are run on the same codebase, and to speed up the testing. Then, it 17 | # runs the tester some number of times. It will run some number of testers in 18 | # parallel, and when that number of running testers has been reached, it will 19 | # wait for the oldest one it spawned to finish before spawning another. The 20 | # output from each test i is stored in test-$i.log and test-$i.err (STDOUT and 21 | # STDERR respectively). 22 | # 23 | # The options you can specify on the command line are: 24 | # 25 | # 1) how many times to run the tester (defaults to 100) 26 | # 2) how many testers to run in parallel (defaults to the number of CPUs) 27 | # 3) which subset of the tests to run (default to all tests) 28 | # 29 | # 3) is simply a regex that is passed to the tester under -test.run; any tests 30 | # matching the regex will be run. 31 | # 32 | # The script is smart enough to clean up after itself if you kill it 33 | # (in-progress tests are killed, their output is discarded, and no failure 34 | # message is printed), and will automatically continue from where it left off 35 | # if you kill it and then start it again. 36 | # 37 | # By now, you know everything that happens below. 38 | # If you still want to read the code, go ahead. 39 | 40 | if [ $# -eq 1 ] && [ "$1" = "--help" ]; then 41 | echo "Usage: $0 [RUNS=100] [PARALLELISM=#cpus] [TESTPATTERN='']" 42 | exit 1 43 | fi 44 | 45 | # If the tests don't even build, don't bother. Also, this gives us a static 46 | # tester binary for higher performance and higher reproducability. 47 | if ! go test -c -o tester; then 48 | echo -e "\e[1;31mERROR: Build failed\e[0m" 49 | exit 1 50 | fi 51 | 52 | # Default to 100 runs unless otherwise specified 53 | runs=100 54 | if [ $# -gt 0 ]; then 55 | runs="$1" 56 | fi 57 | 58 | # Default to one tester per CPU unless otherwise specified 59 | parallelism=$(grep -c processor /proc/cpuinfo) 60 | if [ $# -gt 1 ]; then 61 | parallelism="$2" 62 | fi 63 | 64 | # Default to no test filtering unless otherwise specified 65 | test="" 66 | if [ $# -gt 2 ]; then 67 | test="$3" 68 | fi 69 | 70 | # Figure out where we left off 71 | logs=$(find . -maxdepth 1 -name 'test-*.log' -type f -printf '.' | wc -c) 72 | success=$(grep -E '^PASS$' test-*.log | wc -l) 73 | ((failed = logs - success)) 74 | 75 | # Finish checks the exit status of the tester with the given PID, updates the 76 | # success/failed counters appropriately, and prints a pretty message. 77 | finish() { 78 | if ! wait "$1"; then 79 | if command -v notify-send >/dev/null 2>&1 &&((failed == 0)); then 80 | notify-send -i weather-storm "Tests started failing" \ 81 | "$(pwd)\n$(grep FAIL: -- *.log | sed -e 's/.*FAIL: / - /' -e 's/ (.*)//' | sort -u)" 82 | fi 83 | ((failed += 1)) 84 | else 85 | ((success += 1)) 86 | fi 87 | 88 | if [ "$failed" -eq 0 ]; then 89 | printf "\e[1;32m"; 90 | else 91 | printf "\e[1;31m"; 92 | fi 93 | 94 | printf "Done %03d/%d; %d ok, %d failed\n\e[0m" \ 95 | $((success+failed)) \ 96 | "$runs" \ 97 | "$success" \ 98 | "$failed" 99 | } 100 | 101 | waits=() # which tester PIDs are we waiting on? 102 | is=() # and which iteration does each one correspond to? 103 | 104 | # Cleanup is called when the process is killed. 105 | # It kills any remaining tests and removes their output files before exiting. 106 | cleanup() { 107 | for pid in "${waits[@]}"; do 108 | kill "$pid" 109 | wait "$pid" 110 | rm -rf "test-${is[0]}.err" "test-${is[0]}.log" 111 | is=("${is[@]:1}") 112 | done 113 | exit 0 114 | } 115 | trap cleanup SIGHUP SIGINT SIGTERM 116 | 117 | # Run remaining iterations (we may already have run some) 118 | for i in $(seq "$((success+failed+1))" "$runs"); do 119 | # If we have already spawned the max # of testers, wait for one to 120 | # finish. We'll wait for the oldest one beause it's easy. 121 | if [[ ${#waits[@]} -eq "$parallelism" ]]; then 122 | finish "${waits[0]}" 123 | waits=("${waits[@]:1}") # this funky syntax removes the first 124 | is=("${is[@]:1}") # element from the array 125 | fi 126 | 127 | # Store this tester's iteration index 128 | # It's important that this happens before appending to waits(), 129 | # otherwise we could get an out-of-bounds in cleanup() 130 | is=("${is[@]}" $i) 131 | 132 | # Run the tester, passing -test.run if necessary 133 | if [[ -z "$test" ]]; then 134 | ./tester -test.v 2> "test-${i}.err" > "test-${i}.log" & 135 | pid=$! 136 | else 137 | ./tester -test.run "$test" -test.v 2> "test-${i}.err" > "test-${i}.log" & 138 | pid=$! 139 | fi 140 | 141 | # Remember the tester's PID so we can wait on it later 142 | waits=("${waits[@]}" $pid) 143 | done 144 | 145 | # Wait for remaining testers 146 | for pid in "${waits[@]}"; do 147 | finish "$pid" 148 | done 149 | 150 | if ((failed>0)); then 151 | exit 1 152 | fi 153 | exit 0 154 | -------------------------------------------------------------------------------- /src/raft/leader_election.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "math/rand" 5 | "sync" 6 | "time" 7 | ) 8 | 9 | // return currentTerm and whether this server 10 | // believes it is the leader. 11 | func (rf *Raft) GetState() (int, bool) { 12 | rf.mu.Lock() 13 | defer rf.mu.Unlock() 14 | term := rf.currentTerm 15 | isleader := rf.state == Leader 16 | return term, isleader 17 | } 18 | 19 | func (rf *Raft) resetElectionTimer() { 20 | t := time.Now() 21 | electionTimeout := time.Duration(150 + rand.Intn(150)) * time.Millisecond 22 | rf.electionTime = t.Add(electionTimeout) 23 | } 24 | 25 | func (rf *Raft) setNewTerm(term int) { 26 | if term > rf.currentTerm || rf.currentTerm == 0 { 27 | rf.state = Follower 28 | rf.currentTerm = term 29 | rf.votedFor = -1 30 | DPrintf("[%d]: set term %v\n", rf.me, rf.currentTerm) 31 | rf.persist() 32 | } 33 | } 34 | 35 | func (rf *Raft) leaderElection() { 36 | rf.currentTerm++ 37 | rf.state = Candidate 38 | rf.votedFor = rf.me 39 | rf.persist() 40 | rf.resetElectionTimer() 41 | term := rf.currentTerm 42 | voteCounter := 1 43 | lastLog := rf.log.lastLog() 44 | DPrintf("[%v]: start leader election, term %d\n", rf.me, rf.currentTerm) 45 | args := RequestVoteArgs{ 46 | Term: term, 47 | CandidateId: rf.me, 48 | LastLogIndex: lastLog.Index, 49 | LastLogTerm: lastLog.Term, 50 | } 51 | 52 | var becomeLeader sync.Once 53 | for serverId, _ := range rf.peers { 54 | if serverId != rf.me { 55 | go rf.candidateRequestVote(serverId, &args, &voteCounter, &becomeLeader) 56 | } 57 | } 58 | } 59 | -------------------------------------------------------------------------------- /src/raft/log.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "fmt" 5 | "strings" 6 | ) 7 | 8 | type Log struct { 9 | Entries []Entry 10 | Index0 int 11 | } 12 | 13 | type Entry struct { 14 | Command interface{} 15 | Term int 16 | Index int 17 | } 18 | 19 | func (l *Log) append(entries ...Entry) { 20 | l.Entries = append(l.Entries, entries...) 21 | } 22 | 23 | func makeEmptyLog() Log { 24 | log := Log{ 25 | Entries: make([]Entry, 0), 26 | Index0: 0, 27 | } 28 | return log 29 | } 30 | 31 | func (l *Log) at(idx int) *Entry { 32 | return &l.Entries[idx] 33 | } 34 | 35 | func (l *Log) truncate(idx int) { 36 | l.Entries = l.Entries[:idx] 37 | } 38 | 39 | func (l *Log) slice(idx int) []Entry { 40 | return l.Entries[idx:] 41 | } 42 | 43 | func (l *Log) len() int { 44 | return len(l.Entries) 45 | } 46 | 47 | func (l *Log) lastLog() *Entry { 48 | return l.at(l.len() - 1) 49 | } 50 | 51 | func (e *Entry) String() string { 52 | return fmt.Sprint(e.Term) 53 | } 54 | 55 | 56 | func (l *Log) String() string { 57 | nums := []string{} 58 | for _, entry := range l.Entries { 59 | nums = append(nums, fmt.Sprintf("%4d", entry.Term)) 60 | } 61 | return fmt.Sprint(strings.Join(nums, "|")) 62 | } -------------------------------------------------------------------------------- /src/raft/persister.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | // 4 | // support for Raft and kvraft to save persistent 5 | // Raft state (log &c) and k/v server snapshots. 6 | // 7 | // we will use the original persister.go to test your code for grading. 8 | // so, while you can modify this code to help you debug, please 9 | // test with the original before submitting. 10 | // 11 | 12 | import "sync" 13 | 14 | type Persister struct { 15 | mu sync.Mutex 16 | raftstate []byte 17 | snapshot []byte 18 | } 19 | 20 | func MakePersister() *Persister { 21 | return &Persister{} 22 | } 23 | 24 | func clone(orig []byte) []byte { 25 | x := make([]byte, len(orig)) 26 | copy(x, orig) 27 | return x 28 | } 29 | 30 | func (ps *Persister) Copy() *Persister { 31 | ps.mu.Lock() 32 | defer ps.mu.Unlock() 33 | np := MakePersister() 34 | np.raftstate = ps.raftstate 35 | np.snapshot = ps.snapshot 36 | return np 37 | } 38 | 39 | func (ps *Persister) SaveRaftState(state []byte) { 40 | ps.mu.Lock() 41 | defer ps.mu.Unlock() 42 | ps.raftstate = clone(state) 43 | } 44 | 45 | func (ps *Persister) ReadRaftState() []byte { 46 | ps.mu.Lock() 47 | defer ps.mu.Unlock() 48 | return clone(ps.raftstate) 49 | } 50 | 51 | func (ps *Persister) RaftStateSize() int { 52 | ps.mu.Lock() 53 | defer ps.mu.Unlock() 54 | return len(ps.raftstate) 55 | } 56 | 57 | // Save both Raft state and K/V snapshot as a single atomic action, 58 | // to help avoid them getting out of sync. 59 | func (ps *Persister) SaveStateAndSnapshot(state []byte, snapshot []byte) { 60 | ps.mu.Lock() 61 | defer ps.mu.Unlock() 62 | ps.raftstate = clone(state) 63 | ps.snapshot = clone(snapshot) 64 | } 65 | 66 | func (ps *Persister) ReadSnapshot() []byte { 67 | ps.mu.Lock() 68 | defer ps.mu.Unlock() 69 | return clone(ps.snapshot) 70 | } 71 | 72 | func (ps *Persister) SnapshotSize() int { 73 | ps.mu.Lock() 74 | defer ps.mu.Unlock() 75 | return len(ps.snapshot) 76 | } -------------------------------------------------------------------------------- /src/raft/raft.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | // 4 | // this is an outline of the API that raft must expose to 5 | // the service (or tester). see comments below for 6 | // each of these functions for more details. 7 | // 8 | // rf = Make(...) 9 | // create a new Raft server. 10 | // rf.Start(command interface{}) (index, term, isleader) 11 | // start agreement on a new log entry 12 | // rf.GetState() (term, isLeader) 13 | // ask a Raft for its current term, and whether it thinks it is leader 14 | // ApplyMsg 15 | // each time a new entry is committed to the log, each Raft peer 16 | // should send an ApplyMsg to the service (or tester) 17 | // in the same server. 18 | // 19 | 20 | import ( 21 | "bytes" 22 | "fmt" 23 | "log" 24 | "strings" 25 | "sync" 26 | "sync/atomic" 27 | "time" 28 | 29 | "6.824/labgob" 30 | "6.824/labrpc" 31 | ) 32 | 33 | // as each Raft peer becomes aware that successive log entries are 34 | // committed, the peer should send an ApplyMsg to the service (or 35 | // tester) on the same server, via the applyCh passed to Make(). set 36 | // CommandValid to true to indicate that the ApplyMsg contains a newly 37 | // committed log entry. 38 | // 39 | // in part 2D you'll want to send other kinds of messages (e.g., 40 | // snapshots) on the applyCh, but set CommandValid to false for these 41 | // other uses. 42 | type ApplyMsg struct { 43 | CommandValid bool 44 | Command interface{} 45 | CommandIndex int 46 | 47 | // For 2D: 48 | SnapshotValid bool 49 | Snapshot []byte 50 | SnapshotTerm int 51 | SnapshotIndex int 52 | } 53 | 54 | type RaftState string 55 | 56 | const ( 57 | Follower RaftState = "Follower" 58 | Candidate = "Candidate" 59 | Leader = "Leader" 60 | ) 61 | 62 | // A Go object implementing a single Raft peer. 63 | type Raft struct { 64 | mu sync.Mutex // Lock to protect shared access to this peer's state 65 | peers []*labrpc.ClientEnd // RPC end points of all peers 66 | persister *Persister // Object to hold this peer's persisted state 67 | me int // this peer's index into peers[] 68 | dead int32 // set by Kill() 69 | 70 | // Your data here (2A, 2B, 2C). 71 | // Look at the paper's Figure 2 for a description of what 72 | // state a Raft server must maintain. 73 | 74 | state RaftState 75 | appendEntryCh chan *Entry 76 | heartBeat time.Duration 77 | electionTime time.Time 78 | 79 | // Persistent state on all servers: 80 | currentTerm int 81 | votedFor int 82 | log Log 83 | 84 | // Volatile state on all servers: 85 | commitIndex int 86 | lastApplied int 87 | 88 | // Volatile state on leaders: 89 | nextIndex []int 90 | matchIndex []int 91 | 92 | applyCh chan ApplyMsg 93 | applyCond *sync.Cond 94 | } 95 | 96 | // save Raft's persistent state to stable storage, 97 | // where it can later be retrieved after a crash and restart. 98 | // see paper's Figure 2 for a description of what should be persistent. 99 | func (rf *Raft) persist() { 100 | DPrintVerbose("[%v]: STATE: %v", rf.me, rf.log.String()) 101 | w := new(bytes.Buffer) 102 | e := labgob.NewEncoder(w) 103 | e.Encode(rf.currentTerm) 104 | e.Encode(rf.votedFor) 105 | e.Encode(rf.log) 106 | data := w.Bytes() 107 | rf.persister.SaveRaftState(data) 108 | } 109 | 110 | // restore previously persisted state. 111 | func (rf *Raft) readPersist(data []byte) { 112 | if data == nil || len(data) < 1 { // bootstrap without any state? 113 | return 114 | } 115 | 116 | r := bytes.NewBuffer(data) 117 | d := labgob.NewDecoder(r) 118 | var currentTerm int 119 | var votedFor int 120 | var logs Log 121 | 122 | if d.Decode(¤tTerm) != nil || d.Decode(&votedFor) != nil || d.Decode(&logs) != nil { 123 | log.Fatal("failed to read persist\n") 124 | } else { 125 | rf.currentTerm = currentTerm 126 | rf.votedFor = votedFor 127 | rf.log = logs 128 | } 129 | } 130 | 131 | // A service wants to switch to snapshot. Only do so if Raft hasn't 132 | // have more recent info since it communicate the snapshot on applyCh. 133 | func (rf *Raft) CondInstallSnapshot(lastIncludedTerm int, lastIncludedIndex int, snapshot []byte) bool { 134 | 135 | // Your code here (2D). 136 | 137 | return true 138 | } 139 | 140 | // the service says it has created a snapshot that has 141 | // all info up to and including index. this means the 142 | // service no longer needs the log through (and including) 143 | // that index. Raft should now trim its log as much as possible. 144 | func (rf *Raft) Snapshot(index int, snapshot []byte) { 145 | // Your code here (2D). 146 | 147 | } 148 | 149 | // the service using Raft (e.g. a k/v server) wants to start 150 | // agreement on the next command to be appended to Raft's log. if this 151 | // server isn't the leader, returns false. otherwise start the 152 | // agreement and return immediately. there is no guarantee that this 153 | // command will ever be committed to the Raft log, since the leader 154 | // may fail or lose an election. even if the Raft instance has been killed, 155 | // this function should return gracefully. 156 | // 157 | // the first return value is the index that the command will appear at 158 | // if it's ever committed. the second return value is the current 159 | // term. the third return value is true if this server believes it is 160 | // the leader. 161 | func (rf *Raft) Start(command interface{}) (int, int, bool) { 162 | rf.mu.Lock() 163 | defer rf.mu.Unlock() 164 | if rf.state != Leader { 165 | return -1, rf.currentTerm, false 166 | } 167 | index := rf.log.lastLog().Index + 1 168 | term := rf.currentTerm 169 | 170 | log := Entry{ 171 | Command: command, 172 | Index: index, 173 | Term: term, 174 | } 175 | rf.log.append(log) 176 | rf.persist() 177 | DPrintf("[%v]: term %v Start %v", rf.me, term, log) 178 | rf.appendEntries(false) 179 | 180 | return index, term, true 181 | } 182 | 183 | // the tester doesn't halt goroutines created by Raft after each test, 184 | // but it does call the Kill() method. your code can use killed() to 185 | // check whether Kill() has been called. the use of atomic avoids the 186 | // need for a lock. 187 | // 188 | // the issue is that long-running goroutines use memory and may chew 189 | // up CPU time, perhaps causing later tests to fail and generating 190 | // confusing debug output. any goroutine with a long-running loop 191 | // should call killed() to check whether it should stop. 192 | func (rf *Raft) Kill() { 193 | atomic.StoreInt32(&rf.dead, 1) 194 | // Your code here, if desired. 195 | } 196 | 197 | func (rf *Raft) killed() bool { 198 | z := atomic.LoadInt32(&rf.dead) 199 | return z == 1 200 | } 201 | 202 | // The ticker go routine starts a new election if this peer hasn't received 203 | // heartsbeats recently. 204 | func (rf *Raft) ticker() { 205 | for rf.killed() == false { 206 | 207 | // Your code here to check if a leader election should 208 | // be started and to randomize sleeping time using 209 | // time.Sleep(). 210 | time.Sleep(rf.heartBeat) 211 | rf.mu.Lock() 212 | if rf.state == Leader { 213 | rf.appendEntries(true) 214 | } 215 | if time.Now().After(rf.electionTime) { 216 | rf.leaderElection() 217 | } 218 | rf.mu.Unlock() 219 | } 220 | } 221 | 222 | // the service or tester wants to create a Raft server. the ports 223 | // of all the Raft servers (including this one) are in peers[]. this 224 | // server's port is peers[me]. all the servers' peers[] arrays 225 | // have the same order. persister is a place for this server to 226 | // save its persistent state, and also initially holds the most 227 | // recent saved state, if any. applyCh is a channel on which the 228 | // tester or service expects Raft to send ApplyMsg messages. 229 | // Make() must return quickly, so it should start goroutines 230 | // for any long-running work. 231 | func Make(peers []*labrpc.ClientEnd, me int, 232 | persister *Persister, applyCh chan ApplyMsg) *Raft { 233 | rf := &Raft{} 234 | rf.peers = peers 235 | rf.persister = persister 236 | rf.me = me 237 | 238 | // Your initialization code here (2A, 2B, 2C). 239 | rf.state = Follower 240 | rf.currentTerm = 0 241 | rf.votedFor = -1 242 | rf.heartBeat = 50 * time.Millisecond 243 | rf.resetElectionTimer() 244 | 245 | rf.log = makeEmptyLog() 246 | rf.log.append(Entry{-1, 0, 0}) 247 | rf.commitIndex = 0 248 | rf.lastApplied = 0 249 | rf.nextIndex = make([]int, len(rf.peers)) 250 | rf.matchIndex = make([]int, len(rf.peers)) 251 | 252 | rf.applyCh = applyCh 253 | rf.applyCond = sync.NewCond(&rf.mu) 254 | 255 | // initialize from state persisted before a crash 256 | rf.readPersist(persister.ReadRaftState()) 257 | 258 | // start ticker goroutine to start elections 259 | go rf.ticker() 260 | 261 | go rf.applier() 262 | return rf 263 | } 264 | 265 | func (rf *Raft) apply() { 266 | rf.applyCond.Broadcast() 267 | DPrintf("[%v]: rf.applyCond.Broadcast()", rf.me) 268 | } 269 | 270 | func (rf *Raft) applier() { 271 | rf.mu.Lock() 272 | defer rf.mu.Unlock() 273 | 274 | for !rf.killed() { 275 | // all server rule 1 276 | if rf.commitIndex > rf.lastApplied && rf.log.lastLog().Index > rf.lastApplied { 277 | rf.lastApplied++ 278 | applyMsg := ApplyMsg{ 279 | CommandValid: true, 280 | Command: rf.log.at(rf.lastApplied).Command, 281 | CommandIndex: rf.lastApplied, 282 | } 283 | DPrintVerbose("[%v]: COMMIT %d: %v", rf.me, rf.lastApplied, rf.commits()) 284 | rf.mu.Unlock() 285 | rf.applyCh <- applyMsg 286 | rf.mu.Lock() 287 | } else { 288 | rf.applyCond.Wait() 289 | DPrintf("[%v]: rf.applyCond.Wait()", rf.me) 290 | } 291 | } 292 | } 293 | 294 | func (rf *Raft) commits() string { 295 | nums := []string{} 296 | for i := 0; i <= rf.lastApplied; i++ { 297 | nums = append(nums, fmt.Sprintf("%4d", rf.log.at(i).Command)) 298 | } 299 | return fmt.Sprint(strings.Join(nums, "|")) 300 | } 301 | -------------------------------------------------------------------------------- /src/raft/request_vote.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "sync" 5 | ) 6 | 7 | // example RequestVote RPC arguments structure. 8 | // field names must start with capital letters! 9 | type RequestVoteArgs struct { 10 | // Your data here (2A, 2B). 11 | Term int 12 | CandidateId int 13 | LastLogIndex int 14 | LastLogTerm int 15 | } 16 | 17 | // example RequestVote RPC reply structure. 18 | // field names must start with capital letters! 19 | type RequestVoteReply struct { 20 | // Your data here (2A). 21 | Term int 22 | VoteGranted bool 23 | } 24 | 25 | // example RequestVote RPC handler. 26 | func (rf *Raft) RequestVote(args *RequestVoteArgs, reply *RequestVoteReply) { 27 | // Your code here (2A, 2B). 28 | rf.mu.Lock() 29 | defer rf.mu.Unlock() 30 | 31 | // rules for servers 32 | // all servers 2 33 | if args.Term > rf.currentTerm { 34 | rf.setNewTerm(args.Term) 35 | } 36 | 37 | // request vote rpc receiver 1 38 | if args.Term < rf.currentTerm { 39 | reply.Term = rf.currentTerm 40 | reply.VoteGranted = false 41 | return 42 | } 43 | 44 | // request vote rpc receiver 2 45 | myLastLog := rf.log.lastLog() 46 | upToDate := args.LastLogTerm > myLastLog.Term || 47 | (args.LastLogTerm == myLastLog.Term && args.LastLogIndex >= myLastLog.Index) 48 | if (rf.votedFor == -1 || rf.votedFor == args.CandidateId) && upToDate { 49 | reply.VoteGranted = true 50 | rf.votedFor = args.CandidateId 51 | rf.persist() 52 | rf.resetElectionTimer() 53 | DPrintf("[%v]: term %v vote %v", rf.me, rf.currentTerm, rf.votedFor) 54 | } else { 55 | reply.VoteGranted = false 56 | } 57 | reply.Term = rf.currentTerm 58 | } 59 | 60 | // example code to send a RequestVote RPC to a server. 61 | // server is the index of the target server in rf.peers[]. 62 | // expects RPC arguments in args. 63 | // fills in *reply with RPC reply, so caller should 64 | // pass &reply. 65 | // the types of the args and reply passed to Call() must be 66 | // the same as the types of the arguments declared in the 67 | // handler function (including whether they are pointers). 68 | // 69 | // The labrpc package simulates a lossy network, in which servers 70 | // may be unreachable, and in which requests and replies may be lost. 71 | // Call() sends a request and waits for a reply. If a reply arrives 72 | // within a timeout interval, Call() returns true; otherwise 73 | // Call() returns false. Thus Call() may not return for a while. 74 | // A false return can be caused by a dead server, a live server that 75 | // can't be reached, a lost request, or a lost reply. 76 | // 77 | // Call() is guaranteed to return (perhaps after a delay) *except* if the 78 | // handler function on the server side does not return. Thus there 79 | // is no need to implement your own timeouts around Call(). 80 | // 81 | // look at the comments in 6.824/labrpc/labrpc.go for more details. 82 | // 83 | // if you're having trouble getting RPC to work, check that you've 84 | // capitalized all field names in structs passed over RPC, and 85 | // that the caller passes the address of the reply struct with &, not 86 | // the struct itself. 87 | func (rf *Raft) sendRequestVote(server int, args *RequestVoteArgs, reply *RequestVoteReply) bool { 88 | ok := rf.peers[server].Call("Raft.RequestVote", args, reply) 89 | return ok 90 | } 91 | 92 | func (rf *Raft) candidateRequestVote(serverId int, args *RequestVoteArgs, voteCounter *int, becomeLeader *sync.Once) { 93 | DPrintf("[%d]: term %v send vote request to %d\n", rf.me, args.Term, serverId) 94 | reply := RequestVoteReply{} 95 | ok := rf.sendRequestVote(serverId, args, &reply) 96 | if !ok { 97 | return 98 | } 99 | rf.mu.Lock() 100 | defer rf.mu.Unlock() 101 | if reply.Term > args.Term { 102 | DPrintf("[%d]: %d 在新的term,更新term,结束\n", rf.me, serverId) 103 | rf.setNewTerm(reply.Term) 104 | return 105 | } 106 | if reply.Term < args.Term { 107 | DPrintf("[%d]: %d 的term %d 已经失效,结束\n", rf.me, serverId, reply.Term) 108 | return 109 | } 110 | if !reply.VoteGranted { 111 | DPrintf("[%d]: %d 没有投给me,结束\n", rf.me, serverId) 112 | return 113 | } 114 | DPrintf("[%d]: from %d term一致,且投给%d\n", rf.me, serverId, rf.me) 115 | 116 | *voteCounter++ 117 | 118 | if *voteCounter > len(rf.peers)/2 && 119 | rf.currentTerm == args.Term && 120 | rf.state == Candidate { 121 | DPrintf("[%d]: 获得多数选票,可以提前结束\n", rf.me) 122 | becomeLeader.Do(func() { 123 | DPrintf("[%d]: 当前term %d 结束\n", rf.me, rf.currentTerm) 124 | rf.state = Leader 125 | lastLogIndex := rf.log.lastLog().Index 126 | for i, _ := range rf.peers { 127 | rf.nextIndex[i] = lastLogIndex + 1 128 | rf.matchIndex[i] = 0 129 | } 130 | DPrintf("[%d]: leader - nextIndex %#v", rf.me, rf.nextIndex) 131 | rf.appendEntries(true) 132 | }) 133 | } 134 | } 135 | -------------------------------------------------------------------------------- /src/raft/test.sh: -------------------------------------------------------------------------------- 1 | #!/bin/zsh 2 | 3 | sh ./go-test-many.sh 1000 4 2A -------------------------------------------------------------------------------- /src/raft/test_test.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | // 4 | // Raft tests. 5 | // 6 | // we will use the original test_test.go to test your code for grading. 7 | // so, while you can modify this code to help you debug, please 8 | // test with the original before submitting. 9 | // 10 | 11 | import "testing" 12 | import "fmt" 13 | import "time" 14 | import "math/rand" 15 | import "sync/atomic" 16 | import "sync" 17 | 18 | // The tester generously allows solutions to complete elections in one second 19 | // (much more than the paper's range of timeouts). 20 | const RaftElectionTimeout = 1000 * time.Millisecond 21 | 22 | func TestInitialElection2A(t *testing.T) { 23 | servers := 3 24 | cfg := make_config(t, servers, false, false) 25 | defer cfg.cleanup() 26 | 27 | cfg.begin("Test (2A): initial election") 28 | 29 | // is a leader elected? 30 | cfg.checkOneLeader() 31 | 32 | // sleep a bit to avoid racing with followers learning of the 33 | // election, then check that all peers agree on the term. 34 | time.Sleep(50 * time.Millisecond) 35 | term1 := cfg.checkTerms() 36 | if term1 < 1 { 37 | t.Fatalf("term is %v, but should be at least 1", term1) 38 | } 39 | 40 | // does the leader+term stay the same if there is no network failure? 41 | time.Sleep(2 * RaftElectionTimeout) 42 | term2 := cfg.checkTerms() 43 | if term1 != term2 { 44 | fmt.Printf("warning: term changed even though there were no failures") 45 | } 46 | 47 | // there should still be a leader. 48 | cfg.checkOneLeader() 49 | 50 | cfg.end() 51 | } 52 | 53 | func TestReElection2A(t *testing.T) { 54 | servers := 3 55 | cfg := make_config(t, servers, false, false) 56 | defer cfg.cleanup() 57 | 58 | cfg.begin("Test (2A): election after network failure") 59 | 60 | leader1 := cfg.checkOneLeader() 61 | 62 | // if the leader disconnects, a new one should be elected. 63 | cfg.disconnect(leader1) 64 | cfg.checkOneLeader() 65 | 66 | // if the old leader rejoins, that shouldn't 67 | // disturb the new leader. 68 | cfg.connect(leader1) 69 | leader2 := cfg.checkOneLeader() 70 | 71 | // if there's no quorum, no leader should 72 | // be elected. 73 | cfg.disconnect(leader2) 74 | cfg.disconnect((leader2 + 1) % servers) 75 | time.Sleep(2 * RaftElectionTimeout) 76 | cfg.checkNoLeader() 77 | 78 | // if a quorum arises, it should elect a leader. 79 | cfg.connect((leader2 + 1) % servers) 80 | cfg.checkOneLeader() 81 | 82 | // re-join of last node shouldn't prevent leader from existing. 83 | cfg.connect(leader2) 84 | cfg.checkOneLeader() 85 | 86 | cfg.end() 87 | } 88 | 89 | func TestManyElections2A(t *testing.T) { 90 | servers := 7 91 | cfg := make_config(t, servers, false, false) 92 | defer cfg.cleanup() 93 | 94 | cfg.begin("Test (2A): multiple elections") 95 | 96 | cfg.checkOneLeader() 97 | 98 | iters := 10 99 | for ii := 1; ii < iters; ii++ { 100 | // disconnect three nodes 101 | i1 := rand.Int() % servers 102 | i2 := rand.Int() % servers 103 | i3 := rand.Int() % servers 104 | cfg.disconnect(i1) 105 | cfg.disconnect(i2) 106 | cfg.disconnect(i3) 107 | 108 | // either the current leader should still be alive, 109 | // or the remaining four should elect a new one. 110 | cfg.checkOneLeader() 111 | 112 | cfg.connect(i1) 113 | cfg.connect(i2) 114 | cfg.connect(i3) 115 | } 116 | 117 | cfg.checkOneLeader() 118 | 119 | cfg.end() 120 | } 121 | 122 | func TestBasicAgree2B(t *testing.T) { 123 | servers := 3 124 | cfg := make_config(t, servers, false, false) 125 | defer cfg.cleanup() 126 | 127 | cfg.begin("Test (2B): basic agreement") 128 | 129 | iters := 3 130 | for index := 1; index < iters+1; index++ { 131 | nd, _ := cfg.nCommitted(index) 132 | if nd > 0 { 133 | t.Fatalf("some have committed before Start()") 134 | } 135 | 136 | xindex := cfg.one(index*100, servers, false) 137 | if xindex != index { 138 | t.Fatalf("got index %v but expected %v", xindex, index) 139 | } 140 | } 141 | 142 | cfg.end() 143 | } 144 | 145 | // 146 | // check, based on counting bytes of RPCs, that 147 | // each command is sent to each peer just once. 148 | // 149 | func TestRPCBytes2B(t *testing.T) { 150 | servers := 3 151 | cfg := make_config(t, servers, false, false) 152 | defer cfg.cleanup() 153 | 154 | cfg.begin("Test (2B): RPC byte count") 155 | 156 | cfg.one(99, servers, false) 157 | bytes0 := cfg.bytesTotal() 158 | 159 | iters := 10 160 | var sent int64 = 0 161 | for index := 2; index < iters+2; index++ { 162 | cmd := randstring(5000) 163 | xindex := cfg.one(cmd, servers, false) 164 | if xindex != index { 165 | t.Fatalf("got index %v but expected %v", xindex, index) 166 | } 167 | sent += int64(len(cmd)) 168 | } 169 | 170 | bytes1 := cfg.bytesTotal() 171 | got := bytes1 - bytes0 172 | expected := int64(servers) * sent 173 | if got > expected+50000 { 174 | t.Fatalf("too many RPC bytes; got %v, expected %v", got, expected) 175 | } 176 | 177 | cfg.end() 178 | } 179 | 180 | func TestFailAgree2B(t *testing.T) { 181 | servers := 3 182 | cfg := make_config(t, servers, false, false) 183 | defer cfg.cleanup() 184 | 185 | cfg.begin("Test (2B): agreement despite follower disconnection") 186 | 187 | cfg.one(101, servers, false) 188 | 189 | // disconnect one follower from the network. 190 | leader := cfg.checkOneLeader() 191 | cfg.disconnect((leader + 1) % servers) 192 | 193 | // the leader and remaining follower should be 194 | // able to agree despite the disconnected follower. 195 | cfg.one(102, servers-1, false) 196 | cfg.one(103, servers-1, false) 197 | time.Sleep(RaftElectionTimeout) 198 | cfg.one(104, servers-1, false) 199 | cfg.one(105, servers-1, false) 200 | 201 | // re-connect 202 | cfg.connect((leader + 1) % servers) 203 | 204 | // the full set of servers should preserve 205 | // previous agreements, and be able to agree 206 | // on new commands. 207 | cfg.one(106, servers, true) 208 | time.Sleep(RaftElectionTimeout) 209 | cfg.one(107, servers, true) 210 | 211 | cfg.end() 212 | } 213 | 214 | func TestFailNoAgree2B(t *testing.T) { 215 | servers := 5 216 | cfg := make_config(t, servers, false, false) 217 | defer cfg.cleanup() 218 | 219 | cfg.begin("Test (2B): no agreement if too many followers disconnect") 220 | 221 | cfg.one(10, servers, false) 222 | 223 | // 3 of 5 followers disconnect 224 | leader := cfg.checkOneLeader() 225 | cfg.disconnect((leader + 1) % servers) 226 | cfg.disconnect((leader + 2) % servers) 227 | cfg.disconnect((leader + 3) % servers) 228 | 229 | index, _, ok := cfg.rafts[leader].Start(20) 230 | if ok != true { 231 | t.Fatalf("leader rejected Start()") 232 | } 233 | if index != 2 { 234 | t.Fatalf("expected index 2, got %v", index) 235 | } 236 | 237 | time.Sleep(2 * RaftElectionTimeout) 238 | 239 | n, _ := cfg.nCommitted(index) 240 | if n > 0 { 241 | t.Fatalf("%v committed but no majority", n) 242 | } 243 | 244 | // repair 245 | cfg.connect((leader + 1) % servers) 246 | cfg.connect((leader + 2) % servers) 247 | cfg.connect((leader + 3) % servers) 248 | 249 | // the disconnected majority may have chosen a leader from 250 | // among their own ranks, forgetting index 2. 251 | leader2 := cfg.checkOneLeader() 252 | index2, _, ok2 := cfg.rafts[leader2].Start(30) 253 | if ok2 == false { 254 | t.Fatalf("leader2 rejected Start()") 255 | } 256 | if index2 < 2 || index2 > 3 { 257 | t.Fatalf("unexpected index %v", index2) 258 | } 259 | 260 | cfg.one(1000, servers, true) 261 | 262 | cfg.end() 263 | } 264 | 265 | func TestConcurrentStarts2B(t *testing.T) { 266 | servers := 3 267 | cfg := make_config(t, servers, false, false) 268 | defer cfg.cleanup() 269 | 270 | cfg.begin("Test (2B): concurrent Start()s") 271 | 272 | var success bool 273 | loop: 274 | for try := 0; try < 5; try++ { 275 | if try > 0 { 276 | // give solution some time to settle 277 | time.Sleep(3 * time.Second) 278 | } 279 | 280 | leader := cfg.checkOneLeader() 281 | _, term, ok := cfg.rafts[leader].Start(1) 282 | if !ok { 283 | // leader moved on really quickly 284 | continue 285 | } 286 | 287 | iters := 5 288 | var wg sync.WaitGroup 289 | is := make(chan int, iters) 290 | for ii := 0; ii < iters; ii++ { 291 | wg.Add(1) 292 | go func(i int) { 293 | defer wg.Done() 294 | i, term1, ok := cfg.rafts[leader].Start(100 + i) 295 | if term1 != term { 296 | return 297 | } 298 | if ok != true { 299 | return 300 | } 301 | is <- i 302 | }(ii) 303 | } 304 | 305 | wg.Wait() 306 | close(is) 307 | 308 | for j := 0; j < servers; j++ { 309 | if t, _ := cfg.rafts[j].GetState(); t != term { 310 | // term changed -- can't expect low RPC counts 311 | continue loop 312 | } 313 | } 314 | 315 | failed := false 316 | cmds := []int{} 317 | for index := range is { 318 | cmd := cfg.wait(index, servers, term) 319 | if ix, ok := cmd.(int); ok { 320 | if ix == -1 { 321 | // peers have moved on to later terms 322 | // so we can't expect all Start()s to 323 | // have succeeded 324 | failed = true 325 | break 326 | } 327 | cmds = append(cmds, ix) 328 | } else { 329 | t.Fatalf("value %v is not an int", cmd) 330 | } 331 | } 332 | 333 | if failed { 334 | // avoid leaking goroutines 335 | go func() { 336 | for range is { 337 | } 338 | }() 339 | continue 340 | } 341 | 342 | for ii := 0; ii < iters; ii++ { 343 | x := 100 + ii 344 | ok := false 345 | for j := 0; j < len(cmds); j++ { 346 | if cmds[j] == x { 347 | ok = true 348 | } 349 | } 350 | if ok == false { 351 | t.Fatalf("cmd %v missing in %v", x, cmds) 352 | } 353 | } 354 | 355 | success = true 356 | break 357 | } 358 | 359 | if !success { 360 | t.Fatalf("term changed too often") 361 | } 362 | 363 | cfg.end() 364 | } 365 | 366 | func TestRejoin2B(t *testing.T) { 367 | servers := 3 368 | cfg := make_config(t, servers, false, false) 369 | defer cfg.cleanup() 370 | 371 | cfg.begin("Test (2B): rejoin of partitioned leader") 372 | 373 | cfg.one(101, servers, true) 374 | 375 | // leader network failure 376 | leader1 := cfg.checkOneLeader() 377 | cfg.disconnect(leader1) 378 | 379 | // make old leader try to agree on some entries 380 | cfg.rafts[leader1].Start(102) 381 | cfg.rafts[leader1].Start(103) 382 | cfg.rafts[leader1].Start(104) 383 | 384 | // new leader commits, also for index=2 385 | cfg.one(103, 2, true) 386 | 387 | // new leader network failure 388 | leader2 := cfg.checkOneLeader() 389 | cfg.disconnect(leader2) 390 | 391 | // old leader connected again 392 | cfg.connect(leader1) 393 | 394 | cfg.one(104, 2, true) 395 | 396 | // all together now 397 | cfg.connect(leader2) 398 | 399 | cfg.one(105, servers, true) 400 | 401 | cfg.end() 402 | } 403 | 404 | func TestBackup2B(t *testing.T) { 405 | servers := 5 406 | cfg := make_config(t, servers, false, false) 407 | defer cfg.cleanup() 408 | 409 | cfg.begin("Test (2B): leader backs up quickly over incorrect follower logs") 410 | 411 | cfg.one(rand.Int(), servers, true) 412 | 413 | // put leader and one follower in a partition 414 | leader1 := cfg.checkOneLeader() 415 | cfg.disconnect((leader1 + 2) % servers) 416 | cfg.disconnect((leader1 + 3) % servers) 417 | cfg.disconnect((leader1 + 4) % servers) 418 | 419 | // submit lots of commands that won't commit 420 | for i := 0; i < 50; i++ { 421 | cfg.rafts[leader1].Start(rand.Int()) 422 | } 423 | 424 | time.Sleep(RaftElectionTimeout / 2) 425 | 426 | cfg.disconnect((leader1 + 0) % servers) 427 | cfg.disconnect((leader1 + 1) % servers) 428 | 429 | // allow other partition to recover 430 | cfg.connect((leader1 + 2) % servers) 431 | cfg.connect((leader1 + 3) % servers) 432 | cfg.connect((leader1 + 4) % servers) 433 | 434 | // lots of successful commands to new group. 435 | for i := 0; i < 50; i++ { 436 | cfg.one(rand.Int(), 3, true) 437 | } 438 | 439 | // now another partitioned leader and one follower 440 | leader2 := cfg.checkOneLeader() 441 | other := (leader1 + 2) % servers 442 | if leader2 == other { 443 | other = (leader2 + 1) % servers 444 | } 445 | cfg.disconnect(other) 446 | 447 | // lots more commands that won't commit 448 | for i := 0; i < 50; i++ { 449 | cfg.rafts[leader2].Start(rand.Int()) 450 | } 451 | 452 | time.Sleep(RaftElectionTimeout / 2) 453 | 454 | // bring original leader back to life, 455 | for i := 0; i < servers; i++ { 456 | cfg.disconnect(i) 457 | } 458 | cfg.connect((leader1 + 0) % servers) 459 | cfg.connect((leader1 + 1) % servers) 460 | cfg.connect(other) 461 | 462 | // lots of successful commands to new group. 463 | for i := 0; i < 50; i++ { 464 | cfg.one(rand.Int(), 3, true) 465 | } 466 | 467 | // now everyone 468 | for i := 0; i < servers; i++ { 469 | cfg.connect(i) 470 | } 471 | cfg.one(rand.Int(), servers, true) 472 | 473 | cfg.end() 474 | } 475 | 476 | func TestCount2B(t *testing.T) { 477 | servers := 3 478 | cfg := make_config(t, servers, false, false) 479 | defer cfg.cleanup() 480 | 481 | cfg.begin("Test (2B): RPC counts aren't too high") 482 | 483 | rpcs := func() (n int) { 484 | for j := 0; j < servers; j++ { 485 | n += cfg.rpcCount(j) 486 | } 487 | return 488 | } 489 | 490 | leader := cfg.checkOneLeader() 491 | 492 | total1 := rpcs() 493 | 494 | if total1 > 30 || total1 < 1 { 495 | t.Fatalf("too many or few RPCs (%v) to elect initial leader\n", total1) 496 | } 497 | 498 | var total2 int 499 | var success bool 500 | loop: 501 | for try := 0; try < 5; try++ { 502 | if try > 0 { 503 | // give solution some time to settle 504 | time.Sleep(3 * time.Second) 505 | } 506 | 507 | leader = cfg.checkOneLeader() 508 | total1 = rpcs() 509 | 510 | iters := 10 511 | starti, term, ok := cfg.rafts[leader].Start(1) 512 | if !ok { 513 | // leader moved on really quickly 514 | continue 515 | } 516 | cmds := []int{} 517 | for i := 1; i < iters+2; i++ { 518 | x := int(rand.Int31()) 519 | cmds = append(cmds, x) 520 | index1, term1, ok := cfg.rafts[leader].Start(x) 521 | if term1 != term { 522 | // Term changed while starting 523 | continue loop 524 | } 525 | if !ok { 526 | // No longer the leader, so term has changed 527 | continue loop 528 | } 529 | if starti+i != index1 { 530 | t.Fatalf("Start() failed") 531 | } 532 | } 533 | 534 | for i := 1; i < iters+1; i++ { 535 | cmd := cfg.wait(starti+i, servers, term) 536 | if ix, ok := cmd.(int); ok == false || ix != cmds[i-1] { 537 | if ix == -1 { 538 | // term changed -- try again 539 | continue loop 540 | } 541 | t.Fatalf("wrong value %v committed for index %v; expected %v\n", cmd, starti+i, cmds) 542 | } 543 | } 544 | 545 | failed := false 546 | total2 = 0 547 | for j := 0; j < servers; j++ { 548 | if t, _ := cfg.rafts[j].GetState(); t != term { 549 | // term changed -- can't expect low RPC counts 550 | // need to keep going to update total2 551 | failed = true 552 | } 553 | total2 += cfg.rpcCount(j) 554 | } 555 | 556 | if failed { 557 | continue loop 558 | } 559 | 560 | if total2-total1 > (iters+1+3)*3 { 561 | t.Fatalf("too many RPCs (%v) for %v entries\n", total2-total1, iters) 562 | } 563 | 564 | success = true 565 | break 566 | } 567 | 568 | if !success { 569 | t.Fatalf("term changed too often") 570 | } 571 | 572 | time.Sleep(RaftElectionTimeout) 573 | 574 | total3 := 0 575 | for j := 0; j < servers; j++ { 576 | total3 += cfg.rpcCount(j) 577 | } 578 | 579 | if total3-total2 > 3*20 { 580 | t.Fatalf("too many RPCs (%v) for 1 second of idleness\n", total3-total2) 581 | } 582 | 583 | cfg.end() 584 | } 585 | 586 | func TestPersist12C(t *testing.T) { 587 | servers := 3 588 | cfg := make_config(t, servers, false, false) 589 | defer cfg.cleanup() 590 | 591 | cfg.begin("Test (2C): basic persistence") 592 | 593 | cfg.one(11, servers, true) 594 | 595 | // crash and re-start all 596 | for i := 0; i < servers; i++ { 597 | cfg.start1(i, cfg.applier) 598 | } 599 | for i := 0; i < servers; i++ { 600 | cfg.disconnect(i) 601 | cfg.connect(i) 602 | } 603 | 604 | cfg.one(12, servers, true) 605 | 606 | leader1 := cfg.checkOneLeader() 607 | cfg.disconnect(leader1) 608 | cfg.start1(leader1, cfg.applier) 609 | cfg.connect(leader1) 610 | 611 | cfg.one(13, servers, true) 612 | 613 | leader2 := cfg.checkOneLeader() 614 | cfg.disconnect(leader2) 615 | cfg.one(14, servers-1, true) 616 | cfg.start1(leader2, cfg.applier) 617 | cfg.connect(leader2) 618 | 619 | cfg.wait(4, servers, -1) // wait for leader2 to join before killing i3 620 | 621 | i3 := (cfg.checkOneLeader() + 1) % servers 622 | cfg.disconnect(i3) 623 | cfg.one(15, servers-1, true) 624 | cfg.start1(i3, cfg.applier) 625 | cfg.connect(i3) 626 | 627 | cfg.one(16, servers, true) 628 | 629 | cfg.end() 630 | } 631 | 632 | func TestPersist22C(t *testing.T) { 633 | servers := 5 634 | cfg := make_config(t, servers, false, false) 635 | defer cfg.cleanup() 636 | 637 | cfg.begin("Test (2C): more persistence") 638 | 639 | index := 1 640 | for iters := 0; iters < 5; iters++ { 641 | cfg.one(10+index, servers, true) 642 | index++ 643 | 644 | leader1 := cfg.checkOneLeader() 645 | 646 | cfg.disconnect((leader1 + 1) % servers) 647 | cfg.disconnect((leader1 + 2) % servers) 648 | 649 | cfg.one(10+index, servers-2, true) 650 | index++ 651 | 652 | cfg.disconnect((leader1 + 0) % servers) 653 | cfg.disconnect((leader1 + 3) % servers) 654 | cfg.disconnect((leader1 + 4) % servers) 655 | 656 | cfg.start1((leader1+1)%servers, cfg.applier) 657 | cfg.start1((leader1+2)%servers, cfg.applier) 658 | cfg.connect((leader1 + 1) % servers) 659 | cfg.connect((leader1 + 2) % servers) 660 | 661 | time.Sleep(RaftElectionTimeout) 662 | 663 | cfg.start1((leader1+3)%servers, cfg.applier) 664 | cfg.connect((leader1 + 3) % servers) 665 | 666 | cfg.one(10+index, servers-2, true) 667 | index++ 668 | 669 | cfg.connect((leader1 + 4) % servers) 670 | cfg.connect((leader1 + 0) % servers) 671 | } 672 | 673 | cfg.one(1000, servers, true) 674 | 675 | cfg.end() 676 | } 677 | 678 | func TestPersist32C(t *testing.T) { 679 | servers := 3 680 | cfg := make_config(t, servers, false, false) 681 | defer cfg.cleanup() 682 | 683 | cfg.begin("Test (2C): partitioned leader and one follower crash, leader restarts") 684 | 685 | cfg.one(101, 3, true) 686 | 687 | leader := cfg.checkOneLeader() 688 | cfg.disconnect((leader + 2) % servers) 689 | 690 | cfg.one(102, 2, true) 691 | 692 | cfg.crash1((leader + 0) % servers) 693 | cfg.crash1((leader + 1) % servers) 694 | cfg.connect((leader + 2) % servers) 695 | cfg.start1((leader+0)%servers, cfg.applier) 696 | cfg.connect((leader + 0) % servers) 697 | 698 | cfg.one(103, 2, true) 699 | 700 | cfg.start1((leader+1)%servers, cfg.applier) 701 | cfg.connect((leader + 1) % servers) 702 | 703 | cfg.one(104, servers, true) 704 | 705 | cfg.end() 706 | } 707 | 708 | // 709 | // Test the scenarios described in Figure 8 of the extended Raft paper. Each 710 | // iteration asks a leader, if there is one, to insert a command in the Raft 711 | // log. If there is a leader, that leader will fail quickly with a high 712 | // probability (perhaps without committing the command), or crash after a while 713 | // with low probability (most likey committing the command). If the number of 714 | // alive servers isn't enough to form a majority, perhaps start a new server. 715 | // The leader in a new term may try to finish replicating log entries that 716 | // haven't been committed yet. 717 | // 718 | func TestFigure82C(t *testing.T) { 719 | servers := 5 720 | cfg := make_config(t, servers, false, false) 721 | defer cfg.cleanup() 722 | 723 | cfg.begin("Test (2C): Figure 8") 724 | 725 | cfg.one(rand.Int(), 1, true) 726 | 727 | nup := servers 728 | for iters := 0; iters < 1000; iters++ { 729 | leader := -1 730 | for i := 0; i < servers; i++ { 731 | if cfg.rafts[i] != nil { 732 | _, _, ok := cfg.rafts[i].Start(rand.Int()) 733 | if ok { 734 | leader = i 735 | } 736 | } 737 | } 738 | 739 | if (rand.Int() % 1000) < 100 { 740 | ms := rand.Int63() % (int64(RaftElectionTimeout/time.Millisecond) / 2) 741 | time.Sleep(time.Duration(ms) * time.Millisecond) 742 | } else { 743 | ms := (rand.Int63() % 13) 744 | time.Sleep(time.Duration(ms) * time.Millisecond) 745 | } 746 | 747 | if leader != -1 { 748 | cfg.crash1(leader) 749 | nup -= 1 750 | } 751 | 752 | if nup < 3 { 753 | s := rand.Int() % servers 754 | if cfg.rafts[s] == nil { 755 | cfg.start1(s, cfg.applier) 756 | cfg.connect(s) 757 | nup += 1 758 | } 759 | } 760 | } 761 | 762 | for i := 0; i < servers; i++ { 763 | if cfg.rafts[i] == nil { 764 | cfg.start1(i, cfg.applier) 765 | cfg.connect(i) 766 | } 767 | } 768 | 769 | cfg.one(rand.Int(), servers, true) 770 | 771 | cfg.end() 772 | } 773 | 774 | func TestUnreliableAgree2C(t *testing.T) { 775 | servers := 5 776 | cfg := make_config(t, servers, true, false) 777 | defer cfg.cleanup() 778 | 779 | cfg.begin("Test (2C): unreliable agreement") 780 | 781 | var wg sync.WaitGroup 782 | 783 | for iters := 1; iters < 50; iters++ { 784 | for j := 0; j < 4; j++ { 785 | wg.Add(1) 786 | go func(iters, j int) { 787 | defer wg.Done() 788 | cfg.one((100*iters)+j, 1, true) 789 | }(iters, j) 790 | } 791 | cfg.one(iters, 1, true) 792 | } 793 | 794 | cfg.setunreliable(false) 795 | 796 | wg.Wait() 797 | 798 | cfg.one(100, servers, true) 799 | 800 | cfg.end() 801 | } 802 | 803 | func TestFigure8Unreliable2C(t *testing.T) { 804 | servers := 5 805 | cfg := make_config(t, servers, true, false) 806 | defer cfg.cleanup() 807 | 808 | cfg.begin("Test (2C): Figure 8 (unreliable)") 809 | 810 | cfg.one(rand.Int()%10000, 1, true) 811 | 812 | nup := servers 813 | for iters := 0; iters < 1000; iters++ { 814 | if iters == 200 { 815 | cfg.setlongreordering(true) 816 | } 817 | leader := -1 818 | for i := 0; i < servers; i++ { 819 | _, _, ok := cfg.rafts[i].Start(rand.Int() % 10000) 820 | if ok && cfg.connected[i] { 821 | leader = i 822 | } 823 | } 824 | 825 | if (rand.Int() % 1000) < 100 { 826 | ms := rand.Int63() % (int64(RaftElectionTimeout/time.Millisecond) / 2) 827 | time.Sleep(time.Duration(ms) * time.Millisecond) 828 | } else { 829 | ms := (rand.Int63() % 13) 830 | time.Sleep(time.Duration(ms) * time.Millisecond) 831 | } 832 | 833 | if leader != -1 && (rand.Int()%1000) < int(RaftElectionTimeout/time.Millisecond)/2 { 834 | cfg.disconnect(leader) 835 | nup -= 1 836 | } 837 | 838 | if nup < 3 { 839 | s := rand.Int() % servers 840 | if cfg.connected[s] == false { 841 | cfg.connect(s) 842 | nup += 1 843 | } 844 | } 845 | } 846 | 847 | for i := 0; i < servers; i++ { 848 | if cfg.connected[i] == false { 849 | cfg.connect(i) 850 | } 851 | } 852 | 853 | cfg.one(rand.Int()%10000, servers, true) 854 | 855 | cfg.end() 856 | } 857 | 858 | func internalChurn(t *testing.T, unreliable bool) { 859 | 860 | servers := 5 861 | cfg := make_config(t, servers, unreliable, false) 862 | defer cfg.cleanup() 863 | 864 | if unreliable { 865 | cfg.begin("Test (2C): unreliable churn") 866 | } else { 867 | cfg.begin("Test (2C): churn") 868 | } 869 | 870 | stop := int32(0) 871 | 872 | // create concurrent clients 873 | cfn := func(me int, ch chan []int) { 874 | var ret []int 875 | ret = nil 876 | defer func() { ch <- ret }() 877 | values := []int{} 878 | for atomic.LoadInt32(&stop) == 0 { 879 | x := rand.Int() 880 | index := -1 881 | ok := false 882 | for i := 0; i < servers; i++ { 883 | // try them all, maybe one of them is a leader 884 | cfg.mu.Lock() 885 | rf := cfg.rafts[i] 886 | cfg.mu.Unlock() 887 | if rf != nil { 888 | index1, _, ok1 := rf.Start(x) 889 | if ok1 { 890 | ok = ok1 891 | index = index1 892 | } 893 | } 894 | } 895 | if ok { 896 | // maybe leader will commit our value, maybe not. 897 | // but don't wait forever. 898 | for _, to := range []int{10, 20, 50, 100, 200} { 899 | nd, cmd := cfg.nCommitted(index) 900 | if nd > 0 { 901 | if xx, ok := cmd.(int); ok { 902 | if xx == x { 903 | values = append(values, x) 904 | } 905 | } else { 906 | cfg.t.Fatalf("wrong command type") 907 | } 908 | break 909 | } 910 | time.Sleep(time.Duration(to) * time.Millisecond) 911 | } 912 | } else { 913 | time.Sleep(time.Duration(79+me*17) * time.Millisecond) 914 | } 915 | } 916 | ret = values 917 | } 918 | 919 | ncli := 3 920 | cha := []chan []int{} 921 | for i := 0; i < ncli; i++ { 922 | cha = append(cha, make(chan []int)) 923 | go cfn(i, cha[i]) 924 | } 925 | 926 | for iters := 0; iters < 20; iters++ { 927 | if (rand.Int() % 1000) < 200 { 928 | i := rand.Int() % servers 929 | cfg.disconnect(i) 930 | } 931 | 932 | if (rand.Int() % 1000) < 500 { 933 | i := rand.Int() % servers 934 | if cfg.rafts[i] == nil { 935 | cfg.start1(i, cfg.applier) 936 | } 937 | cfg.connect(i) 938 | } 939 | 940 | if (rand.Int() % 1000) < 200 { 941 | i := rand.Int() % servers 942 | if cfg.rafts[i] != nil { 943 | cfg.crash1(i) 944 | } 945 | } 946 | 947 | // Make crash/restart infrequent enough that the peers can often 948 | // keep up, but not so infrequent that everything has settled 949 | // down from one change to the next. Pick a value smaller than 950 | // the election timeout, but not hugely smaller. 951 | time.Sleep((RaftElectionTimeout * 7) / 10) 952 | } 953 | 954 | time.Sleep(RaftElectionTimeout) 955 | cfg.setunreliable(false) 956 | for i := 0; i < servers; i++ { 957 | if cfg.rafts[i] == nil { 958 | cfg.start1(i, cfg.applier) 959 | } 960 | cfg.connect(i) 961 | } 962 | 963 | atomic.StoreInt32(&stop, 1) 964 | 965 | values := []int{} 966 | for i := 0; i < ncli; i++ { 967 | vv := <-cha[i] 968 | if vv == nil { 969 | t.Fatal("client failed") 970 | } 971 | values = append(values, vv...) 972 | } 973 | 974 | time.Sleep(RaftElectionTimeout) 975 | 976 | lastIndex := cfg.one(rand.Int(), servers, true) 977 | 978 | really := make([]int, lastIndex+1) 979 | for index := 1; index <= lastIndex; index++ { 980 | v := cfg.wait(index, servers, -1) 981 | if vi, ok := v.(int); ok { 982 | really = append(really, vi) 983 | } else { 984 | t.Fatalf("not an int") 985 | } 986 | } 987 | 988 | for _, v1 := range values { 989 | ok := false 990 | for _, v2 := range really { 991 | if v1 == v2 { 992 | ok = true 993 | } 994 | } 995 | if ok == false { 996 | cfg.t.Fatalf("didn't find a value") 997 | } 998 | } 999 | 1000 | cfg.end() 1001 | } 1002 | 1003 | func TestReliableChurn2C(t *testing.T) { 1004 | internalChurn(t, false) 1005 | } 1006 | 1007 | func TestUnreliableChurn2C(t *testing.T) { 1008 | internalChurn(t, true) 1009 | } 1010 | 1011 | const MAXLOGSIZE = 2000 1012 | 1013 | func snapcommon(t *testing.T, name string, disconnect bool, reliable bool, crash bool) { 1014 | iters := 30 1015 | servers := 3 1016 | cfg := make_config(t, servers, !reliable, true) 1017 | defer cfg.cleanup() 1018 | 1019 | cfg.begin(name) 1020 | 1021 | cfg.one(rand.Int(), servers, true) 1022 | leader1 := cfg.checkOneLeader() 1023 | 1024 | for i := 0; i < iters; i++ { 1025 | victim := (leader1 + 1) % servers 1026 | sender := leader1 1027 | if i%3 == 1 { 1028 | sender = (leader1 + 1) % servers 1029 | victim = leader1 1030 | } 1031 | 1032 | if disconnect { 1033 | cfg.disconnect(victim) 1034 | cfg.one(rand.Int(), servers-1, true) 1035 | } 1036 | if crash { 1037 | cfg.crash1(victim) 1038 | cfg.one(rand.Int(), servers-1, true) 1039 | } 1040 | // send enough to get a snapshot 1041 | for i := 0; i < SnapShotInterval+1; i++ { 1042 | cfg.rafts[sender].Start(rand.Int()) 1043 | } 1044 | // let applier threads catch up with the Start()'s 1045 | cfg.one(rand.Int(), servers-1, true) 1046 | 1047 | if cfg.LogSize() >= MAXLOGSIZE { 1048 | cfg.t.Fatalf("Log size too large") 1049 | } 1050 | if disconnect { 1051 | // reconnect a follower, who maybe behind and 1052 | // needs to rceive a snapshot to catch up. 1053 | cfg.connect(victim) 1054 | cfg.one(rand.Int(), servers, true) 1055 | leader1 = cfg.checkOneLeader() 1056 | } 1057 | if crash { 1058 | cfg.start1(victim, cfg.applierSnap) 1059 | cfg.connect(victim) 1060 | cfg.one(rand.Int(), servers, true) 1061 | leader1 = cfg.checkOneLeader() 1062 | } 1063 | } 1064 | cfg.end() 1065 | } 1066 | 1067 | func TestSnapshotBasic2D(t *testing.T) { 1068 | snapcommon(t, "Test (2D): snapshots basic", false, true, false) 1069 | } 1070 | 1071 | func TestSnapshotInstall2D(t *testing.T) { 1072 | snapcommon(t, "Test (2D): install snapshots (disconnect)", true, true, false) 1073 | } 1074 | 1075 | func TestSnapshotInstallUnreliable2D(t *testing.T) { 1076 | snapcommon(t, "Test (2D): install snapshots (disconnect+unreliable)", 1077 | true, false, false) 1078 | } 1079 | 1080 | func TestSnapshotInstallCrash2D(t *testing.T) { 1081 | snapcommon(t, "Test (2D): install snapshots (crash)", false, true, true) 1082 | } 1083 | 1084 | func TestSnapshotInstallUnCrash2D(t *testing.T) { 1085 | snapcommon(t, "Test (2D): install snapshots (unreliable+crash)", false, false, true) 1086 | } -------------------------------------------------------------------------------- /src/raft/util.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import "log" 4 | 5 | // Debugging 6 | const Debug = false 7 | 8 | func DPrintf(format string, a ...interface{}) (n int, err error) { 9 | if Debug { 10 | log.Printf(format, a...) 11 | } 12 | return 13 | } 14 | 15 | func DPrintVerbose(format string, a ...interface{}) (n int, err error) { 16 | if Debug { 17 | log.Printf(format, a...) 18 | } 19 | return 20 | } 21 | 22 | func min(a int, b int) int { 23 | if a > b { 24 | return b 25 | } 26 | return a 27 | } 28 | 29 | func max(a int, b int) int { 30 | if a > b { 31 | return a 32 | } 33 | return b 34 | } 35 | --------------------------------------------------------------------------------