├── .travis.yml ├── .gitignore ├── peers_test.go ├── rpc.go ├── LICENSE ├── example_test.go ├── README.md ├── transport_test.go ├── configuration.go ├── peers.go ├── server_internals_test.go ├── JOINT-CONSENSUS.md ├── transport.go ├── log_test.go ├── server_test.go ├── log.go └── server.go /.travis.yml: -------------------------------------------------------------------------------- 1 | language: go 2 | 3 | go: 4 | - 1.1 5 | - tip 6 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | # Compiled Object files, Static and Dynamic libs (Shared Objects) 2 | *.o 3 | *.a 4 | *.so 5 | 6 | # Folders 7 | _obj 8 | _test 9 | 10 | # Architecture specific extensions/prefixes 11 | *.[568vq] 12 | [568vq].out 13 | 14 | *.cgo1.go 15 | *.cgo2.c 16 | _cgo_defun.c 17 | _cgo_gotypes.go 18 | _cgo_export.* 19 | 20 | _testmain.go 21 | 22 | *.exe 23 | -------------------------------------------------------------------------------- /peers_test.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "testing" 5 | ) 6 | 7 | func TestQuorum(t *testing.T) { 8 | for _, tuple := range []struct { 9 | n int 10 | expected int 11 | }{ 12 | {0, 1}, 13 | {1, 1}, 14 | {2, 2}, 15 | {3, 2}, 16 | {4, 3}, 17 | {5, 3}, 18 | {6, 4}, 19 | {7, 4}, 20 | {8, 5}, 21 | {9, 5}, 22 | {10, 6}, 23 | {11, 6}, 24 | } { 25 | pm := peerMap{} 26 | for i := 0; i < tuple.n; i++ { 27 | pm[uint64(i+1)] = nonresponsivePeer(i + 1) 28 | } 29 | if expected, got := tuple.expected, pm.quorum(); expected != got { 30 | t.Errorf("Quorum of %d: expected %d, got %d", tuple.n, expected, got) 31 | } 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /rpc.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | type appendEntriesTuple struct { 4 | Request appendEntries 5 | Response chan appendEntriesResponse 6 | } 7 | 8 | type requestVoteTuple struct { 9 | Request requestVote 10 | Response chan requestVoteResponse 11 | } 12 | 13 | // appendEntries represents an appendEntries RPC. 14 | type appendEntries struct { 15 | Term uint64 `json:"term"` 16 | LeaderID uint64 `json:"leader_id"` 17 | PrevLogIndex uint64 `json:"prev_log_index"` 18 | PrevLogTerm uint64 `json:"prev_log_term"` 19 | Entries []logEntry `json:"entries"` 20 | CommitIndex uint64 `json:"commit_index"` 21 | } 22 | 23 | // appendEntriesResponse represents the response to an appendEntries RPC. 24 | type appendEntriesResponse struct { 25 | Term uint64 `json:"term"` 26 | Success bool `json:"success"` 27 | reason string 28 | } 29 | 30 | // requestVote represents a requestVote RPC. 31 | type requestVote struct { 32 | Term uint64 `json:"term"` 33 | CandidateID uint64 `json:"candidate_id"` 34 | LastLogIndex uint64 `json:"last_log_index"` 35 | LastLogTerm uint64 `json:"last_log_term"` 36 | } 37 | 38 | // requestVoteResponse represents the response to a requestVote RPC. 39 | type requestVoteResponse struct { 40 | Term uint64 `json:"term"` 41 | VoteGranted bool `json:"vote_granted"` 42 | reason string 43 | } 44 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Copyright (c) 2013, Peter Bourgon, SoundCloud Ltd. 2 | All rights reserved. 3 | 4 | Redistribution and use in source and binary forms, with or without 5 | modification, are permitted provided that the following conditions are met: 6 | 7 | Redistributions of source code must retain the above copyright notice, this 8 | list of conditions and the following disclaimer. 9 | 10 | Redistributions in binary form must reproduce the above copyright notice, this 11 | list of conditions and the following disclaimer in the documentation and/or 12 | other materials provided with the distribution. 13 | 14 | THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND 15 | ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED 16 | WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE 17 | DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE 18 | FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL 19 | DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR 20 | SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER 21 | CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, 22 | OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE 23 | OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 24 | -------------------------------------------------------------------------------- /example_test.go: -------------------------------------------------------------------------------- 1 | package raft_test 2 | 3 | import ( 4 | "bytes" 5 | "fmt" 6 | "net/http" 7 | "net/url" 8 | 9 | "github.com/peterbourgon/raft" 10 | ) 11 | 12 | func ExampleNewServer_hTTP() { 13 | // A no-op ApplyFunc 14 | a := func(uint64, []byte) []byte { return []byte{} } 15 | 16 | // Helper function to parse URLs 17 | mustParseURL := func(rawurl string) *url.URL { 18 | u, err := url.Parse(rawurl) 19 | if err != nil { 20 | panic(err) 21 | } 22 | u.Path = "" 23 | return u 24 | } 25 | 26 | // Helper function to construct HTTP Peers 27 | mustNewHTTPPeer := func(u *url.URL) raft.Peer { 28 | p, err := raft.NewHTTPPeer(u) 29 | if err != nil { 30 | panic(err) 31 | } 32 | return p 33 | } 34 | 35 | // Construct the server 36 | s := raft.NewServer(1, &bytes.Buffer{}, a) 37 | 38 | // Expose the server using a HTTP transport 39 | raft.HTTPTransport(http.DefaultServeMux, s) 40 | go http.ListenAndServe(":8080", nil) 41 | 42 | // Set the initial server configuration 43 | s.SetConfiguration( 44 | mustNewHTTPPeer(mustParseURL("http://127.0.0.1:8080")), // this server 45 | mustNewHTTPPeer(mustParseURL("http://10.1.1.11:8080")), 46 | mustNewHTTPPeer(mustParseURL("http://10.1.1.12:8080")), 47 | mustNewHTTPPeer(mustParseURL("http://10.1.1.13:8080")), 48 | mustNewHTTPPeer(mustParseURL("http://10.1.1.14:8080")), 49 | ) 50 | 51 | // Start the server 52 | s.Start() 53 | } 54 | 55 | func ExampleServer_Command() { 56 | // A no-op ApplyFunc that always returns "PONG" 57 | ponger := func(uint64, []byte) []byte { return []byte(`PONG`) } 58 | 59 | // Assuming you have a server started 60 | s := raft.NewServer(1, &bytes.Buffer{}, ponger) 61 | 62 | // Issue a command into the network 63 | response := make(chan []byte) 64 | if err := s.Command([]byte(`PING`), response); err != nil { 65 | panic(err) // command not accepted 66 | } 67 | 68 | // After the command is replicated, we'll receive the response 69 | fmt.Printf("%s\n", <-response) 70 | } 71 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # raft 2 | 3 | This is an implementation of the [Raft distributed consensus protocol][paper]. 4 | It's heavily influenced by [benbjohnson's implementation][goraft]. 5 | It focuses on providing a clean and usable API, and well-structured internals. 6 | 7 | [![Build Status][buildimg]][buildurl] 8 | 9 | [paper]: https://ramcloud.stanford.edu/wiki/download/attachments/11370504/raft.pdf 10 | [goraft]: https://github.com/benbjohnson/go-raft 11 | [buildimg]: https://secure.travis-ci.org/peterbourgon/raft.png 12 | [buildurl]: http://www.travis-ci.org/peterbourgon/raft 13 | 14 | ## Usage 15 | 16 | A node in a Raft network is represented by a [Server][server] structure. In a 17 | typical application, nodes will create a Server, and expose it to other nodes 18 | using a [Peer][peer] interface. 19 | 20 | [server]: http://godoc.org/github.com/peterbourgon/raft#Server 21 | [peer]: http://godoc.org/github.com/peterbourgon/raft#Peer 22 | 23 | Servers are only useful when they can communicate with other servers. This 24 | library includes a [HTTP Transport][httpt] (ingress) and [HTTP Peer][httpp] 25 | (egress) which combine to allow communication via REST-ish endpoints. For now, 26 | it's the simplest way to embed a Raft server in your application. See [this 27 | complete example][example-http]. 28 | 29 | [httpt]: http://godoc.org/github.com/peterbourgon/raft#HTTPTransport 30 | [httpp]: http://godoc.org/github.com/peterbourgon/raft#NewHTTPPeer 31 | [example-http]: http://godoc.org/github.com/peterbourgon/raft#_example_NewServer--HTTP 32 | 33 | Several other transports are coming; see TODO, below. 34 | 35 | 36 | ## Adding and removing nodes 37 | 38 | The Raft protocol has no affordance for node discovery or "join/leave" 39 | semantics. Rather, the protocol assumes an ideal network configuration that's 40 | known _a priori_ to nodes in the network, and describes a mechanism (called 41 | joint-consensus) to safely replicate that configuration. 42 | 43 | My implementation of joint-consensus abides those fundamental assumptions. Nodes 44 | may be added or removed dynamically by requesting a **SetConfiguration** that 45 | describes a complete network topology. 46 | 47 | 48 | ## TODO 49 | 50 | * ~~Leader election~~ _done_ 51 | * ~~Log replication~~ _done_ 52 | * ~~Basic unit tests~~ _done_ 53 | * ~~HTTP transport~~ _done_ 54 | * [net/rpc][netrpc] transport 55 | * Other transports? 56 | * ~~Configuration changes (joint-consensus mode)~~ _done_ 57 | * Log compaction 58 | * Robust demo application ☜ **in progress** 59 | * Complex unit tests (one per scenario described in the paper) 60 | 61 | [netrpc]: http://golang.org/pkg/net/rpc 62 | 63 | -------------------------------------------------------------------------------- /transport_test.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "bytes" 5 | "log" 6 | "net/http" 7 | "net/http/httptest" 8 | "net/url" 9 | "os" 10 | "sync" 11 | "testing" 12 | "time" 13 | ) 14 | 15 | func Test3ServersOverHTTP(t *testing.T) { 16 | testNServersOverHTTP(t, 3) 17 | } 18 | 19 | func Test8ServersOverHTTP(t *testing.T) { 20 | testNServersOverHTTP(t, 8) 21 | } 22 | 23 | func testNServersOverHTTP(t *testing.T, n int) { 24 | if n <= 0 { 25 | t.Fatalf("n <= 0") 26 | } 27 | 28 | logBuffer := &bytes.Buffer{} 29 | log.SetOutput(logBuffer) 30 | defer log.SetOutput(os.Stdout) 31 | defer printOnFailure(t, logBuffer) 32 | oldMin, oldMax := resetElectionTimeoutMS(100, 200) 33 | defer resetElectionTimeoutMS(oldMin, oldMax) 34 | 35 | // node = Raft protocol server + a HTTP server + a transport bridge 36 | stateMachines := make([]*protectedSlice, n) 37 | raftServers := make([]*Server, n) 38 | httpServers := make([]*httptest.Server, n) 39 | 40 | // create them individually 41 | for i := 0; i < n; i++ { 42 | // create a state machine 43 | stateMachines[i] = &protectedSlice{} 44 | 45 | // create a Raft protocol server 46 | raftServers[i] = NewServer(uint64(i+1), &bytes.Buffer{}, appender(stateMachines[i])) 47 | 48 | // expose that server with a HTTP transport 49 | mux := http.NewServeMux() 50 | HTTPTransport(mux, raftServers[i]) 51 | 52 | // bind the HTTP transport to a concrete HTTP server 53 | httpServers[i] = httptest.NewServer(mux) 54 | // TODO this sometimes hangs, presuambly because we still have open 55 | // (active?) connections. Need to debug that. 56 | //defer httpServers[i].Close() 57 | 58 | // we have to start the HTTP server, so the NewHTTPPeer ID check works 59 | // (it can work without starting the actual Raft protocol server) 60 | t.Logf("Server id=%d @ %s", raftServers[i].id, httpServers[i].URL) 61 | } 62 | 63 | // build the common set of peers in the network 64 | peers := []Peer{} 65 | for i := 0; i < n; i++ { 66 | u, err := url.Parse(httpServers[i].URL) 67 | if err != nil { 68 | t.Fatal(err) 69 | } 70 | peer, err := NewHTTPPeer(u) 71 | if err != nil { 72 | t.Fatal(err) 73 | } 74 | peers = append(peers, peer) 75 | } 76 | 77 | // inject each Raft protocol server with its peers 78 | for _, raftServer := range raftServers { 79 | raftServer.SetConfiguration(peers...) 80 | } 81 | 82 | // start each Raft protocol server 83 | for _, raftServer := range raftServers { 84 | raftServer.Start() 85 | defer raftServer.Stop() 86 | } 87 | 88 | // wait for them to organize 89 | time.Sleep(time.Duration(n) * maximumElectionTimeout()) 90 | 91 | // send a command into the network 92 | cmd := []byte(`{"do_something":true}`) 93 | response := make(chan []byte, 1) 94 | if err := raftServers[0].Command(cmd, response); err != nil { 95 | t.Fatal(err) 96 | } 97 | select { 98 | case resp := <-response: 99 | t.Logf("got %d-byte command response ('%s')", len(resp), resp) 100 | case <-time.After(2 * maximumElectionTimeout()): 101 | t.Fatal("timeout waiting for command response") 102 | } 103 | 104 | // ensure it was replicated 105 | wg := sync.WaitGroup{} 106 | wg.Add(len(stateMachines)) 107 | for i := 0; i < len(stateMachines); i++ { 108 | go func(i int) { 109 | defer wg.Done() 110 | backoff := 5 * time.Millisecond 111 | for { 112 | slice := stateMachines[i].Get() 113 | if len(slice) != 1 { 114 | t.Logf("stateMachines[%d] not yet replicated", i) 115 | time.Sleep(backoff) 116 | backoff *= 2 117 | continue 118 | } 119 | if bytes.Compare(slice[0], cmd) != 0 { 120 | t.Fatalf("stateMachines[%d]: expected '%s' (%d-byte), got '%s' (%d-byte)", i, string(cmd), len(cmd), string(slice[0]), len(slice[0])) 121 | return 122 | } 123 | t.Logf("stateMachines[%d] replicated OK", i) 124 | return 125 | } 126 | }(i) 127 | } 128 | done := make(chan struct{}, 1) 129 | go func() { wg.Wait(); done <- struct{}{} }() 130 | select { 131 | case <-done: 132 | t.Logf("all state machines successfully replicated") 133 | case <-time.After(2 * maximumElectionTimeout()): 134 | t.Fatalf("timeout waiting for state machines to replicate") 135 | } 136 | } 137 | 138 | type protectedSlice struct { 139 | sync.RWMutex 140 | slice [][]byte 141 | } 142 | 143 | func (ps *protectedSlice) Get() [][]byte { 144 | ps.RLock() 145 | defer ps.RUnlock() 146 | return ps.slice 147 | } 148 | 149 | func (ps *protectedSlice) Add(buf []byte) { 150 | ps.Lock() 151 | defer ps.Unlock() 152 | ps.slice = append(ps.slice, buf) 153 | } 154 | 155 | func appender(ps *protectedSlice) ApplyFunc { 156 | return func(commitIndex uint64, cmd []byte) []byte { 157 | ps.Add(cmd) 158 | return []byte(`{"ok":true}`) 159 | } 160 | } 161 | -------------------------------------------------------------------------------- /configuration.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "bytes" 5 | "encoding/gob" 6 | "errors" 7 | "fmt" 8 | "sync" 9 | ) 10 | 11 | var ( 12 | errConfigurationAlreadyChanging = errors.New("configuration already changing") 13 | ) 14 | 15 | const ( 16 | cOld = "C_old" 17 | cOldNew = "C_old,new" 18 | ) 19 | 20 | // configuration represents the sets of peers and behaviors required to 21 | // implement joint-consensus. 22 | type configuration struct { 23 | sync.RWMutex 24 | state string 25 | cOldPeers peerMap 26 | cNewPeers peerMap 27 | } 28 | 29 | // newConfiguration returns a new configuration in stable (C_old) state based 30 | // on the passed peers. 31 | func newConfiguration(pm peerMap) *configuration { 32 | return &configuration{ 33 | state: cOld, // start in a stable state, 34 | cOldPeers: pm, // with only C_old 35 | } 36 | } 37 | 38 | // directSet is used when bootstrapping, and when receiving a replicated 39 | // configuration from a leader. It directly sets the configuration to the 40 | // passed peers. It's assumed this is called on a non-leader, and therefore 41 | // requires no consistency dance. 42 | func (c *configuration) directSet(pm peerMap) error { 43 | c.Lock() 44 | defer c.Unlock() 45 | 46 | c.cOldPeers = pm 47 | c.cNewPeers = peerMap{} 48 | c.state = cOld 49 | return nil 50 | } 51 | 52 | func (c *configuration) get(id uint64) (Peer, bool) { 53 | c.RLock() 54 | defer c.RUnlock() 55 | 56 | if peer, ok := c.cOldPeers[id]; ok { 57 | return peer, true 58 | } 59 | if peer, ok := c.cNewPeers[id]; ok { 60 | return peer, true 61 | } 62 | return nil, false 63 | } 64 | 65 | func (c *configuration) encode() ([]byte, error) { 66 | buf := &bytes.Buffer{} 67 | if err := gob.NewEncoder(buf).Encode(c.allPeers()); err != nil { 68 | return []byte{}, err 69 | } 70 | return buf.Bytes(), nil 71 | } 72 | 73 | // allPeers returns the union set of all peers in the configuration. 74 | func (c *configuration) allPeers() peerMap { 75 | c.RLock() 76 | defer c.RUnlock() 77 | 78 | union := peerMap{} 79 | for id, peer := range c.cOldPeers { 80 | union[id] = peer 81 | } 82 | for id, peer := range c.cNewPeers { 83 | union[id] = peer 84 | } 85 | return union 86 | } 87 | 88 | // pass returns true if the votes represented by the votes map are sufficient 89 | // to constitute a quorum. pass respects C_old,new requirements, which dictate 90 | // that any request must receive a majority from both C_old and C_new to pass. 91 | func (c *configuration) pass(votes map[uint64]bool) bool { 92 | c.RLock() 93 | defer c.RUnlock() 94 | 95 | // Count the votes 96 | cOldHave, cOldRequired := 0, c.cOldPeers.quorum() 97 | for id := range c.cOldPeers { 98 | if votes[id] { 99 | cOldHave++ 100 | } 101 | if cOldHave >= cOldRequired { 102 | break 103 | } 104 | } 105 | 106 | // If we've already failed, we can stop here 107 | if cOldHave < cOldRequired { 108 | return false 109 | } 110 | 111 | // C_old passes: if we're in C_old, we pass 112 | if c.state == cOld { 113 | return true 114 | } 115 | 116 | // Not in C_old, so make sure we have some peers in C_new 117 | if len(c.cNewPeers) <= 0 { 118 | panic(fmt.Sprintf("configuration state '%s', but no C_new peers", c.state)) 119 | } 120 | 121 | // Since we're in C_old,new, we need to also pass C_new to pass overall. 122 | // It's important that we range through C_new and check our votes map, and 123 | // not the other way around: if a server casts a vote but doesn't exist in 124 | // a particular configuration, that vote should not be counted. 125 | cNewHave, cNewRequired := 0, c.cNewPeers.quorum() 126 | for id := range c.cNewPeers { 127 | if votes[id] { 128 | cNewHave++ 129 | } 130 | if cNewHave >= cNewRequired { 131 | break 132 | } 133 | } 134 | 135 | return cNewHave >= cNewRequired 136 | } 137 | 138 | // changeTo signals a request to change to the configuration represented by the 139 | // passed peers. changeTo puts the configuration in the C_old,new state. 140 | // changeTo should be eventually followed by ChangeCommitted or ChangeAborted. 141 | func (c *configuration) changeTo(pm peerMap) error { 142 | c.Lock() 143 | defer c.Unlock() 144 | 145 | if c.state != cOld { 146 | return errConfigurationAlreadyChanging 147 | } 148 | 149 | if len(c.cNewPeers) > 0 { 150 | panic(fmt.Sprintf("configuration ChangeTo in state '%s', but have C_new peers already", c.state)) 151 | } 152 | 153 | c.cNewPeers = pm 154 | c.state = cOldNew 155 | return nil 156 | } 157 | 158 | // changeCommitted moves a configuration from C_old,new to C_new. 159 | func (c *configuration) changeCommitted() { 160 | c.Lock() 161 | defer c.Unlock() 162 | 163 | if c.state != cOldNew { 164 | panic("configuration ChangeCommitted, but not in C_old,new") 165 | } 166 | 167 | if len(c.cNewPeers) <= 0 { 168 | panic("configuration ChangeCommitted, but C_new peers are empty") 169 | } 170 | 171 | c.cOldPeers = c.cNewPeers 172 | c.cNewPeers = peerMap{} 173 | c.state = cOld 174 | } 175 | 176 | // changeAborted moves a configuration from C_old,new to C_old. 177 | func (c *configuration) changeAborted() { 178 | c.Lock() 179 | defer c.Unlock() 180 | 181 | if c.state != cOldNew { 182 | panic("configuration ChangeAborted, but not in C_old,new") 183 | } 184 | 185 | c.cNewPeers = peerMap{} 186 | c.state = cOld 187 | } 188 | -------------------------------------------------------------------------------- /peers.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "errors" 5 | "time" 6 | ) 7 | 8 | var ( 9 | errTimeout = errors.New("timeout") 10 | ) 11 | 12 | // Peer is the local representation of a remote node. It's an interface that may 13 | // be backed by any concrete transport: local, HTTP, net/rpc, etc. Peers must be 14 | // encoding/gob encodable. 15 | type Peer interface { 16 | id() uint64 17 | callAppendEntries(appendEntries) appendEntriesResponse 18 | callRequestVote(requestVote) requestVoteResponse 19 | callCommand([]byte, chan<- []byte) error 20 | callSetConfiguration(...Peer) error 21 | } 22 | 23 | // localPeer is the simplest kind of peer, mapped to a server in the 24 | // same process-space. Useful for testing and demonstration; not so 25 | // useful for networks of independent processes. 26 | type localPeer struct { 27 | server *Server 28 | } 29 | 30 | func newLocalPeer(server *Server) *localPeer { return &localPeer{server} } 31 | 32 | func (p *localPeer) id() uint64 { return p.server.id } 33 | 34 | func (p *localPeer) callAppendEntries(ae appendEntries) appendEntriesResponse { 35 | return p.server.appendEntries(ae) 36 | } 37 | 38 | func (p *localPeer) callRequestVote(rv requestVote) requestVoteResponse { 39 | return p.server.requestVote(rv) 40 | } 41 | 42 | func (p *localPeer) callCommand(cmd []byte, response chan<- []byte) error { 43 | return p.server.Command(cmd, response) 44 | } 45 | 46 | func (p *localPeer) callSetConfiguration(peers ...Peer) error { 47 | return p.server.SetConfiguration(peers...) 48 | } 49 | 50 | // requestVoteTimeout issues the requestVote to the given peer. 51 | // If no response is received before timeout, an error is returned. 52 | func requestVoteTimeout(p Peer, rv requestVote, timeout time.Duration) (requestVoteResponse, error) { 53 | c := make(chan requestVoteResponse, 1) 54 | go func() { c <- p.callRequestVote(rv) }() 55 | 56 | select { 57 | case resp := <-c: 58 | return resp, nil 59 | case <-time.After(timeout): 60 | return requestVoteResponse{}, errTimeout 61 | } 62 | } 63 | 64 | // peerMap is a collection of Peer interfaces. It provides some convenience 65 | // functions for actions that should apply to multiple Peers. 66 | type peerMap map[uint64]Peer 67 | 68 | // makePeerMap constructs a peerMap from a list of peers. 69 | func makePeerMap(peers ...Peer) peerMap { 70 | pm := peerMap{} 71 | for _, peer := range peers { 72 | pm[peer.id()] = peer 73 | } 74 | return pm 75 | } 76 | 77 | // explodePeerMap converts a peerMap into a slice of peers. 78 | func explodePeerMap(pm peerMap) []Peer { 79 | a := []Peer{} 80 | for _, peer := range pm { 81 | a = append(a, peer) 82 | } 83 | return a 84 | } 85 | 86 | func (pm peerMap) except(id uint64) peerMap { 87 | except := peerMap{} 88 | for id0, peer := range pm { 89 | if id0 == id { 90 | continue 91 | } 92 | except[id0] = peer 93 | } 94 | return except 95 | } 96 | 97 | func (pm peerMap) count() int { return len(pm) } 98 | 99 | func (pm peerMap) quorum() int { 100 | switch n := len(pm); n { 101 | case 0, 1: 102 | return 1 103 | default: 104 | return (n / 2) + 1 105 | } 106 | } 107 | 108 | // requestVotes sends the passed requestVote RPC to every peer in Peers. It 109 | // forwards responses along the returned requestVoteResponse channel. It makes 110 | // the RPCs with a timeout of BroadcastInterval * 2 (chosen arbitrarily). Peers 111 | // that don't respond within the timeout are retried forever. The retry loop 112 | // stops only when all peers have responded, or a Cancel signal is sent via the 113 | // returned canceler. 114 | func (pm peerMap) requestVotes(r requestVote) (chan voteResponseTuple, canceler) { 115 | // "[A server entering the candidate stage] issues requestVote RPCs in 116 | // parallel to each of the other servers in the cluster. If the candidate 117 | // receives no response for an RPC, it reissues the RPC repeatedly until a 118 | // response arrives or the election concludes." 119 | 120 | // construct the channels we'll return 121 | abortChan := make(chan struct{}) 122 | tupleChan := make(chan voteResponseTuple) 123 | 124 | go func() { 125 | // We loop until all Peers have given us a response. 126 | // Track which Peers have responded. 127 | respondedAlready := peerMap{} // none yet 128 | 129 | for { 130 | notYetResponded := disjoint(pm, respondedAlready) 131 | if len(notYetResponded) <= 0 { 132 | return // done 133 | } 134 | 135 | // scatter 136 | tupleChan0 := make(chan voteResponseTuple, len(notYetResponded)) 137 | for id, peer := range notYetResponded { 138 | go func(id uint64, peer Peer) { 139 | resp, err := requestVoteTimeout(peer, r, 2*maximumElectionTimeout()) 140 | tupleChan0 <- voteResponseTuple{id, resp, err} 141 | }(id, peer) 142 | } 143 | 144 | // gather 145 | for i := 0; i < cap(tupleChan0); i++ { 146 | select { 147 | case t := <-tupleChan0: 148 | if t.err != nil { 149 | continue // will need to retry 150 | } 151 | respondedAlready[t.id] = nil // set membership semantics 152 | tupleChan <- t 153 | 154 | case <-abortChan: 155 | return // give up 156 | } 157 | } 158 | } 159 | }() 160 | 161 | return tupleChan, cancel(abortChan) 162 | } 163 | 164 | type voteResponseTuple struct { 165 | id uint64 166 | response requestVoteResponse 167 | err error 168 | } 169 | 170 | type canceler interface { 171 | Cancel() 172 | } 173 | 174 | type cancel chan struct{} 175 | 176 | func (c cancel) Cancel() { close(c) } 177 | 178 | func disjoint(all, except peerMap) peerMap { 179 | d := peerMap{} 180 | for id, peer := range all { 181 | if _, ok := except[id]; ok { 182 | continue 183 | } 184 | d[id] = peer 185 | } 186 | return d 187 | } 188 | -------------------------------------------------------------------------------- /server_internals_test.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "bytes" 5 | "encoding/gob" 6 | "fmt" 7 | "testing" 8 | "time" 9 | ) 10 | 11 | func TestFollowerAllegiance(t *testing.T) { 12 | // a follower with allegiance to leader=2 13 | s := Server{ 14 | id: 1, 15 | term: 5, 16 | state: &protectedString{value: follower}, 17 | leader: 2, 18 | log: newRaftLog(&bytes.Buffer{}, noop), 19 | } 20 | 21 | // receives an appendEntries from a future term and different leader 22 | _, stepDown := s.handleAppendEntries(appendEntries{ 23 | Term: 6, 24 | LeaderID: 3, 25 | }) 26 | 27 | // should now step down and have a new term 28 | if !stepDown { 29 | t.Errorf("wasn't told to step down (i.e. abandon leader)") 30 | } 31 | if s.term != 6 { 32 | t.Errorf("no term change") 33 | } 34 | } 35 | 36 | func TestStrongLeader(t *testing.T) { 37 | // a leader in term=2 38 | s := Server{ 39 | id: 1, 40 | term: 2, 41 | state: &protectedString{value: leader}, 42 | leader: 1, 43 | log: newRaftLog(&bytes.Buffer{}, noop), 44 | } 45 | 46 | // receives a requestVote from someone also in term=2 47 | resp, stepDown := s.handleRequestVote(requestVote{ 48 | Term: 2, 49 | CandidateID: 3, 50 | LastLogIndex: 0, 51 | LastLogTerm: 0, 52 | }) 53 | 54 | // and should retain his leadership 55 | if resp.VoteGranted { 56 | t.Errorf("shouldn't have granted vote") 57 | } 58 | if stepDown { 59 | t.Errorf("shouldn't have stepped down") 60 | } 61 | } 62 | 63 | func TestLimitedClientPatience(t *testing.T) { 64 | // a client issues a command 65 | 66 | // it's written to a leader log 67 | 68 | // but the leader is deposed before he can replicate it 69 | 70 | // the new leader truncates the command away 71 | 72 | // the client should not be stuck forever 73 | } 74 | 75 | func TestLenientCommit(t *testing.T) { 76 | // a log that's fully committed 77 | log := &raftLog{ 78 | entries: []logEntry{ 79 | logEntry{Index: 1, Term: 1}, 80 | logEntry{Index: 2, Term: 1}, 81 | logEntry{Index: 3, Term: 2}, 82 | logEntry{Index: 4, Term: 2}, 83 | logEntry{Index: 5, Term: 2}, 84 | }, 85 | commitPos: 4, 86 | } 87 | 88 | // belongs to a follower 89 | s := Server{ 90 | id: 100, 91 | term: 2, 92 | leader: 101, 93 | log: log, 94 | state: &protectedString{value: follower}, 95 | } 96 | 97 | // an appendEntries comes with correct PrevLogIndex but older CommitIndex 98 | resp, stepDown := s.handleAppendEntries(appendEntries{ 99 | Term: 2, 100 | LeaderID: 101, 101 | PrevLogIndex: 5, 102 | PrevLogTerm: 2, 103 | CommitIndex: 4, // i.e. commitPos=3 104 | }) 105 | 106 | // this should not fail 107 | if !resp.Success { 108 | t.Errorf("failed (%s)", resp.reason) 109 | } 110 | if stepDown { 111 | t.Errorf("shouldn't step down") 112 | } 113 | } 114 | 115 | func TestConfigurationReceipt(t *testing.T) { 116 | // a follower 117 | s := Server{ 118 | id: 2, 119 | term: 1, 120 | leader: 1, 121 | log: &raftLog{ 122 | entries: []logEntry{logEntry{Index: 1, Term: 1}}, 123 | commitPos: 0, 124 | }, 125 | state: &protectedString{value: follower}, 126 | config: newConfiguration(peerMap{}), 127 | } 128 | 129 | // receives a configuration change 130 | pm := makePeerMap( 131 | serializablePeer{1, "foo"}, 132 | serializablePeer{2, "bar"}, 133 | serializablePeer{3, "baz"}, 134 | ) 135 | configurationBuf := &bytes.Buffer{} 136 | gob.Register(&serializablePeer{}) 137 | if err := gob.NewEncoder(configurationBuf).Encode(pm); err != nil { 138 | t.Fatal(err) 139 | } 140 | 141 | // via an appendEntries 142 | aer, _ := s.handleAppendEntries(appendEntries{ 143 | Term: 1, 144 | LeaderID: 1, 145 | PrevLogIndex: 1, 146 | PrevLogTerm: 1, 147 | Entries: []logEntry{ 148 | logEntry{ 149 | Index: 2, 150 | Term: 1, 151 | Command: configurationBuf.Bytes(), 152 | isConfiguration: true, 153 | }, 154 | }, 155 | CommitIndex: 1, 156 | }) 157 | 158 | // it should succeed 159 | if !aer.Success { 160 | t.Fatalf("appendEntriesResponse: no success: %s", aer.reason) 161 | } 162 | 163 | // and the follower's configuration should be immediately updated 164 | if expected, got := 3, s.config.allPeers().count(); expected != got { 165 | t.Fatalf("follower peer count: expected %d, got %d", expected, got) 166 | } 167 | peer, ok := s.config.get(3) 168 | if !ok { 169 | t.Fatal("follower didn't get peer 3") 170 | } 171 | if peer.id() != 3 { 172 | t.Fatal("follower got bad peer 3") 173 | } 174 | } 175 | 176 | func TestNonLeaderExpulsion(t *testing.T) { 177 | // a follower 178 | s := Server{ 179 | id: 2, 180 | term: 1, 181 | leader: 1, 182 | log: &raftLog{ 183 | store: &bytes.Buffer{}, 184 | entries: []logEntry{logEntry{Index: 1, Term: 1}}, 185 | commitPos: 0, 186 | }, 187 | state: &protectedString{value: follower}, 188 | config: newConfiguration(peerMap{}), 189 | quit: make(chan chan struct{}), 190 | } 191 | 192 | // receives a configuration change that doesn't include itself 193 | pm := makePeerMap( 194 | serializablePeer{1, "foo"}, 195 | serializablePeer{3, "baz"}, 196 | serializablePeer{5, "bat"}, 197 | ) 198 | configurationBuf := &bytes.Buffer{} 199 | gob.Register(&serializablePeer{}) 200 | if err := gob.NewEncoder(configurationBuf).Encode(pm); err != nil { 201 | t.Fatal(err) 202 | } 203 | 204 | // via an appendEntries 205 | s.handleAppendEntries(appendEntries{ 206 | Term: 1, 207 | LeaderID: 1, 208 | PrevLogIndex: 1, 209 | PrevLogTerm: 1, 210 | Entries: []logEntry{ 211 | logEntry{ 212 | Index: 2, 213 | Term: 1, 214 | Command: configurationBuf.Bytes(), 215 | isConfiguration: true, 216 | }, 217 | }, 218 | CommitIndex: 1, 219 | }) 220 | 221 | // and once committed 222 | s.handleAppendEntries(appendEntries{ 223 | Term: 1, 224 | LeaderID: 1, 225 | PrevLogIndex: 2, 226 | PrevLogTerm: 1, 227 | CommitIndex: 2, 228 | }) 229 | 230 | // the follower should shut down 231 | select { 232 | case q := <-s.quit: 233 | q <- struct{}{} 234 | case <-time.After(maximumElectionTimeout()): 235 | t.Fatal("didn't shut down") 236 | } 237 | } 238 | 239 | type serializablePeer struct { 240 | MyID uint64 241 | Err string 242 | } 243 | 244 | func (p serializablePeer) id() uint64 { return p.MyID } 245 | func (p serializablePeer) callAppendEntries(appendEntries) appendEntriesResponse { 246 | return appendEntriesResponse{} 247 | } 248 | func (p serializablePeer) callRequestVote(requestVote) requestVoteResponse { 249 | return requestVoteResponse{} 250 | } 251 | func (p serializablePeer) callCommand([]byte, chan<- []byte) error { 252 | return fmt.Errorf("%s", p.Err) 253 | } 254 | func (p serializablePeer) callSetConfiguration(...Peer) error { 255 | return fmt.Errorf("%s", p.Err) 256 | } 257 | -------------------------------------------------------------------------------- /JOINT-CONSENSUS.md: -------------------------------------------------------------------------------- 1 | ## Description 2 | 3 | This document describes the rationale behind the joint-consensus implementation, 4 | as a literal analaysis of section 6 (Cluster membership changes) of the 5 | [Raft paper](https://ramcloud.stanford.edu/wiki/download/attachments/11370504/raft.pdf). 6 | Each sentence in the section is indexed, and parsed for requirements, 7 | suggestions, and hints to the implementation. 8 | 9 | 10 | ## Analysis 11 | 12 | > **1** In Raft the cluster first switches to a transitional configuration we 13 | > call joint consensus; once the joint consensus has been committed, the system 14 | > then transitions to the new configuration. 15 | 16 | Every server should probably have a two-phase approach to configuration 17 | management: either in a stable (C_old) or unstable (C_old + C_old,new) state. 18 | 19 | 20 | > **2** The joint consensus combines both the old and new configurations: 21 | > **2a** Log entries are replicated to all servers in both configurations. 22 | 23 | Being in the unstable C_old,new state means sending requests to a union-set of 24 | C_old and C_new servers. 25 | 26 | 27 | > **2b** Any server from either configuration may serve as leader. 28 | 29 | _statement of fact_ 30 | 31 | 32 | > **2c** Agreement (for elections and entry commitment) requires majorities from 33 | > both the old and new configurations. 34 | 35 | Being in the unstable C_old,new state means changes must be independently 36 | validated by a majority of C_old servers AND C_new servers. 37 | 38 | ``` 39 | C_old C_new Result Pass? 40 | A B C A B C D A+ B+ C- D- C_old+ C_new- => FAIL 41 | A B C A B C D A+ B+ C- D+ C_old+ C_new+ => pass 42 | A B C A B C D A+ B+ C+ D- C_old+ C_new+ => pass 43 | ``` 44 | 45 | ``` 46 | func (c *Configuration) Pass(votes map[uint64]bool) bool { ... } 47 | ``` 48 | 49 | So far: in C_old,new, membership in C_old or C_new is notable only to the 50 | leader, for the purposes of vote-counting. 51 | 52 | 53 | > **3** As will be shown below, the joint consensus allows individual servers to 54 | > transition between configurations at different times without compromising 55 | > safety. 56 | 57 | _statement of promise_ 58 | 59 | 60 | > **4** Furthermore, joint consensus allows the cluster to continue servicing 61 | > client requests throughout the configuration change. 62 | 63 | Client requests (i.e. commands) should always be accepted, regardless of 64 | joint-consensus state. 65 | 66 | 67 | > **5** Cluster configurations are stored and communicated using special entries 68 | > in the replicated log; Figure 9 illustrates the configuration change process. 69 | 70 | Configurations need to be communicated through AppendEntries RPCs. 71 | Configurations need to be marshalable/unmarshalable. 72 | Implying: peers need to be marshalable/unmarshalable. 73 | 74 | 75 | > **6** When the leader receives a request to change the configuration from 76 | > C_old to C_new, it stores the configuration for joint consensus (C_old,new in 77 | > the figure) as a log entry and replicates that entry using the mechanisms 78 | > described previously. 79 | 80 | A request to change configuration is forwarded to and dispatched by the 81 | leader, just like a user command. Followers intercept the configuration-change 82 | command and use it to manipulate their configuration. 83 | 84 | handleAppendEntries should attempt to unmarshal the command as a Configuration 85 | change. Followers don't propagate the configuration-change command to the user 86 | state machine. 87 | 88 | 89 | > **7** Once a given server adds the new configuration entry to its log, it uses 90 | > that configuration for all future decisions (it does not wait for the entry to 91 | > become committed). 92 | 93 | A follower immediately applies a received configuration-change command. A 94 | follower can't leverage the apply() functionality as a mechanism to apply 95 | configuration changes. 96 | 97 | 98 | > **8** This means that the leader will use the rules of C_old,new to determine 99 | > when the log entry for C_old,new is committed. 100 | 101 | A leader immediately applies a received configuration-change command. 102 | Assuming it's not already in an unstable state (reject otherwise). 103 | 104 | 105 | > **9** If the leader crashes, a new leader may be chosen under either C_old or 106 | > C_old,new, depending on whether the winning candidate has received C_old,new. 107 | 108 | _statement of fact_ 109 | 110 | 111 | > **10** In any case, C_new cannot make unilateral decisions during this period. 112 | 113 | _statement of fact_ 114 | 115 | 116 | > **11** Once C_old,new has been committed, neither C_old nor C_new can make 117 | > decisions without approval of the other, and the Leader Log Property ensures 118 | > that only servers with the C_old,new log entry can be elected as leader. 119 | 120 | _statement of fact_ 121 | 122 | 123 | > **12** It is now safe for the leader to create a log entry describing C_new 124 | > and replicate it to the cluster. 125 | 126 | Once the configuration-change command from C_old to C_old,new has been 127 | committed, the leader transitions from C_old,new to C_new. The standard apply() 128 | mechanism could be leveraged for this behavior. 129 | 130 | Implementation: Configuration-change entry gets custom apply() that changes 131 | leader Configuration. 132 | 133 | 134 | > **13** Again, this configuration will take effect on each server as soon as it 135 | > is seen. 136 | 137 | _statement of fact_ 138 | 139 | 140 | > **14** When the new configuration has been committed under the rules of C_new, 141 | > the old configuration is irrelevant and servers not in the new configuration 142 | > can be shut down. 143 | 144 | If a server receives a configuration-change command that results in a 145 | configuration that doesn't include its server ID, when that command is 146 | committed, it should shut itself down. 147 | 148 | Implementation: if a configuration will result in the current server being 149 | expelled and shut down, a `committed` channel is injected into the relevant 150 | LogEntry, and a listener installed. If and when the LogEntry is committed, the 151 | listener goroutine signals the server to shutdown. 152 | 153 | 154 | > **15** As shown in Figure 9, there is no time when C_old and C_new can both 155 | > make unilateral decisions; this guarantees safety. 156 | 157 | _statement of fact_ 158 | 159 | 160 | > **16** There are two more issues to address for reconfiguration. 161 | 162 | _statement of promise_ 163 | 164 | 165 | > **17** First, if the leader is part of C_old but not part of C_new, it must 166 | > eventually step down. 167 | 168 | _prefix of 18_ 169 | 170 | 171 | > **18** In Raft the leader steps down immediately after committing a 172 | > configuration entry that does not include itself. 173 | 174 | A leader that finds it's not a part of a new configuration change should step 175 | down immediately after committing that change, but not before. The apply() 176 | mechanism could be leveraged for this behavior. See **12**. Stepping down can be 177 | synonymous with exiting its select loop -- the same behavior as followers. 178 | 179 | 180 | > **19** This means that there will be a period of time (while it is committing 181 | > C_new) where the leader is managing a cluster that does not include itself; it 182 | > replicates log entries but does not count itself in majorities. 183 | 184 | Leaders should take care to not automatically vote for themselves when 185 | counting majorities for a [C_new] state that doesn't include them. 186 | 187 | Implementation: when counting votes, range over the actual configuration 188 | members, and compare against the passed vote-tally -- not the other way around. 189 | 190 | > **20** The leader should not step down earlier, because members not in C_new 191 | > could still be elected, resulting in unnecessary elections. 192 | 193 | _statement of fact_ 194 | 195 | 196 | > **21** The second issue is that new servers may not initially store any log 197 | > entries. 198 | 199 | _statement of fact_ 200 | 201 | 202 | > **22** If they are added to the cluster in this state, it could take quite a 203 | > while for them to catch up, during which time it might not be possible to 204 | > commit new log entries. 205 | 206 | _statement of fact_ 207 | 208 | 209 | > **23** In order to avoid availability gaps, Raft introduces an additional 210 | > phase before the configuration change, in which the new servers join the 211 | > cluster as non-voting members (the leader will replicate log entries to them, 212 | > but they are not considered for majorities). 213 | 214 | I believe this behavior is underspecified, and cannot be reliably implemented 215 | without some kind of sentinel value that's a first-order property of a peer. 216 | Specifically, the case when the leader crashes during this catch-up phase is 217 | not clear to me. Therefore, **I don't implement this functionality** and won't 218 | until I can get it straight in my head. 219 | 220 | 221 | > **24** Once the new servers’ logs have caught up with the rest of the cluster, 222 | > the reconfiguration can proceed as described above. 223 | 224 | See above. 225 | -------------------------------------------------------------------------------- /transport.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "bytes" 5 | "encoding/gob" 6 | "encoding/json" 7 | "fmt" 8 | "io" 9 | "io/ioutil" 10 | "log" 11 | "net/http" 12 | "net/url" 13 | "strconv" 14 | ) 15 | 16 | var ( 17 | // IDPath is where the ID handler (GET) will be installed by the 18 | // HTTPTransport. 19 | IDPath = "/raft/id" 20 | 21 | // AppendEntriesPath is where the AppendEntries RPC handler (POST) will be 22 | // installed by the HTTPTransport. 23 | AppendEntriesPath = "/raft/appendentries" 24 | 25 | // RequestVotePath is where the requestVote RPC handler (POST) will be 26 | // installed by the HTTPTransport. 27 | RequestVotePath = "/raft/requestvote" 28 | 29 | // CommandPath is where the Command RPC handler (POST) will be installed by 30 | // the HTTPTransport. 31 | CommandPath = "/raft/command" 32 | 33 | // SetConfigurationPath is where the SetConfiguration RPC handler (POST) 34 | // will be installed by the HTTPTransport. 35 | SetConfigurationPath = "/raft/setconfiguration" 36 | ) 37 | 38 | var ( 39 | emptyAppendEntriesResponse bytes.Buffer 40 | emptyRequestVoteResponse bytes.Buffer 41 | ) 42 | 43 | func init() { 44 | json.NewEncoder(&emptyAppendEntriesResponse).Encode(appendEntriesResponse{}) 45 | json.NewEncoder(&emptyRequestVoteResponse).Encode(requestVoteResponse{}) 46 | gob.Register(&httpPeer{}) 47 | } 48 | 49 | // HTTPTransport creates an ingress bridge from the outside world to the passed 50 | // server, by installing handlers for all the necessary RPCs to the passed mux. 51 | func HTTPTransport(mux *http.ServeMux, s *Server) { 52 | mux.HandleFunc(IDPath, idHandler(s)) 53 | mux.HandleFunc(AppendEntriesPath, appendEntriesHandler(s)) 54 | mux.HandleFunc(RequestVotePath, requestVoteHandler(s)) 55 | mux.HandleFunc(CommandPath, commandHandler(s)) 56 | mux.HandleFunc(SetConfigurationPath, setConfigurationHandler(s)) 57 | } 58 | 59 | func idHandler(s *Server) http.HandlerFunc { 60 | return func(w http.ResponseWriter, r *http.Request) { 61 | w.Write([]byte(fmt.Sprint(s.id))) 62 | } 63 | } 64 | 65 | func appendEntriesHandler(s *Server) http.HandlerFunc { 66 | return func(w http.ResponseWriter, r *http.Request) { 67 | defer r.Body.Close() 68 | 69 | var ae appendEntries 70 | if err := json.NewDecoder(r.Body).Decode(&ae); err != nil { 71 | http.Error(w, emptyAppendEntriesResponse.String(), http.StatusBadRequest) 72 | return 73 | } 74 | 75 | aer := s.appendEntries(ae) 76 | if err := json.NewEncoder(w).Encode(aer); err != nil { 77 | http.Error(w, emptyAppendEntriesResponse.String(), http.StatusInternalServerError) 78 | return 79 | } 80 | } 81 | } 82 | 83 | func requestVoteHandler(s *Server) http.HandlerFunc { 84 | return func(w http.ResponseWriter, r *http.Request) { 85 | defer r.Body.Close() 86 | 87 | var rv requestVote 88 | if err := json.NewDecoder(r.Body).Decode(&rv); err != nil { 89 | http.Error(w, emptyRequestVoteResponse.String(), http.StatusBadRequest) 90 | return 91 | } 92 | 93 | rvr := s.requestVote(rv) 94 | if err := json.NewEncoder(w).Encode(rvr); err != nil { 95 | http.Error(w, emptyRequestVoteResponse.String(), http.StatusInternalServerError) 96 | return 97 | } 98 | } 99 | } 100 | 101 | func commandHandler(s *Server) http.HandlerFunc { 102 | return func(w http.ResponseWriter, r *http.Request) { 103 | defer r.Body.Close() 104 | 105 | // TODO we're collapsing a lot of errors into an empty response. 106 | // If we can decide on an error format, we could propegate them. 107 | 108 | cmd, err := ioutil.ReadAll(r.Body) 109 | if err != nil { 110 | http.Error(w, "", http.StatusBadRequest) 111 | return 112 | } 113 | 114 | response := make(chan []byte, 1) 115 | if err := s.Command(cmd, response); err != nil { 116 | http.Error(w, "", http.StatusInternalServerError) 117 | return 118 | } 119 | 120 | resp, ok := <-response 121 | if !ok { 122 | http.Error(w, "", http.StatusInternalServerError) 123 | return 124 | } 125 | 126 | w.Write(resp) 127 | } 128 | } 129 | 130 | func setConfigurationHandler(s *Server) http.HandlerFunc { 131 | return func(w http.ResponseWriter, r *http.Request) { 132 | defer r.Body.Close() 133 | 134 | var pm peerMap 135 | if err := gob.NewDecoder(r.Body).Decode(&pm); err != nil { 136 | errBuf, _ := json.Marshal(commaError{err.Error(), false}) 137 | http.Error(w, string(errBuf), http.StatusBadRequest) 138 | return 139 | } 140 | 141 | if err := s.SetConfiguration(explodePeerMap(pm)...); err != nil { 142 | errBuf, _ := json.Marshal(commaError{err.Error(), false}) 143 | http.Error(w, string(errBuf), http.StatusInternalServerError) 144 | return 145 | } 146 | 147 | respBuf, _ := json.Marshal(commaError{"", true}) 148 | w.Write(respBuf) 149 | } 150 | } 151 | 152 | // commaError is the structure returned by the configuration handler, to clients 153 | // that make set-configuration requests over the HTTP Transport. 154 | type commaError struct { 155 | Error string `json:"error,omitempty"` 156 | Success bool `json:"success,omitempty"` 157 | } 158 | 159 | // HTTPPeer represents a remote Raft server in the local process space. The 160 | // remote server is expected to be accessible through an HTTPTransport. 161 | type httpPeer struct { 162 | remoteID uint64 163 | url *url.URL 164 | } 165 | 166 | // NewHTTPPeer constructs a new HTTP peer. Part of construction involves making 167 | // a HTTP GET request against the passed URL at IDPath, to resolve the remote 168 | // server's ID. 169 | func NewHTTPPeer(url *url.URL) (Peer, error) { 170 | url.Path = "" 171 | 172 | idURL := *url 173 | idURL.Path = IDPath 174 | resp, err := http.Get(idURL.String()) 175 | if err != nil { 176 | return nil, err 177 | } 178 | defer resp.Body.Close() 179 | 180 | buf, err := ioutil.ReadAll(resp.Body) 181 | if err != nil { 182 | return nil, err 183 | } 184 | id, err := strconv.ParseUint(string(buf), 10, 64) 185 | if err != nil { 186 | return nil, err 187 | } 188 | if id <= 0 { 189 | return nil, fmt.Errorf("invalid peer ID %d", id) 190 | } 191 | 192 | return &httpPeer{ 193 | remoteID: id, 194 | url: url, 195 | }, nil 196 | } 197 | 198 | // ID returns the Raft-domain ID retrieved during construction of the httpPeer. 199 | func (p *httpPeer) id() uint64 { return p.remoteID } 200 | 201 | // AppendEntries triggers a AppendEntries RPC to the remote server, and 202 | // returns the response. Errors at the transport layers are logged, and 203 | // represented by a default (unsuccessful) response. 204 | func (p *httpPeer) callAppendEntries(ae appendEntries) appendEntriesResponse { 205 | var aer appendEntriesResponse 206 | 207 | var body bytes.Buffer 208 | if err := json.NewEncoder(&body).Encode(ae); err != nil { 209 | log.Printf("Raft: HTTP Peer: AppendEntries: encode request: %s", err) 210 | return aer 211 | } 212 | 213 | var resp bytes.Buffer 214 | if err := p.rpc(&body, AppendEntriesPath, &resp); err != nil { 215 | log.Printf("Raft: HTTP Peer: AppendEntries: during RPC: %s", err) 216 | return aer 217 | } 218 | 219 | if err := json.Unmarshal(resp.Bytes(), &aer); err != nil { 220 | log.Printf("Raft: HTTP Peer: AppendEntries: decode response: %s", err) 221 | return aer 222 | } 223 | 224 | return aer 225 | } 226 | 227 | // RequestVote triggers a requestVote RPC to the remote server, and 228 | // returns the response. Errors at the transport layers are logged, and 229 | // represented by a default (unsuccessful) response. 230 | func (p *httpPeer) callRequestVote(rv requestVote) requestVoteResponse { 231 | var rvr requestVoteResponse 232 | 233 | var body bytes.Buffer 234 | if err := json.NewEncoder(&body).Encode(rv); err != nil { 235 | log.Printf("Raft: HTTP Peer: RequestVote: encode request: %s", err) 236 | return rvr 237 | } 238 | 239 | var resp bytes.Buffer 240 | if err := p.rpc(&body, RequestVotePath, &resp); err != nil { 241 | log.Printf("Raft: HTTP Peer: RequestVote: during RPC: %s", err) 242 | return rvr 243 | } 244 | 245 | if err := json.Unmarshal(resp.Bytes(), &rvr); err != nil { 246 | log.Printf("Raft: HTTP Peer: RequestVote: decode response: %s", err) 247 | return rvr 248 | } 249 | 250 | return rvr 251 | } 252 | 253 | // Command forwards the passed cmd to the remote server. Any error at the 254 | // transport or application layer is returned synchronously. If no error 255 | // occurs, the response (the output of the remote server's ApplyFunc) is 256 | // eventually sent on the passed response chan. 257 | func (p *httpPeer) callCommand(cmd []byte, response chan<- []byte) error { 258 | errChan := make(chan error) 259 | go func() { 260 | var responseBuf bytes.Buffer 261 | err := p.rpc(bytes.NewBuffer(cmd), CommandPath, &responseBuf) 262 | errChan <- err 263 | if err != nil { 264 | return 265 | } 266 | response <- responseBuf.Bytes() 267 | }() 268 | return <-errChan // TODO timeout? 269 | } 270 | 271 | // SetConfiguration forwards the passed network configuration to the remote 272 | // server. Any error at the transport or application layer is returned 273 | // synchronously. If no error occurs, clients may assume the passed 274 | // configuration has been accepted and will be replicated via joint-consensus. 275 | func (p *httpPeer) callSetConfiguration(peers ...Peer) error { 276 | buf := &bytes.Buffer{} 277 | if err := gob.NewEncoder(buf).Encode(&peers); err != nil { 278 | log.Printf("Raft: HTTP Peer: SetConfiguration: encode request: %s", err) 279 | return err 280 | } 281 | 282 | var resp bytes.Buffer 283 | if err := p.rpc(buf, SetConfigurationPath, &resp); err != nil { 284 | log.Printf("Raft: HTTP Peer: SetConfiguration: during RPC: %s", err) 285 | return err 286 | } 287 | 288 | var commaErr commaError 289 | if err := json.Unmarshal(resp.Bytes(), &commaErr); err != nil { 290 | log.Printf("Raft: HTTP Peer: SetConfiguration: decode response: %s", err) 291 | return err 292 | } 293 | 294 | if !commaErr.Success { 295 | return fmt.Errorf(commaErr.Error) 296 | } 297 | return nil 298 | } 299 | 300 | func (p *httpPeer) rpc(request *bytes.Buffer, path string, response *bytes.Buffer) error { 301 | url := *p.url 302 | url.Path = path 303 | resp, err := http.Post(url.String(), "application/json", request) 304 | if err != nil { 305 | log.Printf("Raft: HTTP Peer: rpc POST: %s", err) 306 | return err 307 | } 308 | defer resp.Body.Close() 309 | 310 | if resp.StatusCode != http.StatusOK { 311 | return fmt.Errorf("HTTP %d", resp.StatusCode) 312 | } 313 | 314 | n, err := io.Copy(response, resp.Body) 315 | if err != nil { 316 | return err 317 | } 318 | if l := response.Len(); n < int64(l) { 319 | return fmt.Errorf("short read (%d < %d)", n, l) 320 | } 321 | 322 | return nil 323 | } 324 | -------------------------------------------------------------------------------- /log_test.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "bytes" 5 | "math" 6 | "strings" 7 | "testing" 8 | ) 9 | 10 | func oneshot() chan []byte { 11 | return make(chan []byte, 1) 12 | } 13 | 14 | func noop(uint64, []byte) []byte { 15 | return []byte{} 16 | } 17 | 18 | func TestLogEntriesAfter(t *testing.T) { 19 | c := []byte(`{}`) 20 | buf := &bytes.Buffer{} 21 | log := newRaftLog(buf, noop) 22 | 23 | type tuple struct { 24 | AfterIndex uint64 25 | ExpectedEntries int 26 | ExpectedTerm uint64 27 | } 28 | 29 | for _, tu := range []tuple{ 30 | {0, 0, 0}, 31 | {1, 0, 0}, 32 | {2, 0, 0}, 33 | {3, 0, 0}, 34 | {4, 0, 0}, 35 | } { 36 | entries, term := log.entriesAfter(tu.AfterIndex) 37 | if expected, got := tu.ExpectedEntries, len(entries); expected != got { 38 | t.Errorf("with %d, After(%d): entries: expected %d got %d", 0, tu.AfterIndex, expected, got) 39 | } 40 | if expected, got := tu.ExpectedTerm, term; expected != got { 41 | t.Errorf("with %d, After(%d): expected %d got %d", 0, tu.AfterIndex, expected, got) 42 | } 43 | } 44 | 45 | log.appendEntry(logEntry{1, 1, c, nil, oneshot(), false}) 46 | for _, tu := range []tuple{ 47 | {0, 1, 0}, 48 | {1, 0, 1}, 49 | {2, 0, 1}, 50 | {3, 0, 1}, 51 | {4, 0, 1}, 52 | } { 53 | entries, term := log.entriesAfter(tu.AfterIndex) 54 | if expected, got := tu.ExpectedEntries, len(entries); expected != got { 55 | t.Errorf("with %d, After(%d): entries: expected %d got %d", 1, tu.AfterIndex, expected, got) 56 | } 57 | if expected, got := tu.ExpectedTerm, term; expected != got { 58 | t.Errorf("with %d, After(%d): term: expected %d got %d", 1, tu.AfterIndex, expected, got) 59 | } 60 | } 61 | 62 | log.appendEntry(logEntry{2, 1, c, nil, oneshot(), false}) 63 | for _, tu := range []tuple{ 64 | {0, 2, 0}, 65 | {1, 1, 1}, 66 | {2, 0, 1}, 67 | {3, 0, 1}, 68 | {4, 0, 1}, 69 | } { 70 | entries, term := log.entriesAfter(tu.AfterIndex) 71 | if expected, got := tu.ExpectedEntries, len(entries); expected != got { 72 | t.Errorf("with %d, After(%d): entries: expected %d got %d", 2, tu.AfterIndex, expected, got) 73 | } 74 | if expected, got := tu.ExpectedTerm, term; expected != got { 75 | t.Errorf("with %d, After(%d): term: expected %d got %d", 2, tu.AfterIndex, expected, got) 76 | } 77 | } 78 | 79 | log.appendEntry(logEntry{3, 2, c, nil, oneshot(), false}) 80 | for _, tu := range []tuple{ 81 | {0, 3, 0}, 82 | {1, 2, 1}, 83 | {2, 1, 1}, 84 | {3, 0, 2}, 85 | {4, 0, 2}, 86 | } { 87 | entries, term := log.entriesAfter(tu.AfterIndex) 88 | if expected, got := tu.ExpectedEntries, len(entries); expected != got { 89 | t.Errorf("with %d, After(%d): entries: expected %d got %d", 3, tu.AfterIndex, expected, got) 90 | } 91 | if expected, got := tu.ExpectedTerm, term; expected != got { 92 | t.Errorf("with %d, After(%d): term: expected %d got %d", 3, tu.AfterIndex, expected, got) 93 | } 94 | } 95 | } 96 | 97 | func TestLogEncodeDecode(t *testing.T) { 98 | for _, e := range []logEntry{ 99 | logEntry{1, 1, []byte(`{}`), nil, oneshot(), false}, 100 | logEntry{1, 2, []byte(`{}`), nil, oneshot(), false}, 101 | logEntry{1, 2, []byte(`{}`), nil, oneshot(), false}, 102 | logEntry{2, 2, []byte(`{}`), nil, oneshot(), false}, 103 | logEntry{255, 3, []byte(`{"cmd": 123}`), nil, oneshot(), false}, 104 | logEntry{math.MaxUint64 - 1, math.MaxUint64, []byte(`{}`), nil, oneshot(), false}, 105 | } { 106 | b := &bytes.Buffer{} 107 | if err := e.encode(b); err != nil { 108 | t.Errorf("%v: Encode: %s", e, err) 109 | continue 110 | } 111 | t.Logf("%v: Encode: %s", e, strings.TrimSpace(b.String())) 112 | 113 | var e0 logEntry 114 | if err := e0.decode(b); err != nil { 115 | t.Errorf("%v: Decode: %s", e, err) 116 | } 117 | } 118 | } 119 | 120 | func TestLogAppend(t *testing.T) { 121 | c := []byte(`{}`) 122 | buf := &bytes.Buffer{} 123 | log := newRaftLog(buf, noop) 124 | 125 | // Append 3 valid LogEntries 126 | if err := log.appendEntry(logEntry{1, 1, c, nil, oneshot(), false}); err != nil { 127 | t.Errorf("Append: %s", err) 128 | } 129 | if err := log.appendEntry(logEntry{2, 1, c, nil, oneshot(), false}); err != nil { 130 | t.Errorf("Append: %s", err) 131 | } 132 | if err := log.appendEntry(logEntry{3, 2, c, nil, oneshot(), false}); err != nil { 133 | t.Errorf("Append: %s", err) 134 | } 135 | 136 | // Append some invalid LogEntries 137 | if err := log.appendEntry(logEntry{4, 1, c, nil, oneshot(), false}); err != errTermTooSmall { 138 | t.Errorf("Append: expected ErrTermTooSmall, got %v", err) 139 | } 140 | if err := log.appendEntry(logEntry{2, 2, c, nil, oneshot(), false}); err != errIndexTooSmall { 141 | t.Errorf("Append: expected ErrIndexTooSmall, got %v", nil) 142 | } 143 | 144 | // Check our flush buffer, before doing any commits 145 | precommit := `` 146 | if expected, got := precommit, buf.String(); expected != got { 147 | t.Errorf("before commit, expected:\n%s\ngot:\n%s\n", expected, got) 148 | } 149 | 150 | // Commit the first two, only 151 | if err := log.commitTo(2); err != nil { 152 | t.Fatalf("commitTo: %s", err) 153 | } 154 | 155 | // Check our flush buffer 156 | for i, expected := range log.entries[:2] { 157 | var got logEntry 158 | if err := got.decode(buf); err != nil { 159 | t.Fatalf("after commit, got: %s", err) 160 | } 161 | 162 | if expected.Term != got.Term { 163 | t.Errorf("%d. decode expected term=%d, got %d", i, expected.Term, got.Term) 164 | } 165 | 166 | if expected.Index != got.Index { 167 | t.Errorf("%d. decode expected index=%d, got %d", i, expected.Index, got.Index) 168 | } 169 | 170 | if !bytes.Equal(expected.Command, got.Command) { 171 | t.Errorf("%d. decode expected command=%q, got %q", i, expected.Command, got.Command) 172 | } 173 | } 174 | 175 | // Make some invalid commits 176 | if err := log.commitTo(1); err != errIndexTooSmall { 177 | t.Errorf("Commit: expected ErrIndexTooSmall, got %v", err) 178 | } 179 | if err := log.commitTo(4); err != errIndexTooBig { 180 | t.Errorf("Commit: expected ErrIndexTooBig, got %v", err) 181 | } 182 | 183 | // Commit every logEntry 184 | if err := log.commitTo(3); err != nil { 185 | t.Errorf("commitTo: %s", err) 186 | } 187 | 188 | // Check our buffer again 189 | expected := log.entries[2] 190 | var got logEntry 191 | if err := got.decode(buf); err != nil { 192 | t.Fatalf("after commit, got: %s", err) 193 | } 194 | 195 | if expected.Term != got.Term { 196 | t.Errorf("%d. decode expected term=%d, got %d", 3, expected.Term, got.Term) 197 | } 198 | 199 | if expected.Index != got.Index { 200 | t.Errorf("%d. decode expected index=%d, got %d", 3, expected.Index, got.Index) 201 | } 202 | 203 | if !bytes.Equal(expected.Command, got.Command) { 204 | t.Errorf("%d. decode expected command=%q, got %q", 3, expected.Command, got.Command) 205 | } 206 | } 207 | 208 | func TestLogContains(t *testing.T) { 209 | c := []byte(`{}`) 210 | buf := &bytes.Buffer{} 211 | log := newRaftLog(buf, noop) 212 | 213 | for _, tuple := range []struct { 214 | Index uint64 215 | Term uint64 216 | }{ 217 | {1, 1}, 218 | {2, 1}, 219 | {3, 2}, 220 | } { 221 | e := logEntry{tuple.Index, tuple.Term, c, nil, oneshot(), false} 222 | if err := log.appendEntry(e); err != nil { 223 | t.Fatalf("appendEntry(%v): %s", e, err) 224 | } 225 | } 226 | 227 | for _, tuple := range []struct { 228 | Index uint64 229 | Term uint64 230 | Expected bool 231 | }{ 232 | {0, 1, false}, 233 | {1, 0, false}, 234 | {1, 1, true}, 235 | {2, 1, true}, 236 | {1, 2, false}, 237 | {3, 2, true}, 238 | {3, 3, false}, 239 | {3, 4, false}, 240 | {4, 3, false}, 241 | {4, 4, false}, 242 | {4, 1, false}, 243 | {1, 4, false}, 244 | } { 245 | index, term, expected := tuple.Index, tuple.Term, tuple.Expected 246 | if got := log.contains(index, term); expected != got { 247 | t.Errorf("Contains(%d, %d): expected %v, got %v", index, term, expected, got) 248 | } 249 | } 250 | } 251 | 252 | func TestLogTruncation(t *testing.T) { 253 | c := []byte(`{}`) 254 | buf := &bytes.Buffer{} 255 | log := newRaftLog(buf, noop) 256 | 257 | for _, tuple := range []struct { 258 | Index uint64 259 | Term uint64 260 | }{ 261 | {1, 1}, 262 | {2, 1}, 263 | {3, 2}, 264 | } { 265 | e := logEntry{tuple.Index, tuple.Term, c, nil, oneshot(), false} 266 | if err := log.appendEntry(e); err != nil { 267 | t.Fatalf("appendEntry(%v): %s", e, err) 268 | } 269 | } 270 | 271 | if err := log.commitTo(2); err != nil { 272 | t.Fatal(err) 273 | } 274 | 275 | if expected, got := errIndexTooBig, log.ensureLastIs(4, 3); expected != got { 276 | t.Errorf("expected %s, got %v", expected, got) 277 | } 278 | if expected, got := errIndexTooSmall, log.ensureLastIs(1, 1); expected != got { 279 | t.Errorf("expected %s, got %v", expected, got) // before commitIndex 280 | } 281 | if expected, got := errBadTerm, log.ensureLastIs(3, 4); expected != got { 282 | t.Errorf("expected %s, got %v", expected, got) 283 | } 284 | 285 | if err := log.ensureLastIs(3, 2); err != nil { 286 | t.Fatal(err) 287 | } 288 | if err := log.ensureLastIs(2, 1); err != nil { 289 | t.Fatal(err) 290 | } 291 | if log.contains(3, 2) { 292 | t.Fatal("should have truncated (3,2) but it still exists") 293 | } 294 | if !log.contains(2, 1) { 295 | t.Fatal("(2,1) should still exist but it seems to be missing") 296 | } 297 | } 298 | 299 | func TestLogCommitNoDuplicate(t *testing.T) { 300 | // A pathological case: serial commitTo may double-apply the first command 301 | hits := 0 302 | apply := func(uint64, []byte) []byte { hits++; return []byte{} } 303 | log := newRaftLog(&bytes.Buffer{}, apply) 304 | 305 | log.appendEntry(logEntry{Index: 1, Term: 1, Command: []byte(`{}`)}) 306 | log.commitTo(1) 307 | if expected, got := 1, hits; expected != got { 308 | t.Errorf("expected %d hits, got %d", expected, got) 309 | } 310 | 311 | log.appendEntry(logEntry{Index: 2, Term: 1, Command: []byte(`{}`)}) 312 | log.commitTo(2) 313 | if expected, got := 2, hits; expected != got { 314 | t.Errorf("expected %d hits, got %d", expected, got) 315 | } 316 | 317 | log.appendEntry(logEntry{Index: 3, Term: 1, Command: []byte(`{}`)}) 318 | log.commitTo(3) 319 | if expected, got := 3, hits; expected != got { 320 | t.Errorf("expected %d hits, got %d", expected, got) 321 | } 322 | } 323 | 324 | func TestLogCommitTwice(t *testing.T) { 325 | // A pathological case: commitTo(N) twice in a row should be fine. 326 | log := newRaftLog(&bytes.Buffer{}, noop) 327 | 328 | log.appendEntry(logEntry{Index: 1, Term: 1, Command: []byte(`{}`)}) 329 | log.commitTo(1) 330 | 331 | log.appendEntry(logEntry{Index: 2, Term: 1, Command: []byte(`{}`)}) 332 | log.commitTo(2) 333 | log.commitTo(2) // shouldn't crash 334 | 335 | if expected, got := uint64(2), log.getCommitIndex(); expected != got { 336 | t.Errorf("expected commitIndex %d, got %d", expected, got) 337 | } 338 | } 339 | 340 | func TestCleanLogRecovery(t *testing.T) { 341 | entries := []logEntry{ 342 | {1, 1, []byte("{}"), nil, nil, false}, 343 | {2, 1, []byte("{}"), nil, nil, false}, 344 | {3, 2, []byte("{}"), nil, nil, false}, 345 | } 346 | 347 | buf := new(bytes.Buffer) 348 | for _, entry := range entries { 349 | entry.encode(buf) 350 | } 351 | log := newRaftLog(buf, noop) 352 | 353 | if expected, got := len(entries), len(log.entries); expected != got { 354 | t.Fatalf("expected %d, got %d", expected, got) 355 | } 356 | 357 | if !log.contains(1, 1) { 358 | t.Errorf("log doesn't contain index=1 term=1") 359 | } 360 | if !log.contains(2, 1) { 361 | t.Errorf("log doesn't contain index=2 term=1") 362 | } 363 | if !log.contains(3, 2) { 364 | t.Errorf("log doesn't contain index=3 term=2") 365 | } 366 | 367 | if expected, got := uint64(3), log.getCommitIndex(); expected != got { 368 | t.Errorf("expected commit index = %d, got %d", expected, got) 369 | } 370 | 371 | if expected, got := uint64(2), log.lastTerm(); expected != got { 372 | t.Errorf("expected term = %d, got %d", expected, got) 373 | } 374 | 375 | log.commitTo(3) // should be a no-op 376 | 377 | if buf.Len() > 0 { 378 | t.Errorf("commit to recovered index wrote to buffer") 379 | } 380 | 381 | if err := log.appendEntry(logEntry{ 382 | Index: 4, 383 | Term: 3, 384 | Command: []byte(`{"foo": "bar"}`), 385 | }); err != nil { 386 | t.Fatalf("append entry: %s", err) 387 | } 388 | if expected, got := 4, len(log.entries); expected != got { 389 | t.Fatalf("expected %d, got %d", expected, got) 390 | } 391 | if !log.contains(4, 3) { 392 | t.Errorf("log doesn't contain index=4 term=3") 393 | } 394 | } 395 | 396 | func TestCorruptedLogRecovery(t *testing.T) { 397 | entries := []logEntry{ 398 | {1, 1, []byte("{}"), nil, nil, false}, 399 | } 400 | 401 | buf := &bytes.Buffer{} 402 | for _, entry := range entries { 403 | entry.encode(buf) 404 | } 405 | buf.Write([]byte("garbage")) 406 | log := newRaftLog(buf, noop) 407 | 408 | if expected, got := len(entries), len(log.entries); expected != got { 409 | t.Fatalf("expected %d, got %d", expected, got) 410 | } 411 | 412 | if !log.contains(1, 1) { 413 | t.Errorf("log doesn't contain index=1 term=1") 414 | } 415 | if log.contains(2, 1) { 416 | t.Errorf("log contains corrupted index=2 term=1") 417 | } 418 | if log.contains(3, 2) { 419 | t.Errorf("log contains corrupted index=3 term=2") 420 | } 421 | 422 | if err := log.appendEntry(logEntry{ 423 | Index: 4, 424 | Term: 3, 425 | Command: []byte(`{"foo": "bar"}`), 426 | }); err != nil { 427 | t.Fatalf("append entry: %s", err) 428 | } 429 | if expected, got := 2, len(log.entries); expected != got { 430 | t.Fatalf("expected %d, got %d", expected, got) 431 | } 432 | if !log.contains(4, 3) { 433 | t.Errorf("log doesn't contain index=4 term=3") 434 | } 435 | } 436 | -------------------------------------------------------------------------------- /server_test.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "bufio" 5 | "bytes" 6 | "encoding/json" 7 | "fmt" 8 | "io" 9 | "log" 10 | "math/rand" 11 | "os" 12 | "sync" 13 | "sync/atomic" 14 | "testing" 15 | "time" 16 | ) 17 | 18 | func init() { 19 | rand.Seed(time.Now().UnixNano()) 20 | log.SetFlags(log.Lmicroseconds) 21 | } 22 | 23 | func TestFollowerToCandidate(t *testing.T) { 24 | log.SetOutput(&bytes.Buffer{}) 25 | defer log.SetOutput(os.Stdout) 26 | oldMin, oldMax := resetElectionTimeoutMS(25, 50) 27 | defer resetElectionTimeoutMS(oldMin, oldMax) 28 | 29 | server := NewServer(1, &bytes.Buffer{}, noop) 30 | server.SetConfiguration( 31 | newLocalPeer(server), 32 | nonresponsivePeer(2), 33 | nonresponsivePeer(3), 34 | ) 35 | 36 | server.Start() 37 | defer server.Stop() 38 | if server.state.Get() != follower { 39 | t.Fatalf("didn't start as Follower") 40 | } 41 | 42 | time.Sleep(maximumElectionTimeout()) 43 | 44 | cutoff := time.Now().Add(2 * minimumElectionTimeout()) 45 | backoff := minimumElectionTimeout() 46 | for { 47 | if time.Now().After(cutoff) { 48 | t.Fatal("failed to become Candidate") 49 | } 50 | if state := server.state.Get(); state != candidate { 51 | time.Sleep(backoff) 52 | backoff *= 2 53 | continue 54 | } 55 | t.Logf("became Candidate") 56 | break 57 | } 58 | } 59 | 60 | func TestCandidateToLeader(t *testing.T) { 61 | log.SetOutput(&bytes.Buffer{}) 62 | defer log.SetOutput(os.Stdout) 63 | oldMin, oldMax := resetElectionTimeoutMS(25, 50) 64 | defer resetElectionTimeoutMS(oldMin, oldMax) 65 | 66 | server := NewServer(1, &bytes.Buffer{}, noop) 67 | server.SetConfiguration( 68 | newLocalPeer(server), 69 | approvingPeer(2), 70 | nonresponsivePeer(3), 71 | ) 72 | 73 | server.Start() 74 | defer server.Stop() 75 | time.Sleep(maximumElectionTimeout()) 76 | 77 | cutoff := time.Now().Add(2 * maximumElectionTimeout()) 78 | backoff := maximumElectionTimeout() 79 | for { 80 | if time.Now().After(cutoff) { 81 | t.Fatal("failed to become Leader") 82 | } 83 | if state := server.state.Get(); state != leader { 84 | time.Sleep(backoff) 85 | backoff *= 2 86 | continue 87 | } 88 | t.Logf("became Leader") 89 | break 90 | } 91 | } 92 | 93 | func TestFailedElection(t *testing.T) { 94 | log.SetOutput(&bytes.Buffer{}) 95 | defer log.SetOutput(os.Stdout) 96 | oldMin, oldMax := resetElectionTimeoutMS(25, 50) 97 | defer resetElectionTimeoutMS(oldMin, oldMax) 98 | 99 | server := NewServer(1, &bytes.Buffer{}, noop) 100 | server.SetConfiguration( 101 | newLocalPeer(server), 102 | disapprovingPeer(2), 103 | nonresponsivePeer(3), 104 | ) 105 | 106 | server.Start() 107 | defer server.Stop() 108 | time.Sleep(2 * electionTimeout()) 109 | 110 | if server.state.Get() == leader { 111 | t.Fatalf("erroneously became Leader") 112 | } 113 | t.Logf("remained %s", server.state.Get()) 114 | } 115 | 116 | func TestLeaderExpulsion(t *testing.T) { 117 | // a leader 118 | // receives a configuration that doesn't include itself 119 | // when that configuration is committed 120 | // the leader should shut down 121 | } 122 | 123 | func TestSimpleConsensus(t *testing.T) { 124 | logBuffer := &bytes.Buffer{} 125 | log.SetOutput(logBuffer) 126 | defer log.SetOutput(os.Stdout) 127 | defer printOnFailure(t, logBuffer) 128 | oldMin, oldMax := resetElectionTimeoutMS(25, 50) 129 | defer resetElectionTimeoutMS(oldMin, oldMax) 130 | 131 | type SetValue struct { 132 | Value int32 `json:"value"` 133 | } 134 | 135 | var i1, i2, i3 int32 136 | 137 | applyValue := func(id uint64, i *int32) func(uint64, []byte) []byte { 138 | return func(index uint64, cmd []byte) []byte { 139 | var sv SetValue 140 | if err := json.Unmarshal(cmd, &sv); err != nil { 141 | var buf bytes.Buffer 142 | json.NewEncoder(&buf).Encode(map[string]interface{}{"error": err.Error()}) 143 | return buf.Bytes() 144 | } 145 | atomic.StoreInt32(i, sv.Value) 146 | var buf bytes.Buffer 147 | json.NewEncoder(&buf).Encode(map[string]interface{}{"applied_to_server": id, "applied_value": sv.Value}) 148 | return buf.Bytes() 149 | } 150 | } 151 | 152 | s1 := NewServer(1, &bytes.Buffer{}, applyValue(1, &i1)) 153 | s2 := NewServer(2, &bytes.Buffer{}, applyValue(2, &i2)) 154 | s3 := NewServer(3, &bytes.Buffer{}, applyValue(3, &i3)) 155 | 156 | s1Responses := &synchronizedBuffer{} 157 | s2Responses := &synchronizedBuffer{} 158 | s3Responses := &synchronizedBuffer{} 159 | defer func(sb *synchronizedBuffer) { t.Logf("s1 responses: %s", sb.String()) }(s1Responses) 160 | defer func(sb *synchronizedBuffer) { t.Logf("s2 responses: %s", sb.String()) }(s2Responses) 161 | defer func(sb *synchronizedBuffer) { t.Logf("s3 responses: %s", sb.String()) }(s3Responses) 162 | 163 | p1 := newLocalPeer(s1) 164 | p2 := newLocalPeer(s2) 165 | p3 := newLocalPeer(s3) 166 | s1.SetConfiguration(p1, p2, p3) 167 | s2.SetConfiguration(p1, p2, p3) 168 | s3.SetConfiguration(p1, p2, p3) 169 | 170 | s1.Start() 171 | s2.Start() 172 | s3.Start() 173 | defer s1.Stop() 174 | defer s2.Stop() 175 | defer s3.Stop() 176 | 177 | var v int32 = 42 178 | cmd, _ := json.Marshal(SetValue{v}) 179 | 180 | response := make(chan []byte, 1) 181 | func() { 182 | for { 183 | switch err := p1.callCommand(cmd, response); err { 184 | case nil: 185 | return 186 | case errUnknownLeader: 187 | time.Sleep(minimumElectionTimeout()) 188 | default: 189 | t.Fatal(err) 190 | } 191 | } 192 | }() 193 | 194 | r, ok := <-response 195 | if ok { 196 | s1Responses.Write(r) 197 | } else { 198 | t.Logf("didn't receive command response") 199 | } 200 | 201 | ticker := time.Tick(maximumElectionTimeout()) 202 | timeout := time.After(1 * time.Second) 203 | for { 204 | select { 205 | case <-ticker: 206 | i1l := atomic.LoadInt32(&i1) 207 | i2l := atomic.LoadInt32(&i2) 208 | i3l := atomic.LoadInt32(&i3) 209 | t.Logf("i1=%02d i2=%02d i3=%02d", i1l, i2l, i3l) 210 | if i1l == v && i2l == v && i3l == v { 211 | t.Logf("success!") 212 | return 213 | } 214 | 215 | case <-timeout: 216 | t.Fatal("timeout") 217 | } 218 | } 219 | } 220 | 221 | func TestOrdering_1Server(t *testing.T) { 222 | testOrderTimeout(t, 1, 5*time.Second) 223 | } 224 | 225 | func TestOrdering_2Servers(t *testing.T) { 226 | testOrderTimeout(t, 2, 5*time.Second) 227 | } 228 | 229 | func TestOrdering_3Servers(t *testing.T) { 230 | testOrderTimeout(t, 3, 5*time.Second) 231 | } 232 | 233 | func TestOrdering_4Servers(t *testing.T) { 234 | testOrderTimeout(t, 4, 5*time.Second) 235 | } 236 | 237 | func TestOrdering_5Servers(t *testing.T) { 238 | testOrderTimeout(t, 5, 5*time.Second) 239 | } 240 | 241 | func TestOrdering_6Servers(t *testing.T) { 242 | testOrderTimeout(t, 6, 5*time.Second) 243 | } 244 | 245 | func testOrderTimeout(t *testing.T, nServers int, timeout time.Duration) { 246 | logBuffer := &bytes.Buffer{} 247 | log.SetOutput(logBuffer) 248 | defer log.SetOutput(os.Stdout) 249 | defer printOnFailure(t, logBuffer) 250 | oldMin, oldMax := resetElectionTimeoutMS(50, 100) 251 | defer resetElectionTimeoutMS(oldMin, oldMax) 252 | 253 | done := make(chan struct{}) 254 | go func() { testOrder(t, nServers); close(done) }() 255 | select { 256 | case <-done: 257 | break 258 | case <-time.After(timeout): 259 | t.Fatalf("timeout (infinite loop?)") 260 | } 261 | } 262 | 263 | func testOrder(t *testing.T, nServers int) { 264 | values := rand.Perm(8 + rand.Intn(16)) 265 | 266 | // command and response 267 | type send struct { 268 | Send int `json:"s"` 269 | } 270 | type recv struct { 271 | Recv int `json:"r"` 272 | } 273 | do := func(sb *synchronizedBuffer) func(uint64, []byte) []byte { 274 | return func(index uint64, cmd []byte) []byte { 275 | sb.Write(cmd) // write incoming message 276 | var s send // decode incoming message 277 | json.Unmarshal(cmd, &s) 278 | var buf bytes.Buffer 279 | json.NewEncoder(&buf).Encode(recv{Recv: s.Send}) 280 | return buf.Bytes() // write outgoing message 281 | } 282 | } 283 | 284 | // set up the cluster 285 | servers := []*Server{} // server components 286 | storage := []*bytes.Buffer{} // persistent log storage 287 | buffers := []*synchronizedBuffer{} // the "state machine" for each server 288 | for i := 0; i < nServers; i++ { 289 | buffers = append(buffers, &synchronizedBuffer{}) 290 | storage = append(storage, &bytes.Buffer{}) 291 | servers = append(servers, NewServer(uint64(i+1), storage[i], do(buffers[i]))) 292 | } 293 | peers := []Peer{} 294 | for _, server := range servers { 295 | peers = append(peers, newLocalPeer(server)) 296 | } 297 | for _, server := range servers { 298 | server.SetConfiguration(peers...) 299 | } 300 | 301 | // define cmds 302 | cmds := []send{} 303 | for _, v := range values { 304 | cmds = append(cmds, send{v}) 305 | } 306 | 307 | // the expected "state-machine" output of applying each command 308 | expectedBuffer := &synchronizedBuffer{} 309 | for _, cmd := range cmds { 310 | buf, _ := json.Marshal(cmd) 311 | expectedBuffer.Write(buf) 312 | } 313 | 314 | // boot up the cluster 315 | for _, server := range servers { 316 | server.Start() 317 | defer func(server *Server) { 318 | log.Printf("issuing stop command to server %d", server.id) 319 | server.Stop() 320 | }(server) 321 | } 322 | 323 | // send commands 324 | for i, cmd := range cmds { 325 | index := uint64(rand.Intn(nServers)) 326 | peer := peers[index] 327 | id := peer.id() 328 | buf, _ := json.Marshal(cmd) 329 | 330 | for { 331 | log.Printf("command=%d/%d peer=%d: sending %s", i+1, len(cmds), id, buf) 332 | response := make(chan []byte, 1) 333 | err := peer.callCommand(buf, response) 334 | 335 | switch err { 336 | case nil: 337 | log.Printf("command=%d/%d peer=%d: OK", i+1, len(cmds), id) 338 | break 339 | 340 | case errUnknownLeader, errDeposed: 341 | log.Printf("command=%d/%d peer=%d: failed (%s) -- will retry", i+1, len(cmds), id, err) 342 | time.Sleep(electionTimeout()) 343 | continue 344 | 345 | case errTimeout: 346 | log.Printf("command=%d/%d peer=%d: timed out -- assume it went through", i+1, len(cmds), id) 347 | break 348 | 349 | default: 350 | t.Fatalf("command=%d/%d peer=%d: failed (%s) -- fatal", i+1, len(cmds), id, err) 351 | } 352 | 353 | r, ok := <-response 354 | if !ok { 355 | log.Printf("command=%d/%d peer=%d: truncated, will retry", i+1, len(cmds), id) 356 | continue 357 | } 358 | 359 | log.Printf("command=%d/%d peer=%d: OK, got response %s", i+1, len(cmds), id, string(r)) 360 | break 361 | } 362 | } 363 | 364 | // done sending 365 | log.Printf("testOrder done sending %d command(s) to network", len(cmds)) 366 | 367 | // check the buffers (state machines) 368 | for i, sb := range buffers { 369 | for { 370 | expected, got := expectedBuffer.String(), sb.String() 371 | if len(got) < len(expected) { 372 | t.Logf("server %d: not yet fully replicated, will check again", i+1) 373 | time.Sleep(maximumElectionTimeout()) 374 | continue // retry 375 | } 376 | if expected != got { 377 | t.Errorf("server %d: fully replicated, expected\n\t%s, got\n\t%s", i+1, expected, got) 378 | break 379 | } 380 | t.Logf("server %d: %s OK", i+1, got) 381 | break 382 | } 383 | } 384 | } 385 | 386 | // 387 | // 388 | // 389 | 390 | func printOnFailure(t *testing.T, r io.Reader) { 391 | if !t.Failed() { 392 | return 393 | } 394 | rd := bufio.NewReader(r) 395 | for { 396 | line, err := rd.ReadString('\n') 397 | if err != nil { 398 | return 399 | } 400 | t.Logf("> %s", line) 401 | } 402 | } 403 | 404 | type synchronizedBuffer struct { 405 | sync.RWMutex 406 | buf bytes.Buffer 407 | } 408 | 409 | func (b *synchronizedBuffer) Write(p []byte) (int, error) { 410 | b.Lock() 411 | defer b.Unlock() 412 | return b.buf.Write(p) 413 | } 414 | 415 | func (b *synchronizedBuffer) String() string { 416 | b.RLock() 417 | defer b.RUnlock() 418 | return b.buf.String() 419 | } 420 | 421 | type nonresponsivePeer uint64 422 | 423 | func (p nonresponsivePeer) id() uint64 { return uint64(p) } 424 | func (p nonresponsivePeer) callAppendEntries(appendEntries) appendEntriesResponse { 425 | return appendEntriesResponse{} 426 | } 427 | func (p nonresponsivePeer) callRequestVote(requestVote) requestVoteResponse { 428 | return requestVoteResponse{} 429 | } 430 | func (p nonresponsivePeer) callCommand([]byte, chan<- []byte) error { 431 | return fmt.Errorf("not implemented") 432 | } 433 | func (p nonresponsivePeer) callSetConfiguration(...Peer) error { 434 | return fmt.Errorf("not implemented") 435 | } 436 | 437 | type approvingPeer uint64 438 | 439 | func (p approvingPeer) id() uint64 { return uint64(p) } 440 | func (p approvingPeer) callAppendEntries(appendEntries) appendEntriesResponse { 441 | return appendEntriesResponse{} 442 | } 443 | func (p approvingPeer) callRequestVote(rv requestVote) requestVoteResponse { 444 | return requestVoteResponse{ 445 | Term: rv.Term, 446 | VoteGranted: true, 447 | } 448 | } 449 | func (p approvingPeer) callCommand([]byte, chan<- []byte) error { 450 | return fmt.Errorf("not implemented") 451 | } 452 | func (p approvingPeer) callSetConfiguration(...Peer) error { 453 | return fmt.Errorf("not implemented") 454 | } 455 | 456 | type disapprovingPeer uint64 457 | 458 | func (p disapprovingPeer) id() uint64 { return uint64(p) } 459 | func (p disapprovingPeer) callAppendEntries(appendEntries) appendEntriesResponse { 460 | return appendEntriesResponse{} 461 | } 462 | func (p disapprovingPeer) callRequestVote(rv requestVote) requestVoteResponse { 463 | return requestVoteResponse{ 464 | Term: rv.Term, 465 | VoteGranted: false, 466 | } 467 | } 468 | func (p disapprovingPeer) callCommand([]byte, chan<- []byte) error { 469 | return fmt.Errorf("not implemented") 470 | } 471 | func (p disapprovingPeer) callSetConfiguration(...Peer) error { 472 | return fmt.Errorf("not implemented") 473 | } 474 | -------------------------------------------------------------------------------- /log.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "encoding/binary" 5 | "errors" 6 | "fmt" 7 | "hash/crc32" 8 | "io" 9 | "sync" 10 | "time" 11 | ) 12 | 13 | var ( 14 | errTermTooSmall = errors.New("term too small") 15 | errIndexTooSmall = errors.New("index too small") 16 | errIndexTooBig = errors.New("commit index too big") 17 | errInvalidChecksum = errors.New("invalid checksum") 18 | errNoCommand = errors.New("no command") 19 | errBadIndex = errors.New("bad index") 20 | errBadTerm = errors.New("bad term") 21 | ) 22 | 23 | type raftLog struct { 24 | sync.RWMutex 25 | store io.Writer 26 | entries []logEntry 27 | commitPos int 28 | apply func(uint64, []byte) []byte 29 | } 30 | 31 | func newRaftLog(store io.ReadWriter, apply func(uint64, []byte) []byte) *raftLog { 32 | l := &raftLog{ 33 | store: store, 34 | entries: []logEntry{}, 35 | commitPos: -1, // no commits to begin with 36 | apply: apply, 37 | } 38 | l.recover(store) 39 | return l 40 | } 41 | 42 | // recover reads from the log's store, to populate the log with log entries 43 | // from persistent storage. It should be called once, at log instantiation. 44 | func (l *raftLog) recover(r io.Reader) error { 45 | for { 46 | var entry logEntry 47 | switch err := entry.decode(r); err { 48 | case io.EOF: 49 | return nil // successful completion 50 | case nil: 51 | if err := l.appendEntry(entry); err != nil { 52 | return err 53 | } 54 | l.commitPos++ 55 | l.apply(entry.Index, entry.Command) 56 | default: 57 | return err // unsuccessful completion 58 | } 59 | } 60 | } 61 | 62 | // entriesAfter returns a slice of log entries after (i.e. not including) the 63 | // passed index, and the term of the log entry specified by index, as a 64 | // convenience to the caller. (This function is only used by a leader attempting 65 | // to flush log entries to its followers.) 66 | // 67 | // This function is called to populate an AppendEntries RPC. That implies they 68 | // are destined for a follower, which implies the application of the commands 69 | // should have the response thrown away, which implies we shouldn't pass a 70 | // commandResponse channel (see: commitTo implementation). In the normal case, 71 | // the raftLogEntries we return here will get serialized as they pass thru their 72 | // transport, and lose their commandResponse channel anyway. But in the case of 73 | // a LocalPeer (or equivalent) this doesn't happen. So, we must make sure to 74 | // proactively strip commandResponse channels. 75 | func (l *raftLog) entriesAfter(index uint64) ([]logEntry, uint64) { 76 | l.RLock() 77 | defer l.RUnlock() 78 | 79 | pos := 0 80 | lastTerm := uint64(0) 81 | for ; pos < len(l.entries); pos++ { 82 | if l.entries[pos].Index > index { 83 | break 84 | } 85 | lastTerm = l.entries[pos].Term 86 | } 87 | 88 | a := l.entries[pos:] 89 | if len(a) == 0 { 90 | return []logEntry{}, lastTerm 91 | } 92 | 93 | return stripResponseChannels(a), lastTerm 94 | } 95 | 96 | func stripResponseChannels(a []logEntry) []logEntry { 97 | stripped := make([]logEntry, len(a)) 98 | for i, entry := range a { 99 | stripped[i] = logEntry{ 100 | Index: entry.Index, 101 | Term: entry.Term, 102 | Command: entry.Command, 103 | commandResponse: nil, 104 | } 105 | } 106 | return stripped 107 | } 108 | 109 | // contains returns true if a log entry with the given index and term exists in 110 | // the log. 111 | func (l *raftLog) contains(index, term uint64) bool { 112 | l.RLock() 113 | defer l.RUnlock() 114 | 115 | // It's not necessarily true that l.entries[i] has index == i. 116 | for _, entry := range l.entries { 117 | if entry.Index == index && entry.Term == term { 118 | return true 119 | } 120 | if entry.Index > index || entry.Term > term { 121 | break 122 | } 123 | } 124 | return false 125 | } 126 | 127 | // ensureLastIs deletes all non-committed log entries after the given index and 128 | // term. It will fail if the given index doesn't exist, has already been 129 | // committed, or doesn't match the given term. 130 | // 131 | // This method satisfies the requirement that a log entry in an AppendEntries 132 | // call precisely follows the accompanying LastraftLogTerm and LastraftLogIndex. 133 | func (l *raftLog) ensureLastIs(index, term uint64) error { 134 | l.Lock() 135 | defer l.Unlock() 136 | 137 | // Taken loosely from benbjohnson's impl 138 | 139 | if index < l.getCommitIndexWithLock() { 140 | return errIndexTooSmall 141 | } 142 | 143 | if index > l.lastIndexWithLock() { 144 | return errIndexTooBig 145 | } 146 | 147 | // It's possible that the passed index is 0. It means the leader has come to 148 | // decide we need a complete log rebuild. Of course, that's only valid if we 149 | // haven't committed anything, so this check comes after that one. 150 | if index == 0 { 151 | for pos := 0; pos < len(l.entries); pos++ { 152 | if l.entries[pos].commandResponse != nil { 153 | close(l.entries[pos].commandResponse) 154 | l.entries[pos].commandResponse = nil 155 | } 156 | if l.entries[pos].committed != nil { 157 | l.entries[pos].committed <- false 158 | close(l.entries[pos].committed) 159 | l.entries[pos].committed = nil 160 | } 161 | } 162 | l.entries = []logEntry{} 163 | return nil 164 | } 165 | 166 | // Normal case: find the position of the matching log entry. 167 | pos := 0 168 | for ; pos < len(l.entries); pos++ { 169 | if l.entries[pos].Index < index { 170 | continue // didn't find it yet 171 | } 172 | if l.entries[pos].Index > index { 173 | return errBadIndex // somehow went past it 174 | } 175 | if l.entries[pos].Index != index { 176 | panic("not <, not >, but somehow !=") 177 | } 178 | if l.entries[pos].Term != term { 179 | return errBadTerm 180 | } 181 | break // good 182 | } 183 | 184 | // Sanity check. 185 | if pos < l.commitPos { 186 | panic("index >= commitIndex, but pos < commitPos") 187 | } 188 | 189 | // `pos` is the position of log entry matching index and term. 190 | // We want to truncate everything after that. 191 | truncateFrom := pos + 1 192 | if truncateFrom >= len(l.entries) { 193 | return nil // nothing to truncate 194 | } 195 | 196 | // If we blow away log entries that haven't yet sent responses to clients, 197 | // signal the clients to stop waiting, by closing the channel without a 198 | // response value. 199 | for pos = truncateFrom; pos < len(l.entries); pos++ { 200 | if l.entries[pos].commandResponse != nil { 201 | close(l.entries[pos].commandResponse) 202 | l.entries[pos].commandResponse = nil 203 | } 204 | if l.entries[pos].committed != nil { 205 | l.entries[pos].committed <- false 206 | close(l.entries[pos].committed) 207 | l.entries[pos].committed = nil 208 | } 209 | } 210 | 211 | // Truncate the log. 212 | l.entries = l.entries[:truncateFrom] 213 | 214 | // Done. 215 | return nil 216 | } 217 | 218 | // getCommitIndex returns the commit index of the log. That is, the index of the 219 | // last log entry which can be considered committed. 220 | func (l *raftLog) getCommitIndex() uint64 { 221 | l.RLock() 222 | defer l.RUnlock() 223 | return l.getCommitIndexWithLock() 224 | } 225 | 226 | func (l *raftLog) getCommitIndexWithLock() uint64 { 227 | if l.commitPos < 0 { 228 | return 0 229 | } 230 | if l.commitPos >= len(l.entries) { 231 | panic(fmt.Sprintf("commitPos %d > len(l.entries) %d; bad bookkeeping in raftLog", l.commitPos, len(l.entries))) 232 | } 233 | return l.entries[l.commitPos].Index 234 | } 235 | 236 | // lastIndex returns the index of the most recent log entry. 237 | func (l *raftLog) lastIndex() uint64 { 238 | l.RLock() 239 | defer l.RUnlock() 240 | return l.lastIndexWithLock() 241 | } 242 | 243 | func (l *raftLog) lastIndexWithLock() uint64 { 244 | if len(l.entries) <= 0 { 245 | return 0 246 | } 247 | return l.entries[len(l.entries)-1].Index 248 | } 249 | 250 | // lastTerm returns the term of the most recent log entry. 251 | func (l *raftLog) lastTerm() uint64 { 252 | l.RLock() 253 | defer l.RUnlock() 254 | return l.lastTermWithLock() 255 | } 256 | 257 | func (l *raftLog) lastTermWithLock() uint64 { 258 | if len(l.entries) <= 0 { 259 | return 0 260 | } 261 | return l.entries[len(l.entries)-1].Term 262 | } 263 | 264 | // appendEntry appends the passed log entry to the log. It will return an error 265 | // if the entry's term is smaller than the log's most recent term, or if the 266 | // entry's index is too small relative to the log's most recent entry. 267 | func (l *raftLog) appendEntry(entry logEntry) error { 268 | l.Lock() 269 | defer l.Unlock() 270 | 271 | if len(l.entries) > 0 { 272 | lastTerm := l.lastTermWithLock() 273 | if entry.Term < lastTerm { 274 | return errTermTooSmall 275 | } 276 | lastIndex := l.lastIndexWithLock() 277 | if entry.Term == lastTerm && entry.Index <= lastIndex { 278 | return errIndexTooSmall 279 | } 280 | } 281 | 282 | l.entries = append(l.entries, entry) 283 | return nil 284 | } 285 | 286 | // commitTo commits all log entries up to and including the passed commitIndex. 287 | // Commit means: synchronize the log entry to persistent storage, and call the 288 | // state machine apply function for the log entry's command. 289 | func (l *raftLog) commitTo(commitIndex uint64) error { 290 | if commitIndex == 0 { 291 | panic("commitTo(0)") 292 | } 293 | 294 | l.Lock() 295 | defer l.Unlock() 296 | 297 | // Reject old commit indexes 298 | if commitIndex < l.getCommitIndexWithLock() { 299 | return errIndexTooSmall 300 | } 301 | 302 | // Reject new commit indexes 303 | if commitIndex > l.lastIndexWithLock() { 304 | return errIndexTooBig 305 | } 306 | 307 | // If we've already committed to the commitIndex, great! 308 | if commitIndex == l.getCommitIndexWithLock() { 309 | return nil 310 | } 311 | 312 | // We should start committing at precisely the last commitPos + 1 313 | pos := l.commitPos + 1 314 | if pos < 0 { 315 | panic("pending commit pos < 0") 316 | } 317 | 318 | // Commit entries between our existing commit index and the passed index. 319 | // Remember to include the passed index. 320 | for { 321 | // Sanity checks. TODO replace with plain `for` when this is stable. 322 | if pos >= len(l.entries) { 323 | panic(fmt.Sprintf("commitTo pos=%d advanced past all log entries (%d)", pos, len(l.entries))) 324 | } 325 | if l.entries[pos].Index > commitIndex { 326 | panic("commitTo advanced past the desired commitIndex") 327 | } 328 | 329 | // Encode the entry to persistent storage. 330 | if err := l.entries[pos].encode(l.store); err != nil { 331 | return err 332 | } 333 | 334 | // Forward non-configuration commands to the state machine. 335 | // Send the responses to the waiting client, if applicable. 336 | if !l.entries[pos].isConfiguration { 337 | resp := l.apply(l.entries[pos].Index, l.entries[pos].Command) 338 | if l.entries[pos].commandResponse != nil { 339 | select { 340 | case l.entries[pos].commandResponse <- resp: 341 | break 342 | case <-time.After(maximumElectionTimeout()): // << ElectionInterval 343 | panic("uncoöperative command response receiver") 344 | } 345 | close(l.entries[pos].commandResponse) 346 | l.entries[pos].commandResponse = nil 347 | } 348 | } 349 | 350 | // Signal the entry has been committed, if applicable. 351 | if l.entries[pos].committed != nil { 352 | l.entries[pos].committed <- true 353 | close(l.entries[pos].committed) 354 | l.entries[pos].committed = nil 355 | } 356 | 357 | // Mark our commit position cursor. 358 | l.commitPos = pos 359 | 360 | // If that was the last one, we're done. 361 | if l.entries[pos].Index == commitIndex { 362 | break 363 | } 364 | if l.entries[pos].Index > commitIndex { 365 | panic(fmt.Sprintf( 366 | "current entry Index %d is beyond our desired commitIndex %d", 367 | l.entries[pos].Index, 368 | commitIndex, 369 | )) 370 | } 371 | 372 | // Otherwise, advance! 373 | pos++ 374 | } 375 | 376 | // Done. 377 | return nil 378 | } 379 | 380 | // logEntry is the atomic unit being managed by the distributed log. A log entry 381 | // always has an index (monotonically increasing), a term in which the Raft 382 | // network leader first sees the entry, and a command. The command is what gets 383 | // executed against the node state machine when the log entry is successfully 384 | // replicated. 385 | type logEntry struct { 386 | Index uint64 `json:"index"` 387 | Term uint64 `json:"term"` // when received by leader 388 | Command []byte `json:"command,omitempty"` 389 | committed chan bool `json:"-"` 390 | commandResponse chan<- []byte `json:"-"` // only non-nil on receiver's log 391 | isConfiguration bool `json:"-"` // for configuration change entries 392 | } 393 | 394 | // encode serializes the log entry to the passed io.Writer. 395 | // 396 | // Entries are serialized in a simple binary format: 397 | // 398 | // --------------------------------------------- 399 | // | uint32 | uint64 | uint64 | uint32 | []byte | 400 | // --------------------------------------------- 401 | // | CRC | TERM | INDEX | SIZE | COMMAND | 402 | // --------------------------------------------- 403 | // 404 | func (e *logEntry) encode(w io.Writer) error { 405 | if len(e.Command) <= 0 { 406 | return errNoCommand 407 | } 408 | if e.Index <= 0 { 409 | return errBadIndex 410 | } 411 | if e.Term <= 0 { 412 | return errBadTerm 413 | } 414 | 415 | commandSize := len(e.Command) 416 | buf := make([]byte, 24+commandSize) 417 | 418 | binary.LittleEndian.PutUint64(buf[4:12], e.Term) 419 | binary.LittleEndian.PutUint64(buf[12:20], e.Index) 420 | binary.LittleEndian.PutUint32(buf[20:24], uint32(commandSize)) 421 | 422 | copy(buf[24:], e.Command) 423 | 424 | binary.LittleEndian.PutUint32( 425 | buf[0:4], 426 | crc32.ChecksumIEEE(buf[4:]), 427 | ) 428 | 429 | _, err := w.Write(buf) 430 | return err 431 | } 432 | 433 | // decode deserializes one log entry from the passed io.Reader. 434 | func (e *logEntry) decode(r io.Reader) error { 435 | header := make([]byte, 24) 436 | 437 | if _, err := r.Read(header); err != nil { 438 | return err 439 | } 440 | 441 | command := make([]byte, binary.LittleEndian.Uint32(header[20:24])) 442 | 443 | if _, err := r.Read(command); err != nil { 444 | return err 445 | } 446 | 447 | crc := binary.LittleEndian.Uint32(header[:4]) 448 | 449 | check := crc32.NewIEEE() 450 | check.Write(header[4:]) 451 | check.Write(command) 452 | 453 | if crc != check.Sum32() { 454 | return errInvalidChecksum 455 | } 456 | 457 | e.Term = binary.LittleEndian.Uint64(header[4:12]) 458 | e.Index = binary.LittleEndian.Uint64(header[12:20]) 459 | e.Command = command 460 | 461 | return nil 462 | } 463 | -------------------------------------------------------------------------------- /server.go: -------------------------------------------------------------------------------- 1 | // Package raft is an implementation of the Raft distributed consensus protocol. 2 | package raft 3 | 4 | import ( 5 | "bytes" 6 | "encoding/gob" 7 | "errors" 8 | "fmt" 9 | "io" 10 | "log" 11 | "math" 12 | "math/rand" 13 | "sync" 14 | "sync/atomic" 15 | "time" 16 | ) 17 | 18 | const ( 19 | follower = "Follower" 20 | candidate = "Candidate" 21 | leader = "Leader" 22 | ) 23 | 24 | const ( 25 | unknownLeader = 0 26 | noVote = 0 27 | ) 28 | 29 | var ( 30 | // MinimumElectionTimeoutMS can be set at package initialization. It may be 31 | // raised to achieve more reliable replication in slow networks, or lowered 32 | // to achieve faster replication in fast networks. Lowering is not 33 | // recommended. 34 | MinimumElectionTimeoutMS int32 = 250 35 | 36 | maximumElectionTimeoutMS = 2 * MinimumElectionTimeoutMS 37 | ) 38 | 39 | var ( 40 | errNotLeader = errors.New("not the leader") 41 | errUnknownLeader = errors.New("unknown leader") 42 | errDeposed = errors.New("deposed during replication") 43 | errAppendEntriesRejected = errors.New("appendEntries RPC rejected") 44 | errReplicationFailed = errors.New("command replication failed (but will keep retrying)") 45 | errOutOfSync = errors.New("out of sync") 46 | errAlreadyRunning = errors.New("already running") 47 | ) 48 | 49 | // resetElectionTimeoutMS sets the minimum and maximum election timeouts to the 50 | // passed values, and returns the old values. 51 | func resetElectionTimeoutMS(newMin, newMax int) (int, int) { 52 | oldMin := atomic.LoadInt32(&MinimumElectionTimeoutMS) 53 | oldMax := atomic.LoadInt32(&maximumElectionTimeoutMS) 54 | atomic.StoreInt32(&MinimumElectionTimeoutMS, int32(newMin)) 55 | atomic.StoreInt32(&maximumElectionTimeoutMS, int32(newMax)) 56 | return int(oldMin), int(oldMax) 57 | } 58 | 59 | // minimumElectionTimeout returns the current minimum election timeout. 60 | func minimumElectionTimeout() time.Duration { 61 | return time.Duration(MinimumElectionTimeoutMS) * time.Millisecond 62 | } 63 | 64 | // maximumElectionTimeout returns the current maximum election time. 65 | func maximumElectionTimeout() time.Duration { 66 | return time.Duration(maximumElectionTimeoutMS) * time.Millisecond 67 | } 68 | 69 | // electionTimeout returns a variable time.Duration, between the minimum and 70 | // maximum election timeouts. 71 | func electionTimeout() time.Duration { 72 | n := rand.Intn(int(maximumElectionTimeoutMS - MinimumElectionTimeoutMS)) 73 | d := int(MinimumElectionTimeoutMS) + n 74 | return time.Duration(d) * time.Millisecond 75 | } 76 | 77 | // broadcastInterval returns the interval between heartbeats (AppendEntry RPCs) 78 | // broadcast from the leader. It is the minimum election timeout / 10, as 79 | // dictated by the spec: BroadcastInterval << ElectionTimeout << MTBF. 80 | func broadcastInterval() time.Duration { 81 | d := MinimumElectionTimeoutMS / 10 82 | return time.Duration(d) * time.Millisecond 83 | } 84 | 85 | // protectedString is just a string protected by a mutex. 86 | type protectedString struct { 87 | sync.RWMutex 88 | value string 89 | } 90 | 91 | func (s *protectedString) Get() string { 92 | s.RLock() 93 | defer s.RUnlock() 94 | return s.value 95 | } 96 | 97 | func (s *protectedString) Set(value string) { 98 | s.Lock() 99 | defer s.Unlock() 100 | s.value = value 101 | } 102 | 103 | // protectedBool is just a bool protected by a mutex. 104 | type protectedBool struct { 105 | sync.RWMutex 106 | value bool 107 | } 108 | 109 | func (s *protectedBool) Get() bool { 110 | s.RLock() 111 | defer s.RUnlock() 112 | return s.value 113 | } 114 | 115 | func (s *protectedBool) Set(value bool) { 116 | s.Lock() 117 | defer s.Unlock() 118 | s.value = value 119 | } 120 | 121 | // Server is the agent that performs all of the Raft protocol logic. 122 | // In a typical application, each running process that wants to be part of 123 | // the distributed state machine will contain a server component. 124 | type Server struct { 125 | id uint64 // id of this server 126 | state *protectedString 127 | running *protectedBool 128 | leader uint64 // who we believe is the leader 129 | term uint64 // "current term number, which increases monotonically" 130 | vote uint64 // who we voted for this term, if applicable 131 | log *raftLog 132 | config *configuration 133 | 134 | appendEntriesChan chan appendEntriesTuple 135 | requestVoteChan chan requestVoteTuple 136 | commandChan chan commandTuple 137 | configurationChan chan configurationTuple 138 | 139 | electionTick <-chan time.Time 140 | quit chan chan struct{} 141 | } 142 | 143 | // ApplyFunc is a client-provided function that should apply a successfully 144 | // replicated state transition, represented by cmd, to the local state machine, 145 | // and return a response. commitIndex is the sequence number of the state 146 | // transition, which is guaranteed to be gapless and monotonically increasing, 147 | // but not necessarily duplicate-free. ApplyFuncs are not called concurrently. 148 | // Therefore, clients should ensure they return quickly, i.e. << 149 | // MinimumElectionTimeout. 150 | type ApplyFunc func(commitIndex uint64, cmd []byte) []byte 151 | 152 | // NewServer returns an initialized, un-started server. The ID must be unique in 153 | // the Raft network, and greater than 0. The store will be used by the 154 | // distributed log as a persistence layer. It's read-from during creation, in 155 | // case a crashed server is restarted over an already-persisted log. Then, it's 156 | // written-to during normal operations, when log entries are safely replicated. 157 | // ApplyFunc will be called whenever a (user-domain) command has been safely 158 | // replicated and committed to this server's log. 159 | // 160 | // NewServer creates a server, but you'll need to couple it with a transport to 161 | // make it usable. See the example(s) for usage scenarios. 162 | func NewServer(id uint64, store io.ReadWriter, a ApplyFunc) *Server { 163 | if id <= 0 { 164 | panic("server id must be > 0") 165 | } 166 | 167 | // 5.2 Leader election: "the latest term this server has seen is persisted, 168 | // and is initialized to 0 on first boot." 169 | log := newRaftLog(store, a) 170 | latestTerm := log.lastTerm() 171 | 172 | s := &Server{ 173 | id: id, 174 | state: &protectedString{value: follower}, // "when servers start up they begin as followers" 175 | running: &protectedBool{value: false}, 176 | leader: unknownLeader, // unknown at startup 177 | log: log, 178 | term: latestTerm, 179 | config: newConfiguration(peerMap{}), 180 | 181 | appendEntriesChan: make(chan appendEntriesTuple), 182 | requestVoteChan: make(chan requestVoteTuple), 183 | commandChan: make(chan commandTuple), 184 | configurationChan: make(chan configurationTuple), 185 | 186 | electionTick: nil, 187 | quit: make(chan chan struct{}), 188 | } 189 | s.resetElectionTimeout() 190 | return s 191 | } 192 | 193 | type configurationTuple struct { 194 | Peers []Peer 195 | Err chan error 196 | } 197 | 198 | // SetConfiguration sets the peers that this server will attempt to communicate 199 | // with. The set peers should include a peer that represents this server. 200 | // SetConfiguration must be called before starting the server. Calls to 201 | // SetConfiguration after the server has been started will be replicated 202 | // throughout the Raft network using the joint-consensus mechanism. 203 | // 204 | // TODO we need to refactor how we parse entries: a single code path from any 205 | // source (snapshot, persisted log at startup, or over the network) into the 206 | // log, and as part of that flow, checking if the entry is a configuration and 207 | // emitting it to the configuration structure. This implies an unfortunate 208 | // coupling: whatever processes log entries must have both the configuration 209 | // and the log as data sinks. 210 | func (s *Server) SetConfiguration(peers ...Peer) error { 211 | if !s.running.Get() { 212 | s.config.directSet(makePeerMap(peers...)) 213 | return nil 214 | } 215 | 216 | err := make(chan error) 217 | s.configurationChan <- configurationTuple{peers, err} 218 | return <-err 219 | } 220 | 221 | // Start triggers the server to begin communicating with its peers. 222 | func (s *Server) Start() { 223 | go s.loop() 224 | } 225 | 226 | // Stop terminates the server. Stopped servers should not be restarted. 227 | func (s *Server) Stop() { 228 | q := make(chan struct{}) 229 | s.quit <- q 230 | <-q 231 | s.logGeneric("server stopped") 232 | } 233 | 234 | type commandTuple struct { 235 | Command []byte 236 | CommandResponse chan<- []byte 237 | Err chan error 238 | } 239 | 240 | // Command appends the passed command to the leader log. If error is nil, the 241 | // command will eventually get replicated throughout the Raft network. When the 242 | // command gets committed to the local server log, it's passed to the apply 243 | // function, and the response from that function is provided on the 244 | // passed response chan. 245 | func (s *Server) Command(cmd []byte, response chan<- []byte) error { 246 | err := make(chan error) 247 | s.commandChan <- commandTuple{cmd, response, err} 248 | return <-err 249 | } 250 | 251 | // appendEntries processes the given RPC and returns the response. 252 | func (s *Server) appendEntries(ae appendEntries) appendEntriesResponse { 253 | t := appendEntriesTuple{ 254 | Request: ae, 255 | Response: make(chan appendEntriesResponse), 256 | } 257 | s.appendEntriesChan <- t 258 | return <-t.Response 259 | } 260 | 261 | // requestVote processes the given RPC and returns the response. 262 | func (s *Server) requestVote(rv requestVote) requestVoteResponse { 263 | t := requestVoteTuple{ 264 | Request: rv, 265 | Response: make(chan requestVoteResponse), 266 | } 267 | s.requestVoteChan <- t 268 | return <-t.Response 269 | } 270 | 271 | // times out, 272 | // new election 273 | // | .-----. 274 | // | | | 275 | // v times out, | v receives votes from 276 | // +----------+ starts election +-----------+ majority of servers +--------+ 277 | // | Follower |------------------>| Candidate |---------------------->| Leader | 278 | // +----------+ +-----------+ +--------+ 279 | // ^ ^ | | 280 | // | | discovers current leader | | 281 | // | | or new term | | 282 | // | '------------------------------' | 283 | // | | 284 | // | discovers server with higher term | 285 | // '------------------------------------------------------------------' 286 | // 287 | // 288 | 289 | func (s *Server) loop() { 290 | s.running.Set(true) 291 | for s.running.Get() { 292 | switch state := s.state.Get(); state { 293 | case follower: 294 | s.followerSelect() 295 | case candidate: 296 | s.candidateSelect() 297 | case leader: 298 | s.leaderSelect() 299 | default: 300 | panic(fmt.Sprintf("unknown Server State '%s'", state)) 301 | } 302 | } 303 | } 304 | 305 | func (s *Server) resetElectionTimeout() { 306 | s.electionTick = time.NewTimer(electionTimeout()).C 307 | } 308 | 309 | func (s *Server) logGeneric(format string, args ...interface{}) { 310 | prefix := fmt.Sprintf("id=%d term=%d state=%s: ", s.id, s.term, s.state.Get()) 311 | log.Printf(prefix+format, args...) 312 | } 313 | 314 | func (s *Server) logAppendEntriesResponse(req appendEntries, resp appendEntriesResponse, stepDown bool) { 315 | s.logGeneric( 316 | "got appendEntries, sz=%d leader=%d prevIndex/Term=%d/%d commitIndex=%d: responded with success=%v (reason='%s') stepDown=%v", 317 | len(req.Entries), 318 | req.LeaderID, 319 | req.PrevLogIndex, 320 | req.PrevLogTerm, 321 | req.CommitIndex, 322 | resp.Success, 323 | resp.reason, 324 | stepDown, 325 | ) 326 | } 327 | func (s *Server) logRequestVoteResponse(req requestVote, resp requestVoteResponse, stepDown bool) { 328 | s.logGeneric( 329 | "got RequestVote, candidate=%d: responded with granted=%v (reason='%s') stepDown=%v", 330 | req.CandidateID, 331 | resp.VoteGranted, 332 | resp.reason, 333 | stepDown, 334 | ) 335 | } 336 | 337 | func (s *Server) handleQuit(q chan struct{}) { 338 | s.logGeneric("got quit signal") 339 | s.running.Set(false) 340 | close(q) 341 | } 342 | 343 | func (s *Server) forwardCommand(t commandTuple) { 344 | switch s.leader { 345 | case unknownLeader: 346 | s.logGeneric("got command, but don't know leader") 347 | t.Err <- errUnknownLeader 348 | 349 | case s.id: // I am the leader 350 | panic("impossible state in forwardCommand") 351 | 352 | default: 353 | leader, ok := s.config.get(s.leader) 354 | if !ok { 355 | panic("invalid state in peers") 356 | } 357 | s.logGeneric("got command, forwarding to leader (%d)", s.leader) 358 | // We're blocking our {follower,candidate}Select function in the 359 | // receive-command branch. If we continue to block while forwarding 360 | // the command, the leader won't be able to get a response from us! 361 | go func() { t.Err <- leader.callCommand(t.Command, t.CommandResponse) }() 362 | } 363 | } 364 | 365 | func (s *Server) forwardConfiguration(t configurationTuple) { 366 | switch s.leader { 367 | case unknownLeader: 368 | s.logGeneric("got configuration, but don't know leader") 369 | t.Err <- errUnknownLeader 370 | 371 | case s.id: // I am the leader 372 | panic("impossible state in forwardConfiguration") 373 | 374 | default: 375 | leader, ok := s.config.get(s.leader) 376 | if !ok { 377 | panic("invalid state in peers") 378 | } 379 | s.logGeneric("got configuration, forwarding to leader (%d)", s.leader) 380 | go func() { t.Err <- leader.callSetConfiguration(t.Peers...) }() 381 | } 382 | } 383 | 384 | func (s *Server) followerSelect() { 385 | for { 386 | select { 387 | case q := <-s.quit: 388 | s.handleQuit(q) 389 | return 390 | 391 | case t := <-s.commandChan: 392 | s.forwardCommand(t) 393 | 394 | case t := <-s.configurationChan: 395 | s.forwardConfiguration(t) 396 | 397 | case <-s.electionTick: 398 | // 5.2 Leader election: "A follower increments its current term and 399 | // transitions to candidate state." 400 | if s.config == nil { 401 | s.logGeneric("election timeout, but no configuration: ignoring") 402 | s.resetElectionTimeout() 403 | continue 404 | } 405 | s.logGeneric("election timeout, becoming candidate") 406 | s.term++ 407 | s.vote = noVote 408 | s.leader = unknownLeader 409 | s.state.Set(candidate) 410 | s.resetElectionTimeout() 411 | return 412 | 413 | case t := <-s.appendEntriesChan: 414 | if s.leader == unknownLeader { 415 | s.leader = t.Request.LeaderID 416 | s.logGeneric("discovered Leader %d", s.leader) 417 | } 418 | resp, stepDown := s.handleAppendEntries(t.Request) 419 | s.logAppendEntriesResponse(t.Request, resp, stepDown) 420 | t.Response <- resp 421 | if stepDown { 422 | // stepDown as a Follower means just to reset the leader 423 | if s.leader != unknownLeader { 424 | s.logGeneric("abandoning old leader=%d", s.leader) 425 | } 426 | s.logGeneric("following new leader=%d", t.Request.LeaderID) 427 | s.leader = t.Request.LeaderID 428 | } 429 | 430 | case t := <-s.requestVoteChan: 431 | resp, stepDown := s.handleRequestVote(t.Request) 432 | s.logRequestVoteResponse(t.Request, resp, stepDown) 433 | t.Response <- resp 434 | if stepDown { 435 | // stepDown as a Follower means just to reset the leader 436 | if s.leader != unknownLeader { 437 | s.logGeneric("abandoning old leader=%d", s.leader) 438 | } 439 | s.logGeneric("new leader unknown") 440 | s.leader = unknownLeader 441 | } 442 | } 443 | } 444 | } 445 | 446 | func (s *Server) candidateSelect() { 447 | if s.leader != unknownLeader { 448 | panic("known leader when entering candidateSelect") 449 | } 450 | if s.vote != 0 { 451 | panic("existing vote when entering candidateSelect") 452 | } 453 | 454 | // "[A server entering the candidate stage] issues requestVote RPCs in 455 | // parallel to each of the other servers in the cluster. If the candidate 456 | // receives no response for an RPC, it reissues the RPC repeatedly until a 457 | // response arrives or the election concludes." 458 | 459 | requestVoteResponses, canceler := s.config.allPeers().except(s.id).requestVotes(requestVote{ 460 | Term: s.term, 461 | CandidateID: s.id, 462 | LastLogIndex: s.log.lastIndex(), 463 | LastLogTerm: s.log.lastTerm(), 464 | }) 465 | defer canceler.Cancel() 466 | 467 | // Set up vote tallies (plus, vote for myself) 468 | votes := map[uint64]bool{s.id: true} 469 | s.vote = s.id 470 | s.logGeneric("term=%d election started (configuration state %s)", s.term, s.config.state) 471 | 472 | // catch a weird state 473 | if s.config.pass(votes) { 474 | s.logGeneric("I immediately won the election") 475 | s.leader = s.id 476 | s.state.Set(leader) 477 | s.vote = noVote 478 | return 479 | } 480 | 481 | // "A candidate continues in this state until one of three things happens: 482 | // (a) it wins the election, (b) another server establishes itself as 483 | // leader, or (c) a period of time goes by with no winner." 484 | for { 485 | select { 486 | case q := <-s.quit: 487 | s.handleQuit(q) 488 | return 489 | 490 | case t := <-s.commandChan: 491 | s.forwardCommand(t) 492 | 493 | case t := <-s.configurationChan: 494 | s.forwardConfiguration(t) 495 | 496 | case t := <-requestVoteResponses: 497 | s.logGeneric("got vote: id=%d term=%d granted=%v", t.id, t.response.Term, t.response.VoteGranted) 498 | // "A candidate wins the election if it receives votes from a 499 | // majority of servers in the full cluster for the same term." 500 | if t.response.Term > s.term { 501 | s.logGeneric("got vote from future term (%d>%d); abandoning election", t.response.Term, s.term) 502 | s.leader = unknownLeader 503 | s.state.Set(follower) 504 | s.vote = noVote 505 | return // lose 506 | } 507 | if t.response.Term < s.term { 508 | s.logGeneric("got vote from past term (%d<%d); ignoring", t.response.Term, s.term) 509 | break 510 | } 511 | if t.response.VoteGranted { 512 | s.logGeneric("%d voted for me", t.id) 513 | votes[t.id] = true 514 | } 515 | // "Once a candidate wins an election, it becomes leader." 516 | if s.config.pass(votes) { 517 | s.logGeneric("I won the election") 518 | s.leader = s.id 519 | s.state.Set(leader) 520 | s.vote = noVote 521 | return // win 522 | } 523 | 524 | case t := <-s.appendEntriesChan: 525 | // "While waiting for votes, a candidate may receive an 526 | // appendEntries RPC from another server claiming to be leader. 527 | // If the leader's term (included in its RPC) is at least as 528 | // large as the candidate's current term, then the candidate 529 | // recognizes the leader as legitimate and steps down, meaning 530 | // that it returns to follower state." 531 | resp, stepDown := s.handleAppendEntries(t.Request) 532 | s.logAppendEntriesResponse(t.Request, resp, stepDown) 533 | t.Response <- resp 534 | if stepDown { 535 | s.logGeneric("after an appendEntries, stepping down to Follower (leader=%d)", t.Request.LeaderID) 536 | s.leader = t.Request.LeaderID 537 | s.state.Set(follower) 538 | return // lose 539 | } 540 | 541 | case t := <-s.requestVoteChan: 542 | // We can also be defeated by a more recent candidate 543 | resp, stepDown := s.handleRequestVote(t.Request) 544 | s.logRequestVoteResponse(t.Request, resp, stepDown) 545 | t.Response <- resp 546 | if stepDown { 547 | s.logGeneric("after a requestVote, stepping down to Follower (leader unknown)") 548 | s.leader = unknownLeader 549 | s.state.Set(follower) 550 | return // lose 551 | } 552 | 553 | case <-s.electionTick: 554 | // "The third possible outcome is that a candidate neither wins nor 555 | // loses the election: if many followers become candidates at the 556 | // same time, votes could be split so that no candidate obtains a 557 | // majority. When this happens, each candidate will start a new 558 | // election by incrementing its term and initiating another round of 559 | // requestVote RPCs." 560 | s.logGeneric("election ended with no winner; incrementing term and trying again") 561 | s.resetElectionTimeout() 562 | s.term++ 563 | s.vote = noVote 564 | return // draw 565 | } 566 | } 567 | } 568 | 569 | // 570 | // 571 | // 572 | 573 | type nextIndex struct { 574 | sync.RWMutex 575 | m map[uint64]uint64 // followerId: nextIndex 576 | } 577 | 578 | func newNextIndex(pm peerMap, defaultNextIndex uint64) *nextIndex { 579 | ni := &nextIndex{ 580 | m: map[uint64]uint64{}, 581 | } 582 | for id := range pm { 583 | ni.m[id] = defaultNextIndex 584 | } 585 | return ni 586 | } 587 | 588 | func (ni *nextIndex) bestIndex() uint64 { 589 | ni.RLock() 590 | defer ni.RUnlock() 591 | 592 | if len(ni.m) <= 0 { 593 | return 0 594 | } 595 | 596 | i := uint64(math.MaxUint64) 597 | for _, nextIndex := range ni.m { 598 | if nextIndex < i { 599 | i = nextIndex 600 | } 601 | } 602 | return i 603 | } 604 | 605 | func (ni *nextIndex) prevLogIndex(id uint64) uint64 { 606 | ni.RLock() 607 | defer ni.RUnlock() 608 | if _, ok := ni.m[id]; !ok { 609 | panic(fmt.Sprintf("peer %d not found", id)) 610 | } 611 | return ni.m[id] 612 | } 613 | 614 | func (ni *nextIndex) decrement(id uint64, prev uint64) (uint64, error) { 615 | ni.Lock() 616 | defer ni.Unlock() 617 | 618 | i, ok := ni.m[id] 619 | if !ok { 620 | panic(fmt.Sprintf("peer %d not found", id)) 621 | } 622 | 623 | if i != prev { 624 | return i, errOutOfSync 625 | } 626 | 627 | if i > 0 { 628 | ni.m[id]-- 629 | } 630 | return ni.m[id], nil 631 | } 632 | 633 | func (ni *nextIndex) set(id, index, prev uint64) (uint64, error) { 634 | ni.Lock() 635 | defer ni.Unlock() 636 | 637 | i, ok := ni.m[id] 638 | if !ok { 639 | panic(fmt.Sprintf("peer %d not found", id)) 640 | } 641 | if i != prev { 642 | return i, errOutOfSync 643 | } 644 | 645 | ni.m[id] = index 646 | return index, nil 647 | } 648 | 649 | // flush generates and forwards an appendEntries request that attempts to bring 650 | // the given follower "in sync" with our log. It's idempotent, so it's used for 651 | // both heartbeats and replicating commands. 652 | // 653 | // The appendEntries request we build represents our best attempt at a "delta" 654 | // between our log and the follower's log. The passed nextIndex structure 655 | // manages that state. 656 | // 657 | // flush is synchronous and can block forever if the peer is nonresponsive. 658 | func (s *Server) flush(peer Peer, ni *nextIndex) error { 659 | peerID := peer.id() 660 | currentTerm := s.term 661 | prevLogIndex := ni.prevLogIndex(peerID) 662 | entries, prevLogTerm := s.log.entriesAfter(prevLogIndex) 663 | commitIndex := s.log.getCommitIndex() 664 | s.logGeneric("flush to %d: term=%d leaderId=%d prevLogIndex/Term=%d/%d sz=%d commitIndex=%d", peerID, currentTerm, s.id, prevLogIndex, prevLogTerm, len(entries), commitIndex) 665 | resp := peer.callAppendEntries(appendEntries{ 666 | Term: currentTerm, 667 | LeaderID: s.id, 668 | PrevLogIndex: prevLogIndex, 669 | PrevLogTerm: prevLogTerm, 670 | Entries: entries, 671 | CommitIndex: commitIndex, 672 | }) 673 | 674 | if resp.Term > currentTerm { 675 | s.logGeneric("flush to %d: responseTerm=%d > currentTerm=%d: deposed", peerID, resp.Term, currentTerm) 676 | return errDeposed 677 | } 678 | 679 | // It's possible the leader has timed out waiting for us, and moved on. 680 | // So we should be careful, here, to make only valid state changes to `ni`. 681 | 682 | if !resp.Success { 683 | newPrevLogIndex, err := ni.decrement(peerID, prevLogIndex) 684 | if err != nil { 685 | s.logGeneric("flush to %d: while decrementing prevLogIndex: %s", peerID, err) 686 | return err 687 | } 688 | s.logGeneric("flush to %d: rejected; prevLogIndex(%d) becomes %d", peerID, peerID, newPrevLogIndex) 689 | return errAppendEntriesRejected 690 | } 691 | 692 | if len(entries) > 0 { 693 | newPrevLogIndex, err := ni.set(peer.id(), entries[len(entries)-1].Index, prevLogIndex) 694 | if err != nil { 695 | s.logGeneric("flush to %d: while moving prevLogIndex forward: %s", peerID, err) 696 | return err 697 | } 698 | s.logGeneric("flush to %d: accepted; prevLogIndex(%d) becomes %d", peerID, peerID, newPrevLogIndex) 699 | return nil 700 | } 701 | 702 | s.logGeneric("flush to %d: accepted; prevLogIndex(%d) remains %d", peerID, peerID, ni.prevLogIndex(peerID)) 703 | return nil 704 | } 705 | 706 | // concurrentFlush triggers a concurrent flush to each of the peers. All peers 707 | // must respond (or timeout) before concurrentFlush will return. timeout is per 708 | // peer. 709 | func (s *Server) concurrentFlush(pm peerMap, ni *nextIndex, timeout time.Duration) (int, bool) { 710 | type tuple struct { 711 | id uint64 712 | err error 713 | } 714 | responses := make(chan tuple, len(pm)) 715 | for _, peer := range pm { 716 | go func(peer Peer) { 717 | errChan := make(chan error, 1) 718 | go func() { errChan <- s.flush(peer, ni) }() 719 | go func() { time.Sleep(timeout); errChan <- errTimeout }() 720 | responses <- tuple{peer.id(), <-errChan} // first responder wins 721 | }(peer) 722 | } 723 | 724 | successes, stepDown := 0, false 725 | for i := 0; i < cap(responses); i++ { 726 | switch t := <-responses; t.err { 727 | case nil: 728 | s.logGeneric("concurrentFlush: peer %d: OK (prevLogIndex(%d)=%d)", t.id, t.id, ni.prevLogIndex(t.id)) 729 | successes++ 730 | case errDeposed: 731 | s.logGeneric("concurrentFlush: peer %d: deposed!", t.id) 732 | stepDown = true 733 | default: 734 | s.logGeneric("concurrentFlush: peer %d: %s (prevLogIndex(%d)=%d)", t.id, t.err, t.id, ni.prevLogIndex(t.id)) 735 | // nothing to do but log and continue 736 | } 737 | } 738 | return successes, stepDown 739 | } 740 | 741 | func (s *Server) leaderSelect() { 742 | if s.leader != s.id { 743 | panic(fmt.Sprintf("leader (%d) not me (%d) when entering leaderSelect", s.leader, s.id)) 744 | } 745 | if s.vote != 0 { 746 | panic(fmt.Sprintf("vote (%d) not zero when entering leaderSelect", s.leader)) 747 | } 748 | 749 | // 5.3 Log replication: "The leader maintains a nextIndex for each follower, 750 | // which is the index of the next log entry the leader will send to that 751 | // follower. When a leader first comes to power it initializes all nextIndex 752 | // values to the index just after the last one in its log." 753 | // 754 | // I changed this from lastIndex+1 to simply lastIndex. Every initial 755 | // communication from leader to follower was being rejected and we were 756 | // doing the decrement. This was just annoying, except if you manage to 757 | // sneak in a command before the first heartbeat. Then, it will never get 758 | // properly replicated (it seemed). 759 | ni := newNextIndex(s.config.allPeers().except(s.id), s.log.lastIndex()) // +1) 760 | 761 | flush := make(chan struct{}) 762 | heartbeat := time.NewTicker(broadcastInterval()) 763 | defer heartbeat.Stop() 764 | go func() { 765 | for _ = range heartbeat.C { 766 | flush <- struct{}{} 767 | } 768 | }() 769 | 770 | for { 771 | select { 772 | case q := <-s.quit: 773 | s.handleQuit(q) 774 | return 775 | 776 | case t := <-s.commandChan: 777 | // Append the command to our (leader) log 778 | s.logGeneric("got command, appending") 779 | currentTerm := s.term 780 | entry := logEntry{ 781 | Index: s.log.lastIndex() + 1, 782 | Term: currentTerm, 783 | Command: t.Command, 784 | commandResponse: t.CommandResponse, 785 | } 786 | if err := s.log.appendEntry(entry); err != nil { 787 | t.Err <- err 788 | continue 789 | } 790 | s.logGeneric( 791 | "after append, commitIndex=%d lastIndex=%d lastTerm=%d", 792 | s.log.getCommitIndex(), 793 | s.log.lastIndex(), 794 | s.log.lastTerm(), 795 | ) 796 | 797 | // Now that the entry is in the log, we can fall back to the 798 | // normal flushing mechanism to attempt to replicate the entry 799 | // and advance the commit index. We trigger a manual flush as a 800 | // convenience, so our caller might get a response a bit sooner. 801 | go func() { flush <- struct{}{} }() 802 | t.Err <- nil 803 | 804 | case t := <-s.configurationChan: 805 | // Attempt to change our local configuration 806 | if err := s.config.changeTo(makePeerMap(t.Peers...)); err != nil { 807 | t.Err <- err 808 | continue 809 | } 810 | 811 | // Serialize the local (C_old,new) configuration 812 | encodedConfiguration, err := s.config.encode() 813 | if err != nil { 814 | t.Err <- err 815 | continue 816 | } 817 | 818 | // We're gonna write+replicate that config via log mechanisms. 819 | // Prepare the on-commit callback. 820 | entry := logEntry{ 821 | Index: s.log.lastIndex() + 1, 822 | Term: s.term, 823 | Command: encodedConfiguration, 824 | isConfiguration: true, 825 | committed: make(chan bool), 826 | } 827 | go func() { 828 | committed := <-entry.committed 829 | if !committed { 830 | s.config.changeAborted() 831 | return 832 | } 833 | s.config.changeCommitted() 834 | if _, ok := s.config.allPeers()[s.id]; !ok { 835 | s.logGeneric("leader expelled; shutting down") 836 | q := make(chan struct{}) 837 | s.quit <- q 838 | <-q 839 | } 840 | }() 841 | if err := s.log.appendEntry(entry); err != nil { 842 | t.Err <- err 843 | continue 844 | } 845 | 846 | case <-flush: 847 | // Flushes attempt to sync the follower log with ours. 848 | // That requires per-follower state in the form of nextIndex. 849 | // After every flush, we check if we can advance our commitIndex. 850 | // If so, we do it, and trigger another flush ASAP. 851 | // A flush can cause us to be deposed. 852 | recipients := s.config.allPeers().except(s.id) 853 | 854 | // Special case: network of 1 855 | if len(recipients) <= 0 { 856 | ourLastIndex := s.log.lastIndex() 857 | if ourLastIndex > 0 { 858 | if err := s.log.commitTo(ourLastIndex); err != nil { 859 | s.logGeneric("commitTo(%d): %s", ourLastIndex, err) 860 | continue 861 | } 862 | s.logGeneric("after commitTo(%d), commitIndex=%d", ourLastIndex, s.log.getCommitIndex()) 863 | } 864 | continue 865 | } 866 | 867 | // Normal case: network of at-least-2 868 | successes, stepDown := s.concurrentFlush(recipients, ni, 2*broadcastInterval()) 869 | if stepDown { 870 | s.logGeneric("deposed during flush") 871 | s.state.Set(follower) 872 | s.leader = unknownLeader 873 | return 874 | } 875 | 876 | // Only when we know all followers accepted the flush can we 877 | // consider incrementing commitIndex and pushing out another 878 | // round of flushes. 879 | if successes == len(recipients) { 880 | peersBestIndex := ni.bestIndex() 881 | ourLastIndex := s.log.lastIndex() 882 | ourCommitIndex := s.log.getCommitIndex() 883 | if peersBestIndex > ourLastIndex { 884 | // safety check: we've probably been deposed 885 | s.logGeneric("peers' best index %d > our lastIndex %d", peersBestIndex, ourLastIndex) 886 | s.logGeneric("this is crazy, I'm gonna become a follower") 887 | s.leader = unknownLeader 888 | s.vote = noVote 889 | s.state.Set(follower) 890 | return 891 | } 892 | if peersBestIndex > ourCommitIndex { 893 | if err := s.log.commitTo(peersBestIndex); err != nil { 894 | s.logGeneric("commitTo(%d): %s", peersBestIndex, err) 895 | continue // oh well, next time? 896 | } 897 | if s.log.getCommitIndex() > ourCommitIndex { 898 | s.logGeneric("after commitTo(%d), commitIndex=%d -- queueing another flush", peersBestIndex, s.log.getCommitIndex()) 899 | go func() { flush <- struct{}{} }() 900 | } 901 | } 902 | } 903 | 904 | case t := <-s.appendEntriesChan: 905 | resp, stepDown := s.handleAppendEntries(t.Request) 906 | s.logAppendEntriesResponse(t.Request, resp, stepDown) 907 | t.Response <- resp 908 | if stepDown { 909 | s.logGeneric("after an appendEntries, deposed to Follower (leader=%d)", t.Request.LeaderID) 910 | s.leader = t.Request.LeaderID 911 | s.state.Set(follower) 912 | return // deposed 913 | } 914 | 915 | case t := <-s.requestVoteChan: 916 | resp, stepDown := s.handleRequestVote(t.Request) 917 | s.logRequestVoteResponse(t.Request, resp, stepDown) 918 | t.Response <- resp 919 | if stepDown { 920 | s.logGeneric("after a requestVote, deposed to Follower (leader unknown)") 921 | s.leader = unknownLeader 922 | s.state.Set(follower) 923 | return // deposed 924 | } 925 | } 926 | } 927 | } 928 | 929 | // handleRequestVote will modify s.term and s.vote, but nothing else. 930 | // stepDown means you need to: s.leader=unknownLeader, s.state.Set(Follower). 931 | func (s *Server) handleRequestVote(rv requestVote) (requestVoteResponse, bool) { 932 | // Spec is ambiguous here; basing this (loosely!) on benbjohnson's impl 933 | 934 | // If the request is from an old term, reject 935 | if rv.Term < s.term { 936 | return requestVoteResponse{ 937 | Term: s.term, 938 | VoteGranted: false, 939 | reason: fmt.Sprintf("Term %d < %d", rv.Term, s.term), 940 | }, false 941 | } 942 | 943 | // If the request is from a newer term, reset our state 944 | stepDown := false 945 | if rv.Term > s.term { 946 | s.logGeneric("requestVote from newer term (%d): we defer", rv.Term) 947 | s.term = rv.Term 948 | s.vote = noVote 949 | s.leader = unknownLeader 950 | stepDown = true 951 | } 952 | 953 | // Special case: if we're the leader, and we haven't been deposed by a more 954 | // recent term, then we should always deny the vote 955 | if s.state.Get() == leader && !stepDown { 956 | return requestVoteResponse{ 957 | Term: s.term, 958 | VoteGranted: false, 959 | reason: "already the leader", 960 | }, stepDown 961 | } 962 | 963 | // If we've already voted for someone else this term, reject 964 | if s.vote != 0 && s.vote != rv.CandidateID { 965 | if stepDown { 966 | panic("impossible state in handleRequestVote") 967 | } 968 | return requestVoteResponse{ 969 | Term: s.term, 970 | VoteGranted: false, 971 | reason: fmt.Sprintf("already cast vote for %d", s.vote), 972 | }, stepDown 973 | } 974 | 975 | // If the candidate log isn't at least as recent as ours, reject 976 | if s.log.lastIndex() > rv.LastLogIndex || s.log.lastTerm() > rv.LastLogTerm { 977 | return requestVoteResponse{ 978 | Term: s.term, 979 | VoteGranted: false, 980 | reason: fmt.Sprintf( 981 | "our index/term %d/%d > %d/%d", 982 | s.log.lastIndex(), 983 | s.log.lastTerm(), 984 | rv.LastLogIndex, 985 | rv.LastLogTerm, 986 | ), 987 | }, stepDown 988 | } 989 | 990 | // We passed all the tests: cast vote in favor 991 | s.vote = rv.CandidateID 992 | s.resetElectionTimeout() 993 | return requestVoteResponse{ 994 | Term: s.term, 995 | VoteGranted: true, 996 | }, stepDown 997 | } 998 | 999 | // handleAppendEntries will modify s.term and s.vote, but nothing else. 1000 | // stepDown means you need to: s.leader=r.LeaderID, s.state.Set(Follower). 1001 | func (s *Server) handleAppendEntries(r appendEntries) (appendEntriesResponse, bool) { 1002 | // Spec is ambiguous here; basing this on benbjohnson's impl 1003 | 1004 | // Maybe a nicer way to handle this is to define explicit handler functions 1005 | // for each Server state. Then, we won't try to hide too much logic (i.e. 1006 | // too many protocol rules) in one code path. 1007 | 1008 | // If the request is from an old term, reject 1009 | if r.Term < s.term { 1010 | return appendEntriesResponse{ 1011 | Term: s.term, 1012 | Success: false, 1013 | reason: fmt.Sprintf("Term %d < %d", r.Term, s.term), 1014 | }, false 1015 | } 1016 | 1017 | // If the request is from a newer term, reset our state 1018 | stepDown := false 1019 | if r.Term > s.term { 1020 | s.term = r.Term 1021 | s.vote = noVote 1022 | stepDown = true 1023 | } 1024 | 1025 | // Special case for candidates: "While waiting for votes, a candidate may 1026 | // receive an appendEntries RPC from another server claiming to be leader. 1027 | // If the leader’s term (included in its RPC) is at least as large as the 1028 | // candidate’s current term, then the candidate recognizes the leader as 1029 | // legitimate and steps down, meaning that it returns to follower state." 1030 | if s.state.Get() == candidate && r.LeaderID != s.leader && r.Term >= s.term { 1031 | s.term = r.Term 1032 | s.vote = noVote 1033 | stepDown = true 1034 | } 1035 | 1036 | // In any case, reset our election timeout 1037 | s.resetElectionTimeout() 1038 | 1039 | // Reject if log doesn't contain a matching previous entry 1040 | if err := s.log.ensureLastIs(r.PrevLogIndex, r.PrevLogTerm); err != nil { 1041 | return appendEntriesResponse{ 1042 | Term: s.term, 1043 | Success: false, 1044 | reason: fmt.Sprintf( 1045 | "while ensuring last log entry had index=%d term=%d: error: %s", 1046 | r.PrevLogIndex, 1047 | r.PrevLogTerm, 1048 | err, 1049 | ), 1050 | }, stepDown 1051 | } 1052 | 1053 | // Process the entries 1054 | for i, entry := range r.Entries { 1055 | // Configuration changes requre special preprocessing 1056 | var pm peerMap 1057 | if entry.isConfiguration { 1058 | commandBuf := bytes.NewBuffer(entry.Command) 1059 | if err := gob.NewDecoder(commandBuf).Decode(&pm); err != nil { 1060 | panic("gob decode of peers failed") 1061 | } 1062 | 1063 | if s.state.Get() == leader { 1064 | // TODO should we instead just ignore this entry? 1065 | return appendEntriesResponse{ 1066 | Term: s.term, 1067 | Success: false, 1068 | reason: fmt.Sprintf( 1069 | "AppendEntry %d/%d failed (configuration): %s", 1070 | i+1, 1071 | len(r.Entries), 1072 | "Leader shouldn't receive configurations via appendEntries", 1073 | ), 1074 | }, stepDown 1075 | } 1076 | 1077 | // Expulsion recognition 1078 | if _, ok := pm[s.id]; !ok { 1079 | entry.committed = make(chan bool) 1080 | go func() { 1081 | if <-entry.committed { 1082 | s.logGeneric("non-leader expelled; shutting down") 1083 | q := make(chan struct{}) 1084 | s.quit <- q 1085 | <-q 1086 | } 1087 | }() 1088 | } 1089 | } 1090 | 1091 | // Append entry to the log 1092 | if err := s.log.appendEntry(entry); err != nil { 1093 | return appendEntriesResponse{ 1094 | Term: s.term, 1095 | Success: false, 1096 | reason: fmt.Sprintf( 1097 | "AppendEntry %d/%d failed: %s", 1098 | i+1, 1099 | len(r.Entries), 1100 | err, 1101 | ), 1102 | }, stepDown 1103 | } 1104 | 1105 | // "Once a given server adds the new configuration entry to its log, it 1106 | // uses that configuration for all future decisions (it does not wait 1107 | // for the entry to become committed)." 1108 | if entry.isConfiguration { 1109 | if err := s.config.directSet(pm); err != nil { 1110 | return appendEntriesResponse{ 1111 | Term: s.term, 1112 | Success: false, 1113 | reason: fmt.Sprintf( 1114 | "AppendEntry %d/%d failed (configuration): %s", 1115 | i+1, 1116 | len(r.Entries), 1117 | err, 1118 | ), 1119 | }, stepDown 1120 | } 1121 | } 1122 | } 1123 | 1124 | // Commit up to the commit index. 1125 | // 1126 | // < ptrb> ongardie: if the new leader sends a 0-entry appendEntries 1127 | // with lastIndex=5 commitIndex=4, to a follower that has lastIndex=5 1128 | // commitIndex=5 -- in my impl, this fails, because commitIndex is too 1129 | // small. shouldn't be? 1130 | // <@ongardie> ptrb: i don't think that should fail 1131 | // <@ongardie> there are 4 ways an appendEntries request can fail: (1) 1132 | // network drops packet (2) caller has stale term (3) would leave gap in 1133 | // the recipient's log (4) term of entry preceding the new entries doesn't 1134 | // match the term at the same index on the recipient 1135 | // 1136 | if r.CommitIndex > 0 && r.CommitIndex > s.log.getCommitIndex() { 1137 | if err := s.log.commitTo(r.CommitIndex); err != nil { 1138 | return appendEntriesResponse{ 1139 | Term: s.term, 1140 | Success: false, 1141 | reason: fmt.Sprintf("CommitTo(%d) failed: %s", r.CommitIndex, err), 1142 | }, stepDown 1143 | } 1144 | } 1145 | 1146 | // all good 1147 | return appendEntriesResponse{ 1148 | Term: s.term, 1149 | Success: true, 1150 | }, stepDown 1151 | } 1152 | --------------------------------------------------------------------------------