├── .gitignore ├── LICENSE ├── Makefile ├── README.md ├── append_entries.go ├── append_entries_test.go ├── command.go ├── commands.go ├── config.go ├── context.go ├── debug.go ├── event.go ├── event_dispatcher.go ├── event_dispatcher_test.go ├── http_transporter.go ├── http_transporter_test.go ├── log.go ├── log_entry.go ├── log_test.go ├── peer.go ├── protobuf ├── append_entries_request.pb.go ├── append_entries_request.proto ├── append_entries_responses.pb.go ├── append_entries_responses.proto ├── log_entry.pb.go ├── log_entry.proto ├── request_vote_request.pb.go ├── request_vote_request.proto ├── request_vote_responses.pb.go ├── request_vote_responses.proto ├── snapshot_recovery_request.pb.go ├── snapshot_recovery_request.proto ├── snapshot_recovery_response.pb.go ├── snapshot_recovery_response.proto ├── snapshot_request.pb.go ├── snapshot_request.proto ├── snapshot_response.pb.go └── snapshot_response.proto ├── request_vote.go ├── server.go ├── server_test.go ├── snapshot.go ├── snapshot_test.go ├── statemachine.go ├── statemachine_test.go ├── test.go ├── transporter.go ├── util.go └── z_test.go /.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 | 24 | coverage.html 25 | coverprofile.out 26 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Copyright 2013 go-raft contributors 2 | 3 | Permission is hereby granted, free of charge, to any person obtaining 4 | a copy of this software and associated documentation files (the 5 | "Software"), to deal in the Software without restriction, including 6 | without limitation the rights to use, copy, modify, merge, publish, 7 | distribute, sublicense, and/or sell copies of the Software, and to 8 | permit persons to whom the Software is furnished to do so, subject to 9 | the following conditions: 10 | 11 | The above copyright notice and this permission notice shall be 12 | included in all copies or substantial portions of the Software. 13 | 14 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, 15 | EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF 16 | MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND 17 | NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE 18 | LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION 19 | OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION 20 | WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. 21 | -------------------------------------------------------------------------------- /Makefile: -------------------------------------------------------------------------------- 1 | COVERPROFILE=cover.out 2 | 3 | default: test 4 | 5 | cover: 6 | go test -coverprofile=$(COVERPROFILE) . 7 | go tool cover -html=$(COVERPROFILE) 8 | rm $(COVERPROFILE) 9 | 10 | dependencies: 11 | go get -d . 12 | 13 | test: 14 | go test -i ./... 15 | go test -v ./... 16 | 17 | .PHONY: coverage dependencies test 18 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | go-raft [![Build Status](https://drone.io/github.com/goraft/raft/status.png)](https://drone.io/github.com/goraft/raft/latest) [![Coverage Status](https://coveralls.io/repos/goraft/raft/badge.png?branch=master)](https://coveralls.io/r/goraft/raft?branch=master) 2 | ======= 3 | 4 | ## Overview 5 | 6 | ![unmaintained](http://img.shields.io/badge/status-unmaintained-red.png) 7 | 8 | 9 | **NOTE**: This project is unmaintained. If you are using goraft in a project 10 | and want to carry the project forward please file an issue with your ideas and 11 | intentions. The original project authors have created new raft implementations 12 | now used in [etcd](https://godoc.org/github.com/coreos/etcd/raft) and [InfluxDB](https://godoc.org/github.com/influxdb/influxdb/raft). 13 | 14 | This is a Go implementation of the Raft distributed consensus protocol. 15 | Raft is a protocol by which a cluster of nodes can maintain a replicated state machine. 16 | The state machine is kept in sync through the use of a replicated log. 17 | 18 | For more details on Raft, you can read [In Search of an Understandable Consensus Algorithm][raft-paper] by Diego Ongaro and John Ousterhout. 19 | 20 | ## Project Status 21 | 22 | This library is feature complete but should be considered experimental until it has seen more usage. 23 | If you have any questions on implementing go-raft in your project please file an issue. 24 | There is an [active community][community] of developers who can help. 25 | go-raft is under the MIT license. 26 | 27 | [community]: https://github.com/goraft/raft/contributors 28 | 29 | ### Features 30 | 31 | - Leader election 32 | - Log replication 33 | - Configuration changes 34 | - Log compaction 35 | - Unit tests 36 | - Fast Protobuf Log Encoding 37 | - HTTP transport 38 | 39 | ### Projects 40 | 41 | These projects are built on go-raft: 42 | 43 | 44 | - [goraft/raftd](https://github.com/goraft/raftd) - A reference implementation for using the go-raft library for distributed consensus. 45 | - [Weed File System](https://weed-fs.googlecode.com) - A scalable distributed key-to-file system with O(1) disk access for each read. 46 | - [rqlite](https://github.com/otoolep/rqlite) - A replicated SQLite database, distributing the database replicas across multiple nodes. 47 | 48 | If you have a project that you're using go-raft in, please add it to this README so others can see implementation examples. 49 | 50 | ## Contact and Resources 51 | 52 | - [raft-dev][raft-dev] is a mailing list for discussion about best practices 53 | and implementation of Raft. Not goraft specific but helpful if you have 54 | questions. 55 | - [Slides from Ben's talk][bens-talk] which includes easy to understand 56 | diagrams of leader election and replication 57 | - The [Raft Consensus homepage][raft-home] has links to additional raft 58 | implementations, slides to talks on Raft and general information 59 | 60 | [raft-home]: http://raftconsensus.github.io/ 61 | [raft-dev]: https://groups.google.com/forum/#!forum/raft-dev 62 | [bens-talk]: https://speakerdeck.com/benbjohnson/raft-the-understandable-distributed-consensus-protocol 63 | 64 | ## The Raft Protocol 65 | 66 | This section provides a summary of the Raft protocol from a high level. 67 | For a more detailed explanation on the failover process and election terms please see the full paper describing the protocol: [In Search of an Understandable Consensus Algorithm][raft-paper]. 68 | 69 | ### Overview 70 | 71 | Maintaining state in a single process on a single server is easy. 72 | Your process is a single point of authority so there are no conflicts when reading and writing state. 73 | Even multi-threaded processes can rely on locks or coroutines to serialize access to the data. 74 | 75 | However, in a distributed system there is no single point of authority. 76 | Servers can crash or the network between two machines can become unavailable or any number of other problems can occur. 77 | 78 | A distributed consensus protocol is used for maintaining a consistent state across multiple servers in a cluster. 79 | Many distributed systems are built upon the Paxos protocol but Paxos can be difficult to understand and there are many gaps between Paxos and real world implementation. 80 | 81 | An alternative is the [Raft distributed consensus protocol][raft-paper] by Diego Ongaro and John Ousterhout. 82 | Raft is a protocol built with understandability as a primary tenet and it centers around two things: 83 | 84 | 1. Leader Election 85 | 2. Replicated Log 86 | 87 | With these two constructs, you can build a system that can maintain state across multiple servers -- even in the event of multiple failures. 88 | 89 | ### Leader Election 90 | 91 | The Raft protocol effectively works as a master-slave system whereby state changes are written to a single server in the cluster and are distributed out to the rest of the servers in the cluster. 92 | This simplifies the protocol since there is only one data authority and conflicts will not have to be resolved. 93 | 94 | Raft ensures that there is only one leader at a time. 95 | It does this by performing elections among the nodes in the cluster and requiring that a node must receive a majority of the votes in order to become leader. 96 | For example, if you have 3 nodes in your cluster then a single node would need 2 votes in order to become the leader. 97 | For a 5 node cluster, a server would need 3 votes to become leader. 98 | 99 | ### Replicated Log 100 | 101 | To maintain state, a log of commands is maintained. 102 | Each command makes a change to the state of the server and the command is deterministic. 103 | By ensuring that this log is replicated identically between all the nodes in the cluster we can replicate the state at any point in time in the log by running each command sequentially. 104 | 105 | Replicating the log under normal conditions is done by sending an `AppendEntries` RPC from the leader to each of the other servers in the cluster (called Peers). 106 | Each peer will append the entries from the leader through a 2-phase commit process which ensure that a majority of servers in the cluster have entries written to log. 107 | 108 | 109 | ## Raft in Practice 110 | 111 | ### Optimal Cluster Size 112 | 113 | The primary consideration when choosing the node count in your Raft cluster is the number of nodes that can simultaneously fail. 114 | Because Raft requires a majority of nodes to be available to make progress, the number of node failures the cluster can tolerate is `(n / 2) - 1`. 115 | 116 | This means that a 3-node cluster can tolerate 1 node failure. 117 | If 2 nodes fail then the cluster cannot commit entries or elect a new leader so progress stops. 118 | A 5-node cluster can tolerate 2 node failures. A 9-node cluster can tolerate 4 node failures. 119 | It is unlikely that 4 nodes will simultaneously fail so clusters larger than 9 nodes are not common. 120 | 121 | Another consideration is performance. 122 | The leader must replicate log entries for each follower node so CPU and networking resources can quickly be bottlenecked under stress in a large cluster. 123 | 124 | 125 | ### Scaling Raft 126 | 127 | Once you grow beyond the maximum size of your cluster there are a few options for scaling Raft: 128 | 129 | 1. *Core nodes with dumb replication.* 130 | This option requires you to maintain a small cluster (e.g. 5 nodes) that is involved in the Raft process and then replicate only committed log entries to the remaining nodes in the cluster. 131 | This works well if you have reads in your system that can be stale. 132 | 133 | 2. *Sharding.* 134 | This option requires that you segment your data into different clusters. 135 | This option works well if you need very strong consistency and therefore need to read and write heavily from the leader. 136 | 137 | If you have a very large cluster that you need to replicate to using Option 1 then you may want to look at performing hierarchical replication so that nodes can better share the load. 138 | 139 | 140 | ## History 141 | 142 | Ben Johnson started this library for use in his behavioral analytics database called [Sky](https://github.com/skydb/sky). 143 | He put it under the MIT license in the hopes that it would be useful for other projects too. 144 | 145 | [raft-paper]: https://ramcloud.stanford.edu/raft.pdf 146 | -------------------------------------------------------------------------------- /append_entries.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "io" 5 | "io/ioutil" 6 | 7 | "code.google.com/p/gogoprotobuf/proto" 8 | "github.com/goraft/raft/protobuf" 9 | ) 10 | 11 | // The request sent to a server to append entries to the log. 12 | type AppendEntriesRequest struct { 13 | Term uint64 14 | PrevLogIndex uint64 15 | PrevLogTerm uint64 16 | CommitIndex uint64 17 | LeaderName string 18 | Entries []*protobuf.LogEntry 19 | } 20 | 21 | // The response returned from a server appending entries to the log. 22 | type AppendEntriesResponse struct { 23 | pb *protobuf.AppendEntriesResponse 24 | peer string 25 | append bool 26 | } 27 | 28 | // Creates a new AppendEntries request. 29 | func newAppendEntriesRequest(term uint64, prevLogIndex uint64, prevLogTerm uint64, 30 | commitIndex uint64, leaderName string, entries []*LogEntry) *AppendEntriesRequest { 31 | pbEntries := make([]*protobuf.LogEntry, len(entries)) 32 | 33 | for i := range entries { 34 | pbEntries[i] = entries[i].pb 35 | } 36 | 37 | return &AppendEntriesRequest{ 38 | Term: term, 39 | PrevLogIndex: prevLogIndex, 40 | PrevLogTerm: prevLogTerm, 41 | CommitIndex: commitIndex, 42 | LeaderName: leaderName, 43 | Entries: pbEntries, 44 | } 45 | } 46 | 47 | // Encodes the AppendEntriesRequest to a buffer. Returns the number of bytes 48 | // written and any error that may have occurred. 49 | func (req *AppendEntriesRequest) Encode(w io.Writer) (int, error) { 50 | pb := &protobuf.AppendEntriesRequest{ 51 | Term: proto.Uint64(req.Term), 52 | PrevLogIndex: proto.Uint64(req.PrevLogIndex), 53 | PrevLogTerm: proto.Uint64(req.PrevLogTerm), 54 | CommitIndex: proto.Uint64(req.CommitIndex), 55 | LeaderName: proto.String(req.LeaderName), 56 | Entries: req.Entries, 57 | } 58 | 59 | p, err := proto.Marshal(pb) 60 | if err != nil { 61 | return -1, err 62 | } 63 | 64 | return w.Write(p) 65 | } 66 | 67 | // Decodes the AppendEntriesRequest from a buffer. Returns the number of bytes read and 68 | // any error that occurs. 69 | func (req *AppendEntriesRequest) Decode(r io.Reader) (int, error) { 70 | data, err := ioutil.ReadAll(r) 71 | 72 | if err != nil { 73 | return -1, err 74 | } 75 | 76 | pb := new(protobuf.AppendEntriesRequest) 77 | if err := proto.Unmarshal(data, pb); err != nil { 78 | return -1, err 79 | } 80 | 81 | req.Term = pb.GetTerm() 82 | req.PrevLogIndex = pb.GetPrevLogIndex() 83 | req.PrevLogTerm = pb.GetPrevLogTerm() 84 | req.CommitIndex = pb.GetCommitIndex() 85 | req.LeaderName = pb.GetLeaderName() 86 | req.Entries = pb.GetEntries() 87 | 88 | return len(data), nil 89 | } 90 | 91 | // Creates a new AppendEntries response. 92 | func newAppendEntriesResponse(term uint64, success bool, index uint64, commitIndex uint64) *AppendEntriesResponse { 93 | pb := &protobuf.AppendEntriesResponse{ 94 | Term: proto.Uint64(term), 95 | Index: proto.Uint64(index), 96 | Success: proto.Bool(success), 97 | CommitIndex: proto.Uint64(commitIndex), 98 | } 99 | 100 | return &AppendEntriesResponse{ 101 | pb: pb, 102 | } 103 | } 104 | 105 | func (aer *AppendEntriesResponse) Index() uint64 { 106 | return aer.pb.GetIndex() 107 | } 108 | 109 | func (aer *AppendEntriesResponse) CommitIndex() uint64 { 110 | return aer.pb.GetCommitIndex() 111 | } 112 | 113 | func (aer *AppendEntriesResponse) Term() uint64 { 114 | return aer.pb.GetTerm() 115 | } 116 | 117 | func (aer *AppendEntriesResponse) Success() bool { 118 | return aer.pb.GetSuccess() 119 | } 120 | 121 | // Encodes the AppendEntriesResponse to a buffer. Returns the number of bytes 122 | // written and any error that may have occurred. 123 | func (resp *AppendEntriesResponse) Encode(w io.Writer) (int, error) { 124 | b, err := proto.Marshal(resp.pb) 125 | if err != nil { 126 | return -1, err 127 | } 128 | 129 | return w.Write(b) 130 | } 131 | 132 | // Decodes the AppendEntriesResponse from a buffer. Returns the number of bytes read and 133 | // any error that occurs. 134 | func (resp *AppendEntriesResponse) Decode(r io.Reader) (int, error) { 135 | data, err := ioutil.ReadAll(r) 136 | if err != nil { 137 | return -1, err 138 | } 139 | 140 | resp.pb = new(protobuf.AppendEntriesResponse) 141 | if err := proto.Unmarshal(data, resp.pb); err != nil { 142 | return -1, err 143 | } 144 | 145 | return len(data), nil 146 | } 147 | -------------------------------------------------------------------------------- /append_entries_test.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "bytes" 5 | "testing" 6 | ) 7 | 8 | func BenchmarkAppendEntriesRequestEncoding(b *testing.B) { 9 | req, tmp := createTestAppendEntriesRequest(2000) 10 | b.ResetTimer() 11 | for i := 0; i < b.N; i++ { 12 | var buf bytes.Buffer 13 | req.Encode(&buf) 14 | } 15 | b.SetBytes(int64(len(tmp))) 16 | } 17 | 18 | func BenchmarkAppendEntriesRequestDecoding(b *testing.B) { 19 | req, buf := createTestAppendEntriesRequest(2000) 20 | b.ResetTimer() 21 | for i := 0; i < b.N; i++ { 22 | req.Decode(bytes.NewReader(buf)) 23 | } 24 | b.SetBytes(int64(len(buf))) 25 | } 26 | 27 | func BenchmarkAppendEntriesResponseEncoding(b *testing.B) { 28 | req, tmp := createTestAppendEntriesResponse(2000) 29 | b.ResetTimer() 30 | for i := 0; i < b.N; i++ { 31 | var buf bytes.Buffer 32 | req.Encode(&buf) 33 | } 34 | b.SetBytes(int64(len(tmp))) 35 | } 36 | 37 | func BenchmarkAppendEntriesResponseDecoding(b *testing.B) { 38 | req, buf := createTestAppendEntriesResponse(2000) 39 | b.ResetTimer() 40 | for i := 0; i < b.N; i++ { 41 | req.Decode(bytes.NewReader(buf)) 42 | } 43 | b.SetBytes(int64(len(buf))) 44 | } 45 | 46 | func createTestAppendEntriesRequest(entryCount int) (*AppendEntriesRequest, []byte) { 47 | entries := make([]*LogEntry, 0) 48 | for i := 0; i < entryCount; i++ { 49 | command := &DefaultJoinCommand{Name: "localhost:1000"} 50 | entry, _ := newLogEntry(nil, nil, 1, 2, command) 51 | entries = append(entries, entry) 52 | } 53 | req := newAppendEntriesRequest(1, 1, 1, 1, "leader", entries) 54 | 55 | var buf bytes.Buffer 56 | req.Encode(&buf) 57 | 58 | return req, buf.Bytes() 59 | } 60 | 61 | func createTestAppendEntriesResponse(entryCount int) (*AppendEntriesResponse, []byte) { 62 | resp := newAppendEntriesResponse(1, true, 1, 1) 63 | 64 | var buf bytes.Buffer 65 | resp.Encode(&buf) 66 | 67 | return resp, buf.Bytes() 68 | } 69 | -------------------------------------------------------------------------------- /command.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "bytes" 5 | "encoding/json" 6 | "fmt" 7 | "io" 8 | "reflect" 9 | ) 10 | 11 | var commandTypes map[string]Command 12 | 13 | func init() { 14 | commandTypes = map[string]Command{} 15 | } 16 | 17 | // Command represents an action to be taken on the replicated state machine. 18 | type Command interface { 19 | CommandName() string 20 | } 21 | 22 | // CommandApply represents the interface to apply a command to the server. 23 | type CommandApply interface { 24 | Apply(Context) (interface{}, error) 25 | } 26 | 27 | // deprecatedCommandApply represents the old interface to apply a command to the server. 28 | type deprecatedCommandApply interface { 29 | Apply(Server) (interface{}, error) 30 | } 31 | 32 | type CommandEncoder interface { 33 | Encode(w io.Writer) error 34 | Decode(r io.Reader) error 35 | } 36 | 37 | // Creates a new instance of a command by name. 38 | func newCommand(name string, data []byte) (Command, error) { 39 | // Find the registered command. 40 | command := commandTypes[name] 41 | if command == nil { 42 | return nil, fmt.Errorf("raft.Command: Unregistered command type: %s", name) 43 | } 44 | 45 | // Make a copy of the command. 46 | v := reflect.New(reflect.Indirect(reflect.ValueOf(command)).Type()).Interface() 47 | copy, ok := v.(Command) 48 | if !ok { 49 | panic(fmt.Sprintf("raft: Unable to copy command: %s (%v)", command.CommandName(), reflect.ValueOf(v).Kind().String())) 50 | } 51 | 52 | // If data for the command was passed in the decode it. 53 | if data != nil { 54 | if encoder, ok := copy.(CommandEncoder); ok { 55 | if err := encoder.Decode(bytes.NewReader(data)); err != nil { 56 | return nil, err 57 | } 58 | } else { 59 | if err := json.NewDecoder(bytes.NewReader(data)).Decode(copy); err != nil { 60 | return nil, err 61 | } 62 | } 63 | } 64 | 65 | return copy, nil 66 | } 67 | 68 | // Registers a command by storing a reference to an instance of it. 69 | func RegisterCommand(command Command) { 70 | if command == nil { 71 | panic(fmt.Sprintf("raft: Cannot register nil")) 72 | } else if commandTypes[command.CommandName()] != nil { 73 | panic(fmt.Sprintf("raft: Duplicate registration: %s", command.CommandName())) 74 | } 75 | commandTypes[command.CommandName()] = command 76 | } 77 | -------------------------------------------------------------------------------- /commands.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "io" 5 | ) 6 | 7 | // Join command interface 8 | type JoinCommand interface { 9 | Command 10 | NodeName() string 11 | } 12 | 13 | // Join command 14 | type DefaultJoinCommand struct { 15 | Name string `json:"name"` 16 | ConnectionString string `json:"connectionString"` 17 | } 18 | 19 | // Leave command interface 20 | type LeaveCommand interface { 21 | Command 22 | NodeName() string 23 | } 24 | 25 | // Leave command 26 | type DefaultLeaveCommand struct { 27 | Name string `json:"name"` 28 | } 29 | 30 | // NOP command 31 | type NOPCommand struct { 32 | } 33 | 34 | // The name of the Join command in the log 35 | func (c *DefaultJoinCommand) CommandName() string { 36 | return "raft:join" 37 | } 38 | 39 | func (c *DefaultJoinCommand) Apply(server Server) (interface{}, error) { 40 | err := server.AddPeer(c.Name, c.ConnectionString) 41 | 42 | return []byte("join"), err 43 | } 44 | 45 | func (c *DefaultJoinCommand) NodeName() string { 46 | return c.Name 47 | } 48 | 49 | // The name of the Leave command in the log 50 | func (c *DefaultLeaveCommand) CommandName() string { 51 | return "raft:leave" 52 | } 53 | 54 | func (c *DefaultLeaveCommand) Apply(server Server) (interface{}, error) { 55 | err := server.RemovePeer(c.Name) 56 | 57 | return []byte("leave"), err 58 | } 59 | func (c *DefaultLeaveCommand) NodeName() string { 60 | return c.Name 61 | } 62 | 63 | // The name of the NOP command in the log 64 | func (c NOPCommand) CommandName() string { 65 | return "raft:nop" 66 | } 67 | 68 | func (c NOPCommand) Apply(server Server) (interface{}, error) { 69 | return nil, nil 70 | } 71 | 72 | func (c NOPCommand) Encode(w io.Writer) error { 73 | return nil 74 | } 75 | 76 | func (c NOPCommand) Decode(r io.Reader) error { 77 | return nil 78 | } 79 | -------------------------------------------------------------------------------- /config.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | type Config struct { 4 | CommitIndex uint64 `json:"commitIndex"` 5 | // TODO decide what we need to store in peer struct 6 | Peers []*Peer `json:"peers"` 7 | } 8 | -------------------------------------------------------------------------------- /context.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | // Context represents the current state of the server. It is passed into 4 | // a command when the command is being applied since the server methods 5 | // are locked. 6 | type Context interface { 7 | Server() Server 8 | CurrentTerm() uint64 9 | CurrentIndex() uint64 10 | CommitIndex() uint64 11 | } 12 | 13 | // context is the concrete implementation of Context. 14 | type context struct { 15 | server Server 16 | currentIndex uint64 17 | currentTerm uint64 18 | commitIndex uint64 19 | } 20 | 21 | // Server returns a reference to the server. 22 | func (c *context) Server() Server { 23 | return c.server 24 | } 25 | 26 | // CurrentTerm returns current term the server is in. 27 | func (c *context) CurrentTerm() uint64 { 28 | return c.currentTerm 29 | } 30 | 31 | // CurrentIndex returns current index the server is at. 32 | func (c *context) CurrentIndex() uint64 { 33 | return c.currentIndex 34 | } 35 | 36 | // CommitIndex returns last commit index the server is at. 37 | func (c *context) CommitIndex() uint64 { 38 | return c.commitIndex 39 | } 40 | -------------------------------------------------------------------------------- /debug.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "log" 5 | "os" 6 | ) 7 | 8 | //------------------------------------------------------------------------------ 9 | // 10 | // Variables 11 | // 12 | //------------------------------------------------------------------------------ 13 | 14 | const ( 15 | Debug = 1 16 | Trace = 2 17 | ) 18 | 19 | var logLevel int = 0 20 | var logger *log.Logger 21 | 22 | func init() { 23 | logger = log.New(os.Stdout, "[raft]", log.Lmicroseconds) 24 | } 25 | 26 | //------------------------------------------------------------------------------ 27 | // 28 | // Functions 29 | // 30 | //------------------------------------------------------------------------------ 31 | 32 | func LogLevel() int { 33 | return logLevel 34 | } 35 | 36 | func SetLogLevel(level int) { 37 | logLevel = level 38 | } 39 | 40 | //-------------------------------------- 41 | // Warnings 42 | //-------------------------------------- 43 | 44 | // Prints to the standard logger. Arguments are handled in the manner of 45 | // fmt.Print. 46 | func warn(v ...interface{}) { 47 | logger.Print(v...) 48 | } 49 | 50 | // Prints to the standard logger. Arguments are handled in the manner of 51 | // fmt.Printf. 52 | func warnf(format string, v ...interface{}) { 53 | logger.Printf(format, v...) 54 | } 55 | 56 | // Prints to the standard logger. Arguments are handled in the manner of 57 | // fmt.Println. 58 | func warnln(v ...interface{}) { 59 | logger.Println(v...) 60 | } 61 | 62 | //-------------------------------------- 63 | // Basic debugging 64 | //-------------------------------------- 65 | 66 | // Prints to the standard logger if debug mode is enabled. Arguments 67 | // are handled in the manner of fmt.Print. 68 | func debug(v ...interface{}) { 69 | if logLevel >= Debug { 70 | logger.Print(v...) 71 | } 72 | } 73 | 74 | // Prints to the standard logger if debug mode is enabled. Arguments 75 | // are handled in the manner of fmt.Printf. 76 | func debugf(format string, v ...interface{}) { 77 | if logLevel >= Debug { 78 | logger.Printf(format, v...) 79 | } 80 | } 81 | 82 | // Prints to the standard logger if debug mode is enabled. Arguments 83 | // are handled in the manner of fmt.Println. 84 | func debugln(v ...interface{}) { 85 | if logLevel >= Debug { 86 | logger.Println(v...) 87 | } 88 | } 89 | 90 | //-------------------------------------- 91 | // Trace-level debugging 92 | //-------------------------------------- 93 | 94 | // Prints to the standard logger if trace debugging is enabled. Arguments 95 | // are handled in the manner of fmt.Print. 96 | func trace(v ...interface{}) { 97 | if logLevel >= Trace { 98 | logger.Print(v...) 99 | } 100 | } 101 | 102 | // Prints to the standard logger if trace debugging is enabled. Arguments 103 | // are handled in the manner of fmt.Printf. 104 | func tracef(format string, v ...interface{}) { 105 | if logLevel >= Trace { 106 | logger.Printf(format, v...) 107 | } 108 | } 109 | 110 | // Prints to the standard logger if trace debugging is enabled. Arguments 111 | // are handled in the manner of debugln. 112 | func traceln(v ...interface{}) { 113 | if logLevel >= Trace { 114 | logger.Println(v...) 115 | } 116 | } 117 | -------------------------------------------------------------------------------- /event.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | const ( 4 | StateChangeEventType = "stateChange" 5 | LeaderChangeEventType = "leaderChange" 6 | TermChangeEventType = "termChange" 7 | CommitEventType = "commit" 8 | AddPeerEventType = "addPeer" 9 | RemovePeerEventType = "removePeer" 10 | 11 | HeartbeatIntervalEventType = "heartbeatInterval" 12 | ElectionTimeoutThresholdEventType = "electionTimeoutThreshold" 13 | 14 | HeartbeatEventType = "heartbeat" 15 | ) 16 | 17 | // Event represents an action that occurred within the Raft library. 18 | // Listeners can subscribe to event types by using the Server.AddEventListener() function. 19 | type Event interface { 20 | Type() string 21 | Source() interface{} 22 | Value() interface{} 23 | PrevValue() interface{} 24 | } 25 | 26 | // event is the concrete implementation of the Event interface. 27 | type event struct { 28 | typ string 29 | source interface{} 30 | value interface{} 31 | prevValue interface{} 32 | } 33 | 34 | // newEvent creates a new event. 35 | func newEvent(typ string, value interface{}, prevValue interface{}) *event { 36 | return &event{ 37 | typ: typ, 38 | value: value, 39 | prevValue: prevValue, 40 | } 41 | } 42 | 43 | // Type returns the type of event that occurred. 44 | func (e *event) Type() string { 45 | return e.typ 46 | } 47 | 48 | // Source returns the object that dispatched the event. 49 | func (e *event) Source() interface{} { 50 | return e.source 51 | } 52 | 53 | // Value returns the current value associated with the event, if applicable. 54 | func (e *event) Value() interface{} { 55 | return e.value 56 | } 57 | 58 | // PrevValue returns the previous value associated with the event, if applicable. 59 | func (e *event) PrevValue() interface{} { 60 | return e.prevValue 61 | } 62 | -------------------------------------------------------------------------------- /event_dispatcher.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "reflect" 5 | "sync" 6 | ) 7 | 8 | // eventDispatcher is responsible for managing listeners for named events 9 | // and dispatching event notifications to those listeners. 10 | type eventDispatcher struct { 11 | sync.RWMutex 12 | source interface{} 13 | listeners map[string]eventListeners 14 | } 15 | 16 | // EventListener is a function that can receive event notifications. 17 | type EventListener func(Event) 18 | 19 | // EventListeners represents a collection of individual listeners. 20 | type eventListeners []EventListener 21 | 22 | // newEventDispatcher creates a new eventDispatcher instance. 23 | func newEventDispatcher(source interface{}) *eventDispatcher { 24 | return &eventDispatcher{ 25 | source: source, 26 | listeners: make(map[string]eventListeners), 27 | } 28 | } 29 | 30 | // AddEventListener adds a listener function for a given event type. 31 | func (d *eventDispatcher) AddEventListener(typ string, listener EventListener) { 32 | d.Lock() 33 | defer d.Unlock() 34 | d.listeners[typ] = append(d.listeners[typ], listener) 35 | } 36 | 37 | // RemoveEventListener removes a listener function for a given event type. 38 | func (d *eventDispatcher) RemoveEventListener(typ string, listener EventListener) { 39 | d.Lock() 40 | defer d.Unlock() 41 | 42 | // Grab a reference to the function pointer once. 43 | ptr := reflect.ValueOf(listener).Pointer() 44 | 45 | // Find listener by pointer and remove it. 46 | listeners := d.listeners[typ] 47 | for i, l := range listeners { 48 | if reflect.ValueOf(l).Pointer() == ptr { 49 | d.listeners[typ] = append(listeners[:i], listeners[i+1:]...) 50 | } 51 | } 52 | } 53 | 54 | // DispatchEvent dispatches an event. 55 | func (d *eventDispatcher) DispatchEvent(e Event) { 56 | d.RLock() 57 | defer d.RUnlock() 58 | 59 | // Automatically set the event source. 60 | if e, ok := e.(*event); ok { 61 | e.source = d.source 62 | } 63 | 64 | // Dispatch the event to all listeners. 65 | for _, l := range d.listeners[e.Type()] { 66 | l(e) 67 | } 68 | } 69 | -------------------------------------------------------------------------------- /event_dispatcher_test.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "testing" 5 | 6 | "github.com/stretchr/testify/assert" 7 | ) 8 | 9 | // Ensure that we can listen and dispatch events. 10 | func TestDispatchEvent(t *testing.T) { 11 | var count int 12 | dispatcher := newEventDispatcher(nil) 13 | dispatcher.AddEventListener("foo", func(e Event) { 14 | count += 1 15 | }) 16 | dispatcher.AddEventListener("foo", func(e Event) { 17 | count += 10 18 | }) 19 | dispatcher.AddEventListener("bar", func(e Event) { 20 | count += 100 21 | }) 22 | dispatcher.DispatchEvent(&event{typ: "foo", value: nil, prevValue: nil}) 23 | assert.Equal(t, 11, count) 24 | } 25 | 26 | // Ensure that we can add and remove a listener. 27 | func TestRemoveEventListener(t *testing.T) { 28 | var count int 29 | f0 := func(e Event) { 30 | count += 1 31 | } 32 | f1 := func(e Event) { 33 | count += 10 34 | } 35 | 36 | dispatcher := newEventDispatcher(nil) 37 | dispatcher.AddEventListener("foo", f0) 38 | dispatcher.AddEventListener("foo", f1) 39 | dispatcher.DispatchEvent(&event{typ: "foo"}) 40 | dispatcher.RemoveEventListener("foo", f0) 41 | dispatcher.DispatchEvent(&event{typ: "foo"}) 42 | assert.Equal(t, 21, count) 43 | } 44 | 45 | // Ensure that event is properly passed to listener. 46 | func TestEventListener(t *testing.T) { 47 | dispatcher := newEventDispatcher("X") 48 | dispatcher.AddEventListener("foo", func(e Event) { 49 | assert.Equal(t, "foo", e.Type()) 50 | assert.Equal(t, "X", e.Source()) 51 | assert.Equal(t, 10, e.Value()) 52 | assert.Equal(t, 20, e.PrevValue()) 53 | }) 54 | dispatcher.DispatchEvent(&event{typ: "foo", value: 10, prevValue: 20}) 55 | } 56 | 57 | // Benchmark the performance of event dispatch. 58 | func BenchmarkEventDispatch(b *testing.B) { 59 | dispatcher := newEventDispatcher(nil) 60 | dispatcher.AddEventListener("xxx", func(e Event) {}) 61 | for i := 0; i < b.N; i++ { 62 | dispatcher.DispatchEvent(&event{typ: "foo", value: 10, prevValue: 20}) 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /http_transporter.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "bytes" 5 | "fmt" 6 | "io" 7 | "net/http" 8 | "net/url" 9 | "path" 10 | "time" 11 | ) 12 | 13 | // Parts from this transporter were heavily influenced by Peter Bougon's 14 | // raft implementation: https://github.com/peterbourgon/raft 15 | 16 | //------------------------------------------------------------------------------ 17 | // 18 | // Typedefs 19 | // 20 | //------------------------------------------------------------------------------ 21 | 22 | // An HTTPTransporter is a default transport layer used to communicate between 23 | // multiple servers. 24 | type HTTPTransporter struct { 25 | DisableKeepAlives bool 26 | prefix string 27 | appendEntriesPath string 28 | requestVotePath string 29 | snapshotPath string 30 | snapshotRecoveryPath string 31 | httpClient http.Client 32 | Transport *http.Transport 33 | } 34 | 35 | type HTTPMuxer interface { 36 | HandleFunc(string, func(http.ResponseWriter, *http.Request)) 37 | } 38 | 39 | //------------------------------------------------------------------------------ 40 | // 41 | // Constructor 42 | // 43 | //------------------------------------------------------------------------------ 44 | 45 | // Creates a new HTTP transporter with the given path prefix. 46 | func NewHTTPTransporter(prefix string, timeout time.Duration) *HTTPTransporter { 47 | t := &HTTPTransporter{ 48 | DisableKeepAlives: false, 49 | prefix: prefix, 50 | appendEntriesPath: joinPath(prefix, "/appendEntries"), 51 | requestVotePath: joinPath(prefix, "/requestVote"), 52 | snapshotPath: joinPath(prefix, "/snapshot"), 53 | snapshotRecoveryPath: joinPath(prefix, "/snapshotRecovery"), 54 | Transport: &http.Transport{DisableKeepAlives: false}, 55 | } 56 | t.httpClient.Transport = t.Transport 57 | t.Transport.ResponseHeaderTimeout = timeout 58 | return t 59 | } 60 | 61 | //------------------------------------------------------------------------------ 62 | // 63 | // Accessors 64 | // 65 | //------------------------------------------------------------------------------ 66 | 67 | // Retrieves the path prefix used by the transporter. 68 | func (t *HTTPTransporter) Prefix() string { 69 | return t.prefix 70 | } 71 | 72 | // Retrieves the AppendEntries path. 73 | func (t *HTTPTransporter) AppendEntriesPath() string { 74 | return t.appendEntriesPath 75 | } 76 | 77 | // Retrieves the RequestVote path. 78 | func (t *HTTPTransporter) RequestVotePath() string { 79 | return t.requestVotePath 80 | } 81 | 82 | // Retrieves the Snapshot path. 83 | func (t *HTTPTransporter) SnapshotPath() string { 84 | return t.snapshotPath 85 | } 86 | 87 | // Retrieves the SnapshotRecovery path. 88 | func (t *HTTPTransporter) SnapshotRecoveryPath() string { 89 | return t.snapshotRecoveryPath 90 | } 91 | 92 | //------------------------------------------------------------------------------ 93 | // 94 | // Methods 95 | // 96 | //------------------------------------------------------------------------------ 97 | 98 | //-------------------------------------- 99 | // Installation 100 | //-------------------------------------- 101 | 102 | // Applies Raft routes to an HTTP router for a given server. 103 | func (t *HTTPTransporter) Install(server Server, mux HTTPMuxer) { 104 | mux.HandleFunc(t.AppendEntriesPath(), t.appendEntriesHandler(server)) 105 | mux.HandleFunc(t.RequestVotePath(), t.requestVoteHandler(server)) 106 | mux.HandleFunc(t.SnapshotPath(), t.snapshotHandler(server)) 107 | mux.HandleFunc(t.SnapshotRecoveryPath(), t.snapshotRecoveryHandler(server)) 108 | } 109 | 110 | //-------------------------------------- 111 | // Outgoing 112 | //-------------------------------------- 113 | 114 | // Sends an AppendEntries RPC to a peer. 115 | func (t *HTTPTransporter) SendAppendEntriesRequest(server Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse { 116 | var b bytes.Buffer 117 | if _, err := req.Encode(&b); err != nil { 118 | traceln("transporter.ae.encoding.error:", err) 119 | return nil 120 | } 121 | 122 | url := joinPath(peer.ConnectionString, t.AppendEntriesPath()) 123 | traceln(server.Name(), "POST", url) 124 | 125 | httpResp, err := t.httpClient.Post(url, "application/protobuf", &b) 126 | if httpResp == nil || err != nil { 127 | traceln("transporter.ae.response.error:", err) 128 | return nil 129 | } 130 | defer httpResp.Body.Close() 131 | 132 | resp := &AppendEntriesResponse{} 133 | if _, err = resp.Decode(httpResp.Body); err != nil && err != io.EOF { 134 | traceln("transporter.ae.decoding.error:", err) 135 | return nil 136 | } 137 | 138 | return resp 139 | } 140 | 141 | // Sends a RequestVote RPC to a peer. 142 | func (t *HTTPTransporter) SendVoteRequest(server Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse { 143 | var b bytes.Buffer 144 | if _, err := req.Encode(&b); err != nil { 145 | traceln("transporter.rv.encoding.error:", err) 146 | return nil 147 | } 148 | 149 | url := fmt.Sprintf("%s%s", peer.ConnectionString, t.RequestVotePath()) 150 | traceln(server.Name(), "POST", url) 151 | 152 | httpResp, err := t.httpClient.Post(url, "application/protobuf", &b) 153 | if httpResp == nil || err != nil { 154 | traceln("transporter.rv.response.error:", err) 155 | return nil 156 | } 157 | defer httpResp.Body.Close() 158 | 159 | resp := &RequestVoteResponse{} 160 | if _, err = resp.Decode(httpResp.Body); err != nil && err != io.EOF { 161 | traceln("transporter.rv.decoding.error:", err) 162 | return nil 163 | } 164 | 165 | return resp 166 | } 167 | 168 | func joinPath(connectionString, thePath string) string { 169 | u, err := url.Parse(connectionString) 170 | if err != nil { 171 | panic(err) 172 | } 173 | u.Path = path.Join(u.Path, thePath) 174 | return u.String() 175 | } 176 | 177 | // Sends a SnapshotRequest RPC to a peer. 178 | func (t *HTTPTransporter) SendSnapshotRequest(server Server, peer *Peer, req *SnapshotRequest) *SnapshotResponse { 179 | var b bytes.Buffer 180 | if _, err := req.Encode(&b); err != nil { 181 | traceln("transporter.rv.encoding.error:", err) 182 | return nil 183 | } 184 | 185 | url := joinPath(peer.ConnectionString, t.snapshotPath) 186 | traceln(server.Name(), "POST", url) 187 | 188 | httpResp, err := t.httpClient.Post(url, "application/protobuf", &b) 189 | if httpResp == nil || err != nil { 190 | traceln("transporter.rv.response.error:", err) 191 | return nil 192 | } 193 | defer httpResp.Body.Close() 194 | 195 | resp := &SnapshotResponse{} 196 | if _, err = resp.Decode(httpResp.Body); err != nil && err != io.EOF { 197 | traceln("transporter.rv.decoding.error:", err) 198 | return nil 199 | } 200 | 201 | return resp 202 | } 203 | 204 | // Sends a SnapshotRequest RPC to a peer. 205 | func (t *HTTPTransporter) SendSnapshotRecoveryRequest(server Server, peer *Peer, req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse { 206 | var b bytes.Buffer 207 | if _, err := req.Encode(&b); err != nil { 208 | traceln("transporter.rv.encoding.error:", err) 209 | return nil 210 | } 211 | 212 | url := joinPath(peer.ConnectionString, t.snapshotRecoveryPath) 213 | traceln(server.Name(), "POST", url) 214 | 215 | httpResp, err := t.httpClient.Post(url, "application/protobuf", &b) 216 | if httpResp == nil || err != nil { 217 | traceln("transporter.rv.response.error:", err) 218 | return nil 219 | } 220 | defer httpResp.Body.Close() 221 | 222 | resp := &SnapshotRecoveryResponse{} 223 | if _, err = resp.Decode(httpResp.Body); err != nil && err != io.EOF { 224 | traceln("transporter.rv.decoding.error:", err) 225 | return nil 226 | } 227 | 228 | return resp 229 | } 230 | 231 | //-------------------------------------- 232 | // Incoming 233 | //-------------------------------------- 234 | 235 | // Handles incoming AppendEntries requests. 236 | func (t *HTTPTransporter) appendEntriesHandler(server Server) http.HandlerFunc { 237 | return func(w http.ResponseWriter, r *http.Request) { 238 | traceln(server.Name(), "RECV /appendEntries") 239 | 240 | req := &AppendEntriesRequest{} 241 | if _, err := req.Decode(r.Body); err != nil { 242 | http.Error(w, "", http.StatusBadRequest) 243 | return 244 | } 245 | 246 | resp := server.AppendEntries(req) 247 | if resp == nil { 248 | http.Error(w, "Failed creating response.", http.StatusInternalServerError) 249 | return 250 | } 251 | if _, err := resp.Encode(w); err != nil { 252 | http.Error(w, "", http.StatusInternalServerError) 253 | return 254 | } 255 | } 256 | } 257 | 258 | // Handles incoming RequestVote requests. 259 | func (t *HTTPTransporter) requestVoteHandler(server Server) http.HandlerFunc { 260 | return func(w http.ResponseWriter, r *http.Request) { 261 | traceln(server.Name(), "RECV /requestVote") 262 | 263 | req := &RequestVoteRequest{} 264 | if _, err := req.Decode(r.Body); err != nil { 265 | http.Error(w, "", http.StatusBadRequest) 266 | return 267 | } 268 | 269 | resp := server.RequestVote(req) 270 | if resp == nil { 271 | http.Error(w, "Failed creating response.", http.StatusInternalServerError) 272 | return 273 | } 274 | if _, err := resp.Encode(w); err != nil { 275 | http.Error(w, "", http.StatusInternalServerError) 276 | return 277 | } 278 | } 279 | } 280 | 281 | // Handles incoming Snapshot requests. 282 | func (t *HTTPTransporter) snapshotHandler(server Server) http.HandlerFunc { 283 | return func(w http.ResponseWriter, r *http.Request) { 284 | traceln(server.Name(), "RECV /snapshot") 285 | 286 | req := &SnapshotRequest{} 287 | if _, err := req.Decode(r.Body); err != nil { 288 | http.Error(w, "", http.StatusBadRequest) 289 | return 290 | } 291 | 292 | resp := server.RequestSnapshot(req) 293 | if resp == nil { 294 | http.Error(w, "Failed creating response.", http.StatusInternalServerError) 295 | return 296 | } 297 | if _, err := resp.Encode(w); err != nil { 298 | http.Error(w, "", http.StatusInternalServerError) 299 | return 300 | } 301 | } 302 | } 303 | 304 | // Handles incoming SnapshotRecovery requests. 305 | func (t *HTTPTransporter) snapshotRecoveryHandler(server Server) http.HandlerFunc { 306 | return func(w http.ResponseWriter, r *http.Request) { 307 | traceln(server.Name(), "RECV /snapshotRecovery") 308 | 309 | req := &SnapshotRecoveryRequest{} 310 | if _, err := req.Decode(r.Body); err != nil { 311 | http.Error(w, "", http.StatusBadRequest) 312 | return 313 | } 314 | 315 | resp := server.SnapshotRecoveryRequest(req) 316 | if resp == nil { 317 | http.Error(w, "Failed creating response.", http.StatusInternalServerError) 318 | return 319 | } 320 | if _, err := resp.Encode(w); err != nil { 321 | http.Error(w, "", http.StatusInternalServerError) 322 | return 323 | } 324 | } 325 | } 326 | -------------------------------------------------------------------------------- /http_transporter_test.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "fmt" 5 | "net" 6 | "net/http" 7 | "sync" 8 | "testing" 9 | "time" 10 | ) 11 | 12 | // Ensure that we can start several servers and have them communicate. 13 | func TestHTTPTransporter(t *testing.T) { 14 | transporter := NewHTTPTransporter("/raft", testElectionTimeout) 15 | transporter.DisableKeepAlives = true 16 | 17 | servers := []Server{} 18 | f0 := func(server Server, httpServer *http.Server) { 19 | // Stop the leader and wait for an election. 20 | server.Stop() 21 | time.Sleep(testElectionTimeout * 2) 22 | 23 | if servers[1].State() != Leader && servers[2].State() != Leader { 24 | t.Fatal("Expected re-election:", servers[1].State(), servers[2].State()) 25 | } 26 | server.Start() 27 | } 28 | f1 := func(server Server, httpServer *http.Server) { 29 | } 30 | f2 := func(server Server, httpServer *http.Server) { 31 | } 32 | runTestHttpServers(t, &servers, transporter, f0, f1, f2) 33 | } 34 | 35 | // Starts multiple independent Raft servers wrapped with HTTP servers. 36 | func runTestHttpServers(t *testing.T, servers *[]Server, transporter *HTTPTransporter, callbacks ...func(Server, *http.Server)) { 37 | var wg sync.WaitGroup 38 | httpServers := []*http.Server{} 39 | listeners := []net.Listener{} 40 | for i := range callbacks { 41 | wg.Add(1) 42 | port := 9000 + i 43 | 44 | // Create raft server. 45 | server := newTestServer(fmt.Sprintf("localhost:%d", port), transporter) 46 | server.SetHeartbeatInterval(testHeartbeatInterval) 47 | server.SetElectionTimeout(testElectionTimeout) 48 | server.Start() 49 | 50 | defer server.Stop() 51 | *servers = append(*servers, server) 52 | 53 | // Create listener for HTTP server and start it. 54 | listener, err := net.Listen("tcp", fmt.Sprintf(":%d", port)) 55 | if err != nil { 56 | panic(err) 57 | } 58 | defer listener.Close() 59 | listeners = append(listeners, listener) 60 | 61 | // Create wrapping HTTP server. 62 | mux := http.NewServeMux() 63 | transporter.Install(server, mux) 64 | httpServer := &http.Server{Addr: fmt.Sprintf(":%d", port), Handler: mux} 65 | httpServers = append(httpServers, httpServer) 66 | go func() { httpServer.Serve(listener) }() 67 | } 68 | 69 | // Setup configuration. 70 | for _, server := range *servers { 71 | if _, err := (*servers)[0].Do(&DefaultJoinCommand{Name: server.Name(), ConnectionString: fmt.Sprintf("http://%s", server.Name())}); err != nil { 72 | t.Fatalf("Server %s unable to join: %v", server.Name(), err) 73 | } 74 | } 75 | 76 | // Wait for configuration to propagate. 77 | time.Sleep(testHeartbeatInterval * 2) 78 | 79 | // Execute all the callbacks at the same time. 80 | for _i, _f := range callbacks { 81 | i, f := _i, _f 82 | go func() { 83 | defer wg.Done() 84 | f((*servers)[i], httpServers[i]) 85 | }() 86 | } 87 | 88 | // Wait until everything is done. 89 | wg.Wait() 90 | } 91 | 92 | func BenchmarkSpeed(b *testing.B) { 93 | 94 | transporter := NewHTTPTransporter("/raft", testElectionTimeout) 95 | transporter.DisableKeepAlives = true 96 | 97 | servers := []Server{} 98 | 99 | for i := 0; i < 3; i++ { 100 | port := 9000 + i 101 | 102 | // Create raft server. 103 | server := newTestServer(fmt.Sprintf("localhost:%d", port), transporter) 104 | server.SetHeartbeatInterval(testHeartbeatInterval) 105 | server.SetElectionTimeout(testElectionTimeout) 106 | server.Start() 107 | 108 | defer server.Stop() 109 | servers = append(servers, server) 110 | 111 | // Create listener for HTTP server and start it. 112 | listener, err := net.Listen("tcp", fmt.Sprintf(":%d", port)) 113 | if err != nil { 114 | panic(err) 115 | } 116 | defer listener.Close() 117 | 118 | // Create wrapping HTTP server. 119 | mux := http.NewServeMux() 120 | transporter.Install(server, mux) 121 | httpServer := &http.Server{Addr: fmt.Sprintf(":%d", port), Handler: mux} 122 | 123 | go func() { httpServer.Serve(listener) }() 124 | } 125 | 126 | // Setup configuration. 127 | for _, server := range servers { 128 | (servers)[0].Do(&DefaultJoinCommand{Name: server.Name(), ConnectionString: fmt.Sprintf("http://%s", server.Name())}) 129 | } 130 | 131 | c := make(chan bool) 132 | 133 | // Wait for configuration to propagate. 134 | time.Sleep(testHeartbeatInterval * 2) 135 | 136 | b.ResetTimer() 137 | for n := 0; n < b.N; n++ { 138 | for i := 0; i < 1000; i++ { 139 | go send(c, servers[0]) 140 | } 141 | 142 | for i := 0; i < 1000; i++ { 143 | <-c 144 | } 145 | } 146 | } 147 | 148 | func send(c chan bool, s Server) { 149 | for i := 0; i < 20; i++ { 150 | s.Do(&NOPCommand{}) 151 | } 152 | c <- true 153 | } 154 | -------------------------------------------------------------------------------- /log.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "bufio" 5 | "errors" 6 | "fmt" 7 | "io" 8 | "os" 9 | "sync" 10 | 11 | "github.com/goraft/raft/protobuf" 12 | ) 13 | 14 | //------------------------------------------------------------------------------ 15 | // 16 | // Typedefs 17 | // 18 | //------------------------------------------------------------------------------ 19 | 20 | // A log is a collection of log entries that are persisted to durable storage. 21 | type Log struct { 22 | ApplyFunc func(*LogEntry, Command) (interface{}, error) 23 | file *os.File 24 | path string 25 | entries []*LogEntry 26 | commitIndex uint64 27 | mutex sync.RWMutex 28 | startIndex uint64 // the index before the first entry in the Log entries 29 | startTerm uint64 30 | initialized bool 31 | } 32 | 33 | // The results of the applying a log entry. 34 | type logResult struct { 35 | returnValue interface{} 36 | err error 37 | } 38 | 39 | //------------------------------------------------------------------------------ 40 | // 41 | // Constructor 42 | // 43 | //------------------------------------------------------------------------------ 44 | 45 | // Creates a new log. 46 | func newLog() *Log { 47 | return &Log{ 48 | entries: make([]*LogEntry, 0), 49 | } 50 | } 51 | 52 | //------------------------------------------------------------------------------ 53 | // 54 | // Accessors 55 | // 56 | //------------------------------------------------------------------------------ 57 | 58 | //-------------------------------------- 59 | // Log Indices 60 | //-------------------------------------- 61 | 62 | // The last committed index in the log. 63 | func (l *Log) CommitIndex() uint64 { 64 | l.mutex.RLock() 65 | defer l.mutex.RUnlock() 66 | return l.commitIndex 67 | } 68 | 69 | // The current index in the log. 70 | func (l *Log) currentIndex() uint64 { 71 | l.mutex.RLock() 72 | defer l.mutex.RUnlock() 73 | return l.internalCurrentIndex() 74 | } 75 | 76 | // The current index in the log without locking 77 | func (l *Log) internalCurrentIndex() uint64 { 78 | if len(l.entries) == 0 { 79 | return l.startIndex 80 | } 81 | return l.entries[len(l.entries)-1].Index() 82 | } 83 | 84 | // The next index in the log. 85 | func (l *Log) nextIndex() uint64 { 86 | return l.currentIndex() + 1 87 | } 88 | 89 | // Determines if the log contains zero entries. 90 | func (l *Log) isEmpty() bool { 91 | l.mutex.RLock() 92 | defer l.mutex.RUnlock() 93 | return (len(l.entries) == 0) && (l.startIndex == 0) 94 | } 95 | 96 | // The name of the last command in the log. 97 | func (l *Log) lastCommandName() string { 98 | l.mutex.RLock() 99 | defer l.mutex.RUnlock() 100 | if len(l.entries) > 0 { 101 | if entry := l.entries[len(l.entries)-1]; entry != nil { 102 | return entry.CommandName() 103 | } 104 | } 105 | return "" 106 | } 107 | 108 | //-------------------------------------- 109 | // Log Terms 110 | //-------------------------------------- 111 | 112 | // The current term in the log. 113 | func (l *Log) currentTerm() uint64 { 114 | l.mutex.RLock() 115 | defer l.mutex.RUnlock() 116 | 117 | if len(l.entries) == 0 { 118 | return l.startTerm 119 | } 120 | return l.entries[len(l.entries)-1].Term() 121 | } 122 | 123 | //------------------------------------------------------------------------------ 124 | // 125 | // Methods 126 | // 127 | //------------------------------------------------------------------------------ 128 | 129 | //-------------------------------------- 130 | // State 131 | //-------------------------------------- 132 | 133 | // Opens the log file and reads existing entries. The log can remain open and 134 | // continue to append entries to the end of the log. 135 | func (l *Log) open(path string) error { 136 | // Read all the entries from the log if one exists. 137 | var readBytes int64 138 | 139 | var err error 140 | debugln("log.open.open ", path) 141 | // open log file 142 | l.file, err = os.OpenFile(path, os.O_RDWR, 0600) 143 | l.path = path 144 | 145 | if err != nil { 146 | // if the log file does not exist before 147 | // we create the log file and set commitIndex to 0 148 | if os.IsNotExist(err) { 149 | l.file, err = os.OpenFile(path, os.O_WRONLY|os.O_CREATE, 0600) 150 | debugln("log.open.create ", path) 151 | if err == nil { 152 | l.initialized = true 153 | } 154 | return err 155 | } 156 | return err 157 | } 158 | debugln("log.open.exist ", path) 159 | 160 | // Read the file and decode entries. 161 | for { 162 | // Instantiate log entry and decode into it. 163 | entry, _ := newLogEntry(l, nil, 0, 0, nil) 164 | entry.Position, _ = l.file.Seek(0, os.SEEK_CUR) 165 | 166 | n, err := entry.Decode(l.file) 167 | if err != nil { 168 | if err == io.EOF { 169 | debugln("open.log.append: finish ") 170 | } else { 171 | if err = os.Truncate(path, readBytes); err != nil { 172 | return fmt.Errorf("raft.Log: Unable to recover: %v", err) 173 | } 174 | } 175 | break 176 | } 177 | if entry.Index() > l.startIndex { 178 | // Append entry. 179 | l.entries = append(l.entries, entry) 180 | if entry.Index() <= l.commitIndex { 181 | command, err := newCommand(entry.CommandName(), entry.Command()) 182 | if err != nil { 183 | continue 184 | } 185 | l.ApplyFunc(entry, command) 186 | } 187 | debugln("open.log.append log index ", entry.Index()) 188 | } 189 | 190 | readBytes += int64(n) 191 | } 192 | debugln("open.log.recovery number of log ", len(l.entries)) 193 | l.initialized = true 194 | return nil 195 | } 196 | 197 | // Closes the log file. 198 | func (l *Log) close() { 199 | l.mutex.Lock() 200 | defer l.mutex.Unlock() 201 | 202 | if l.file != nil { 203 | l.file.Close() 204 | l.file = nil 205 | } 206 | l.entries = make([]*LogEntry, 0) 207 | } 208 | 209 | // sync to disk 210 | func (l *Log) sync() error { 211 | return l.file.Sync() 212 | } 213 | 214 | //-------------------------------------- 215 | // Entries 216 | //-------------------------------------- 217 | 218 | // Creates a log entry associated with this log. 219 | func (l *Log) createEntry(term uint64, command Command, e *ev) (*LogEntry, error) { 220 | return newLogEntry(l, e, l.nextIndex(), term, command) 221 | } 222 | 223 | // Retrieves an entry from the log. If the entry has been eliminated because 224 | // of a snapshot then nil is returned. 225 | func (l *Log) getEntry(index uint64) *LogEntry { 226 | l.mutex.RLock() 227 | defer l.mutex.RUnlock() 228 | 229 | if index <= l.startIndex || index > (l.startIndex+uint64(len(l.entries))) { 230 | return nil 231 | } 232 | return l.entries[index-l.startIndex-1] 233 | } 234 | 235 | // Checks if the log contains a given index/term combination. 236 | func (l *Log) containsEntry(index uint64, term uint64) bool { 237 | entry := l.getEntry(index) 238 | return (entry != nil && entry.Term() == term) 239 | } 240 | 241 | // Retrieves a list of entries after a given index as well as the term of the 242 | // index provided. A nil list of entries is returned if the index no longer 243 | // exists because a snapshot was made. 244 | func (l *Log) getEntriesAfter(index uint64, maxLogEntriesPerRequest uint64) ([]*LogEntry, uint64) { 245 | l.mutex.RLock() 246 | defer l.mutex.RUnlock() 247 | 248 | // Return nil if index is before the start of the log. 249 | if index < l.startIndex { 250 | traceln("log.entriesAfter.before: ", index, " ", l.startIndex) 251 | return nil, 0 252 | } 253 | 254 | // Return an error if the index doesn't exist. 255 | if index > (uint64(len(l.entries)) + l.startIndex) { 256 | panic(fmt.Sprintf("raft: Index is beyond end of log: %v %v", len(l.entries), index)) 257 | } 258 | 259 | // If we're going from the beginning of the log then return the whole log. 260 | if index == l.startIndex { 261 | traceln("log.entriesAfter.beginning: ", index, " ", l.startIndex) 262 | return l.entries, l.startTerm 263 | } 264 | 265 | traceln("log.entriesAfter.partial: ", index, " ", l.entries[len(l.entries)-1].Index) 266 | 267 | entries := l.entries[index-l.startIndex:] 268 | length := len(entries) 269 | 270 | traceln("log.entriesAfter: startIndex:", l.startIndex, " length", len(l.entries)) 271 | 272 | if uint64(length) < maxLogEntriesPerRequest { 273 | // Determine the term at the given entry and return a subslice. 274 | return entries, l.entries[index-1-l.startIndex].Term() 275 | } else { 276 | return entries[:maxLogEntriesPerRequest], l.entries[index-1-l.startIndex].Term() 277 | } 278 | } 279 | 280 | //-------------------------------------- 281 | // Commit 282 | //-------------------------------------- 283 | 284 | // Retrieves the last index and term that has been committed to the log. 285 | func (l *Log) commitInfo() (index uint64, term uint64) { 286 | l.mutex.RLock() 287 | defer l.mutex.RUnlock() 288 | // If we don't have any committed entries then just return zeros. 289 | if l.commitIndex == 0 { 290 | return 0, 0 291 | } 292 | 293 | // No new commit log after snapshot 294 | if l.commitIndex == l.startIndex { 295 | return l.startIndex, l.startTerm 296 | } 297 | 298 | // Return the last index & term from the last committed entry. 299 | debugln("commitInfo.get.[", l.commitIndex, "/", l.startIndex, "]") 300 | entry := l.entries[l.commitIndex-1-l.startIndex] 301 | return entry.Index(), entry.Term() 302 | } 303 | 304 | // Retrieves the last index and term that has been appended to the log. 305 | func (l *Log) lastInfo() (index uint64, term uint64) { 306 | l.mutex.RLock() 307 | defer l.mutex.RUnlock() 308 | 309 | // If we don't have any entries then just return zeros. 310 | if len(l.entries) == 0 { 311 | return l.startIndex, l.startTerm 312 | } 313 | 314 | // Return the last index & term 315 | entry := l.entries[len(l.entries)-1] 316 | return entry.Index(), entry.Term() 317 | } 318 | 319 | // Updates the commit index 320 | func (l *Log) updateCommitIndex(index uint64) { 321 | l.mutex.Lock() 322 | defer l.mutex.Unlock() 323 | if index > l.commitIndex { 324 | l.commitIndex = index 325 | } 326 | debugln("update.commit.index ", index) 327 | } 328 | 329 | // Updates the commit index and writes entries after that index to the stable storage. 330 | func (l *Log) setCommitIndex(index uint64) error { 331 | l.mutex.Lock() 332 | defer l.mutex.Unlock() 333 | 334 | // this is not error any more after limited the number of sending entries 335 | // commit up to what we already have 336 | if index > l.startIndex+uint64(len(l.entries)) { 337 | debugln("raft.Log: Commit index", index, "set back to ", len(l.entries)) 338 | index = l.startIndex + uint64(len(l.entries)) 339 | } 340 | 341 | // Do not allow previous indices to be committed again. 342 | 343 | // This could happens, since the guarantee is that the new leader has up-to-dated 344 | // log entries rather than has most up-to-dated committed index 345 | 346 | // For example, Leader 1 send log 80 to follower 2 and follower 3 347 | // follower 2 and follow 3 all got the new entries and reply 348 | // leader 1 committed entry 80 and send reply to follower 2 and follower3 349 | // follower 2 receive the new committed index and update committed index to 80 350 | // leader 1 fail to send the committed index to follower 3 351 | // follower 3 promote to leader (server 1 and server 2 will vote, since leader 3 352 | // has up-to-dated the entries) 353 | // when new leader 3 send heartbeat with committed index = 0 to follower 2, 354 | // follower 2 should reply success and let leader 3 update the committed index to 80 355 | 356 | if index < l.commitIndex { 357 | return nil 358 | } 359 | 360 | // Find all entries whose index is between the previous index and the current index. 361 | for i := l.commitIndex + 1; i <= index; i++ { 362 | entryIndex := i - 1 - l.startIndex 363 | entry := l.entries[entryIndex] 364 | 365 | // Update commit index. 366 | l.commitIndex = entry.Index() 367 | 368 | // Decode the command. 369 | command, err := newCommand(entry.CommandName(), entry.Command()) 370 | if err != nil { 371 | return err 372 | } 373 | 374 | // Apply the changes to the state machine and store the error code. 375 | returnValue, err := l.ApplyFunc(entry, command) 376 | 377 | debugf("setCommitIndex.set.result index: %v, entries index: %v", i, entryIndex) 378 | if entry.event != nil { 379 | entry.event.returnValue = returnValue 380 | entry.event.c <- err 381 | } 382 | 383 | _, isJoinCommand := command.(JoinCommand) 384 | 385 | // we can only commit up to the most recent join command 386 | // if there is a join in this batch of commands. 387 | // after this commit, we need to recalculate the majority. 388 | if isJoinCommand { 389 | return nil 390 | } 391 | } 392 | return nil 393 | } 394 | 395 | // Set the commitIndex at the head of the log file to the current 396 | // commit Index. This should be called after obtained a log lock 397 | func (l *Log) flushCommitIndex() { 398 | l.file.Seek(0, os.SEEK_SET) 399 | fmt.Fprintf(l.file, "%8x\n", l.commitIndex) 400 | l.file.Seek(0, os.SEEK_END) 401 | } 402 | 403 | //-------------------------------------- 404 | // Truncation 405 | //-------------------------------------- 406 | 407 | // Truncates the log to the given index and term. This only works if the log 408 | // at the index has not been committed. 409 | func (l *Log) truncate(index uint64, term uint64) error { 410 | l.mutex.Lock() 411 | defer l.mutex.Unlock() 412 | debugln("log.truncate: ", index) 413 | 414 | // Do not allow committed entries to be truncated. 415 | if index < l.commitIndex { 416 | debugln("log.truncate.before") 417 | return fmt.Errorf("raft.Log: Index is already committed (%v): (IDX=%v, TERM=%v)", l.commitIndex, index, term) 418 | } 419 | 420 | // Do not truncate past end of entries. 421 | if index > l.startIndex+uint64(len(l.entries)) { 422 | debugln("log.truncate.after") 423 | return fmt.Errorf("raft.Log: Entry index does not exist (MAX=%v): (IDX=%v, TERM=%v)", len(l.entries), index, term) 424 | } 425 | 426 | // If we're truncating everything then just clear the entries. 427 | if index == l.startIndex { 428 | debugln("log.truncate.clear") 429 | l.file.Truncate(0) 430 | l.file.Seek(0, os.SEEK_SET) 431 | 432 | // notify clients if this node is the previous leader 433 | for _, entry := range l.entries { 434 | if entry.event != nil { 435 | entry.event.c <- errors.New("command failed to be committed due to node failure") 436 | } 437 | } 438 | 439 | l.entries = []*LogEntry{} 440 | } else { 441 | // Do not truncate if the entry at index does not have the matching term. 442 | entry := l.entries[index-l.startIndex-1] 443 | if len(l.entries) > 0 && entry.Term() != term { 444 | debugln("log.truncate.termMismatch") 445 | return fmt.Errorf("raft.Log: Entry at index does not have matching term (%v): (IDX=%v, TERM=%v)", entry.Term(), index, term) 446 | } 447 | 448 | // Otherwise truncate up to the desired entry. 449 | if index < l.startIndex+uint64(len(l.entries)) { 450 | debugln("log.truncate.finish") 451 | position := l.entries[index-l.startIndex].Position 452 | l.file.Truncate(position) 453 | l.file.Seek(position, os.SEEK_SET) 454 | 455 | // notify clients if this node is the previous leader 456 | for i := index - l.startIndex; i < uint64(len(l.entries)); i++ { 457 | entry := l.entries[i] 458 | if entry.event != nil { 459 | entry.event.c <- errors.New("command failed to be committed due to node failure") 460 | } 461 | } 462 | 463 | l.entries = l.entries[0 : index-l.startIndex] 464 | } 465 | } 466 | 467 | return nil 468 | } 469 | 470 | //-------------------------------------- 471 | // Append 472 | //-------------------------------------- 473 | 474 | // Appends a series of entries to the log. 475 | func (l *Log) appendEntries(entries []*protobuf.LogEntry) error { 476 | l.mutex.Lock() 477 | defer l.mutex.Unlock() 478 | 479 | startPosition, _ := l.file.Seek(0, os.SEEK_CUR) 480 | 481 | w := bufio.NewWriter(l.file) 482 | 483 | var size int64 484 | var err error 485 | // Append each entry but exit if we hit an error. 486 | for i := range entries { 487 | logEntry := &LogEntry{ 488 | log: l, 489 | Position: startPosition, 490 | pb: entries[i], 491 | } 492 | 493 | if size, err = l.writeEntry(logEntry, w); err != nil { 494 | return err 495 | } 496 | 497 | startPosition += size 498 | } 499 | w.Flush() 500 | err = l.sync() 501 | 502 | if err != nil { 503 | panic(err) 504 | } 505 | 506 | return nil 507 | } 508 | 509 | // Writes a single log entry to the end of the log. 510 | func (l *Log) appendEntry(entry *LogEntry) error { 511 | l.mutex.Lock() 512 | defer l.mutex.Unlock() 513 | 514 | if l.file == nil { 515 | return errors.New("raft.Log: Log is not open") 516 | } 517 | 518 | // Make sure the term and index are greater than the previous. 519 | if len(l.entries) > 0 { 520 | lastEntry := l.entries[len(l.entries)-1] 521 | if entry.Term() < lastEntry.Term() { 522 | return fmt.Errorf("raft.Log: Cannot append entry with earlier term (%x:%x <= %x:%x)", entry.Term(), entry.Index(), lastEntry.Term(), lastEntry.Index()) 523 | } else if entry.Term() == lastEntry.Term() && entry.Index() <= lastEntry.Index() { 524 | return fmt.Errorf("raft.Log: Cannot append entry with earlier index in the same term (%x:%x <= %x:%x)", entry.Term(), entry.Index(), lastEntry.Term(), lastEntry.Index()) 525 | } 526 | } 527 | 528 | position, _ := l.file.Seek(0, os.SEEK_CUR) 529 | 530 | entry.Position = position 531 | 532 | // Write to storage. 533 | if _, err := entry.Encode(l.file); err != nil { 534 | return err 535 | } 536 | 537 | // Append to entries list if stored on disk. 538 | l.entries = append(l.entries, entry) 539 | 540 | return nil 541 | } 542 | 543 | // appendEntry with Buffered io 544 | func (l *Log) writeEntry(entry *LogEntry, w io.Writer) (int64, error) { 545 | if l.file == nil { 546 | return -1, errors.New("raft.Log: Log is not open") 547 | } 548 | 549 | // Make sure the term and index are greater than the previous. 550 | if len(l.entries) > 0 { 551 | lastEntry := l.entries[len(l.entries)-1] 552 | if entry.Term() < lastEntry.Term() { 553 | return -1, fmt.Errorf("raft.Log: Cannot append entry with earlier term (%x:%x <= %x:%x)", entry.Term(), entry.Index(), lastEntry.Term(), lastEntry.Index()) 554 | } else if entry.Term() == lastEntry.Term() && entry.Index() <= lastEntry.Index() { 555 | return -1, fmt.Errorf("raft.Log: Cannot append entry with earlier index in the same term (%x:%x <= %x:%x)", entry.Term(), entry.Index(), lastEntry.Term(), lastEntry.Index()) 556 | } 557 | } 558 | 559 | // Write to storage. 560 | size, err := entry.Encode(w) 561 | if err != nil { 562 | return -1, err 563 | } 564 | 565 | // Append to entries list if stored on disk. 566 | l.entries = append(l.entries, entry) 567 | 568 | return int64(size), nil 569 | } 570 | 571 | //-------------------------------------- 572 | // Log compaction 573 | //-------------------------------------- 574 | 575 | // compact the log before index (including index) 576 | func (l *Log) compact(index uint64, term uint64) error { 577 | var entries []*LogEntry 578 | 579 | l.mutex.Lock() 580 | defer l.mutex.Unlock() 581 | 582 | if index == 0 { 583 | return nil 584 | } 585 | // nothing to compaction 586 | // the index may be greater than the current index if 587 | // we just recovery from on snapshot 588 | if index >= l.internalCurrentIndex() { 589 | entries = make([]*LogEntry, 0) 590 | } else { 591 | // get all log entries after index 592 | entries = l.entries[index-l.startIndex:] 593 | } 594 | 595 | // create a new log file and add all the entries 596 | new_file_path := l.path + ".new" 597 | file, err := os.OpenFile(new_file_path, os.O_APPEND|os.O_CREATE|os.O_WRONLY, 0600) 598 | if err != nil { 599 | return err 600 | } 601 | for _, entry := range entries { 602 | position, _ := l.file.Seek(0, os.SEEK_CUR) 603 | entry.Position = position 604 | 605 | if _, err = entry.Encode(file); err != nil { 606 | file.Close() 607 | os.Remove(new_file_path) 608 | return err 609 | } 610 | } 611 | file.Sync() 612 | 613 | old_file := l.file 614 | 615 | // rename the new log file 616 | err = os.Rename(new_file_path, l.path) 617 | if err != nil { 618 | file.Close() 619 | os.Remove(new_file_path) 620 | return err 621 | } 622 | l.file = file 623 | 624 | // close the old log file 625 | old_file.Close() 626 | 627 | // compaction the in memory log 628 | l.entries = entries 629 | l.startIndex = index 630 | l.startTerm = term 631 | return nil 632 | } 633 | -------------------------------------------------------------------------------- /log_entry.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "bytes" 5 | "encoding/json" 6 | "fmt" 7 | "io" 8 | 9 | "code.google.com/p/gogoprotobuf/proto" 10 | "github.com/goraft/raft/protobuf" 11 | ) 12 | 13 | // A log entry stores a single item in the log. 14 | type LogEntry struct { 15 | pb *protobuf.LogEntry 16 | Position int64 // position in the log file 17 | log *Log 18 | event *ev 19 | } 20 | 21 | // Creates a new log entry associated with a log. 22 | func newLogEntry(log *Log, event *ev, index uint64, term uint64, command Command) (*LogEntry, error) { 23 | var buf bytes.Buffer 24 | var commandName string 25 | if command != nil { 26 | commandName = command.CommandName() 27 | if encoder, ok := command.(CommandEncoder); ok { 28 | if err := encoder.Encode(&buf); err != nil { 29 | return nil, err 30 | } 31 | } else { 32 | if err := json.NewEncoder(&buf).Encode(command); err != nil { 33 | return nil, err 34 | } 35 | } 36 | } 37 | 38 | pb := &protobuf.LogEntry{ 39 | Index: proto.Uint64(index), 40 | Term: proto.Uint64(term), 41 | CommandName: proto.String(commandName), 42 | Command: buf.Bytes(), 43 | } 44 | 45 | e := &LogEntry{ 46 | pb: pb, 47 | log: log, 48 | event: event, 49 | } 50 | 51 | return e, nil 52 | } 53 | 54 | func (e *LogEntry) Index() uint64 { 55 | return e.pb.GetIndex() 56 | } 57 | 58 | func (e *LogEntry) Term() uint64 { 59 | return e.pb.GetTerm() 60 | } 61 | 62 | func (e *LogEntry) CommandName() string { 63 | return e.pb.GetCommandName() 64 | } 65 | 66 | func (e *LogEntry) Command() []byte { 67 | return e.pb.GetCommand() 68 | } 69 | 70 | // Encodes the log entry to a buffer. Returns the number of bytes 71 | // written and any error that may have occurred. 72 | func (e *LogEntry) Encode(w io.Writer) (int, error) { 73 | b, err := proto.Marshal(e.pb) 74 | if err != nil { 75 | return -1, err 76 | } 77 | 78 | if _, err = fmt.Fprintf(w, "%8x\n", len(b)); err != nil { 79 | return -1, err 80 | } 81 | 82 | return w.Write(b) 83 | } 84 | 85 | // Decodes the log entry from a buffer. Returns the number of bytes read and 86 | // any error that occurs. 87 | func (e *LogEntry) Decode(r io.Reader) (int, error) { 88 | 89 | var length int 90 | _, err := fmt.Fscanf(r, "%8x\n", &length) 91 | if err != nil { 92 | return -1, err 93 | } 94 | 95 | data := make([]byte, length) 96 | _, err = io.ReadFull(r, data) 97 | 98 | if err != nil { 99 | return -1, err 100 | } 101 | 102 | if err = proto.Unmarshal(data, e.pb); err != nil { 103 | return -1, err 104 | } 105 | 106 | return length + 8 + 1, nil 107 | } 108 | -------------------------------------------------------------------------------- /log_test.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "io/ioutil" 5 | "os" 6 | "reflect" 7 | "testing" 8 | ) 9 | 10 | //------------------------------------------------------------------------------ 11 | // 12 | // Tests 13 | // 14 | //------------------------------------------------------------------------------ 15 | 16 | //-------------------------------------- 17 | // Append 18 | //-------------------------------------- 19 | 20 | // Ensure that we can append to a new log. 21 | func TestLogNewLog(t *testing.T) { 22 | path := getLogPath() 23 | log := newLog() 24 | log.ApplyFunc = func(e *LogEntry, c Command) (interface{}, error) { 25 | return nil, nil 26 | } 27 | if err := log.open(path); err != nil { 28 | t.Fatalf("Unable to open log: %v", err) 29 | } 30 | defer log.close() 31 | defer os.Remove(path) 32 | 33 | e, _ := newLogEntry(log, nil, 1, 1, &testCommand1{Val: "foo", I: 20}) 34 | if err := log.appendEntry(e); err != nil { 35 | t.Fatalf("Unable to append: %v", err) 36 | } 37 | e, _ = newLogEntry(log, nil, 2, 1, &testCommand2{X: 100}) 38 | if err := log.appendEntry(e); err != nil { 39 | t.Fatalf("Unable to append: %v", err) 40 | } 41 | e, _ = newLogEntry(log, nil, 3, 2, &testCommand1{Val: "bar", I: 0}) 42 | if err := log.appendEntry(e); err != nil { 43 | t.Fatalf("Unable to append: %v", err) 44 | } 45 | 46 | // Partial commit. 47 | if err := log.setCommitIndex(2); err != nil { 48 | t.Fatalf("Unable to partially commit: %v", err) 49 | } 50 | if index, term := log.commitInfo(); index != 2 || term != 1 { 51 | t.Fatalf("Invalid commit info [IDX=%v, TERM=%v]", index, term) 52 | } 53 | 54 | // Full commit. 55 | if err := log.setCommitIndex(3); err != nil { 56 | t.Fatalf("Unable to commit: %v", err) 57 | } 58 | if index, term := log.commitInfo(); index != 3 || term != 2 { 59 | t.Fatalf("Invalid commit info [IDX=%v, TERM=%v]", index, term) 60 | } 61 | } 62 | 63 | // Ensure that we can decode and encode to an existing log. 64 | func TestLogExistingLog(t *testing.T) { 65 | tmpLog := newLog() 66 | e0, _ := newLogEntry(tmpLog, nil, 1, 1, &testCommand1{Val: "foo", I: 20}) 67 | e1, _ := newLogEntry(tmpLog, nil, 2, 1, &testCommand2{X: 100}) 68 | e2, _ := newLogEntry(tmpLog, nil, 3, 2, &testCommand1{Val: "bar", I: 0}) 69 | log, path := setupLog([]*LogEntry{e0, e1, e2}) 70 | defer log.close() 71 | defer os.Remove(path) 72 | 73 | // Validate existing log entries. 74 | if len(log.entries) != 3 { 75 | t.Fatalf("Expected 3 entries, got %d", len(log.entries)) 76 | } 77 | if log.entries[0].Index() != 1 || log.entries[0].Term() != 1 { 78 | t.Fatalf("Unexpected entry[0]: %v", log.entries[0]) 79 | } 80 | if log.entries[1].Index() != 2 || log.entries[1].Term() != 1 { 81 | t.Fatalf("Unexpected entry[1]: %v", log.entries[1]) 82 | } 83 | if log.entries[2].Index() != 3 || log.entries[2].Term() != 2 { 84 | t.Fatalf("Unexpected entry[2]: %v", log.entries[2]) 85 | } 86 | } 87 | 88 | // Ensure that we can check the contents of the log by index/term. 89 | func TestLogContainsEntries(t *testing.T) { 90 | tmpLog := newLog() 91 | e0, _ := newLogEntry(tmpLog, nil, 1, 1, &testCommand1{Val: "foo", I: 20}) 92 | e1, _ := newLogEntry(tmpLog, nil, 2, 1, &testCommand2{X: 100}) 93 | e2, _ := newLogEntry(tmpLog, nil, 3, 2, &testCommand1{Val: "bar", I: 0}) 94 | log, path := setupLog([]*LogEntry{e0, e1, e2}) 95 | defer log.close() 96 | defer os.Remove(path) 97 | 98 | if log.containsEntry(0, 0) { 99 | t.Fatalf("Zero-index entry should not exist in log.") 100 | } 101 | if log.containsEntry(1, 0) { 102 | t.Fatalf("Entry with mismatched term should not exist") 103 | } 104 | if log.containsEntry(4, 0) { 105 | t.Fatalf("Out-of-range entry should not exist") 106 | } 107 | if !log.containsEntry(2, 1) { 108 | t.Fatalf("Entry 2/1 should exist") 109 | } 110 | if !log.containsEntry(3, 2) { 111 | t.Fatalf("Entry 2/1 should exist") 112 | } 113 | } 114 | 115 | // Ensure that we can recover from an incomplete/corrupt log and continue logging. 116 | func TestLogRecovery(t *testing.T) { 117 | tmpLog := newLog() 118 | e0, _ := newLogEntry(tmpLog, nil, 1, 1, &testCommand1{Val: "foo", I: 20}) 119 | e1, _ := newLogEntry(tmpLog, nil, 2, 1, &testCommand2{X: 100}) 120 | f, _ := ioutil.TempFile("", "raft-log-") 121 | 122 | e0.Encode(f) 123 | e1.Encode(f) 124 | f.WriteString("CORRUPT!") 125 | f.Close() 126 | 127 | log := newLog() 128 | log.ApplyFunc = func(e *LogEntry, c Command) (interface{}, error) { 129 | return nil, nil 130 | } 131 | if err := log.open(f.Name()); err != nil { 132 | t.Fatalf("Unable to open log: %v", err) 133 | } 134 | defer log.close() 135 | defer os.Remove(f.Name()) 136 | 137 | e, _ := newLogEntry(log, nil, 3, 2, &testCommand1{Val: "bat", I: -5}) 138 | if err := log.appendEntry(e); err != nil { 139 | t.Fatalf("Unable to append: %v", err) 140 | } 141 | 142 | // Validate existing log entries. 143 | if len(log.entries) != 3 { 144 | t.Fatalf("Expected 3 entries, got %d", len(log.entries)) 145 | } 146 | if log.entries[0].Index() != 1 || log.entries[0].Term() != 1 { 147 | t.Fatalf("Unexpected entry[0]: %v", log.entries[0]) 148 | } 149 | if log.entries[1].Index() != 2 || log.entries[1].Term() != 1 { 150 | t.Fatalf("Unexpected entry[1]: %v", log.entries[1]) 151 | } 152 | if log.entries[2].Index() != 3 || log.entries[2].Term() != 2 { 153 | t.Fatalf("Unexpected entry[2]: %v", log.entries[2]) 154 | } 155 | } 156 | 157 | //-------------------------------------- 158 | // Append 159 | //-------------------------------------- 160 | 161 | // Ensure that we can truncate uncommitted entries in the log. 162 | func TestLogTruncate(t *testing.T) { 163 | log, path := setupLog(nil) 164 | if err := log.open(path); err != nil { 165 | t.Fatalf("Unable to open log: %v", err) 166 | } 167 | 168 | defer os.Remove(path) 169 | 170 | entry1, _ := newLogEntry(log, nil, 1, 1, &testCommand1{Val: "foo", I: 20}) 171 | if err := log.appendEntry(entry1); err != nil { 172 | t.Fatalf("Unable to append: %v", err) 173 | } 174 | entry2, _ := newLogEntry(log, nil, 2, 1, &testCommand2{X: 100}) 175 | if err := log.appendEntry(entry2); err != nil { 176 | t.Fatalf("Unable to append: %v", err) 177 | } 178 | entry3, _ := newLogEntry(log, nil, 3, 2, &testCommand1{Val: "bar", I: 0}) 179 | if err := log.appendEntry(entry3); err != nil { 180 | t.Fatalf("Unable to append: %v", err) 181 | } 182 | if err := log.setCommitIndex(2); err != nil { 183 | t.Fatalf("Unable to partially commit: %v", err) 184 | } 185 | 186 | // Truncate committed entry. 187 | if err := log.truncate(1, 1); err == nil || err.Error() != "raft.Log: Index is already committed (2): (IDX=1, TERM=1)" { 188 | t.Fatalf("Truncating committed entries shouldn't work: %v", err) 189 | } 190 | // Truncate past end of log. 191 | if err := log.truncate(4, 2); err == nil || err.Error() != "raft.Log: Entry index does not exist (MAX=3): (IDX=4, TERM=2)" { 192 | t.Fatalf("Truncating past end-of-log shouldn't work: %v", err) 193 | } 194 | // Truncate entry with mismatched term. 195 | if err := log.truncate(2, 2); err == nil || err.Error() != "raft.Log: Entry at index does not have matching term (1): (IDX=2, TERM=2)" { 196 | t.Fatalf("Truncating mismatched entries shouldn't work: %v", err) 197 | } 198 | // Truncate end of log. 199 | if err := log.truncate(3, 2); !(err == nil && reflect.DeepEqual(log.entries, []*LogEntry{entry1, entry2, entry3})) { 200 | t.Fatalf("Truncating end of log should work: %v\n\nEntries:\nActual: %v\nExpected: %v", err, log.entries, []*LogEntry{entry1, entry2, entry3}) 201 | } 202 | // Truncate at last commit. 203 | if err := log.truncate(2, 1); !(err == nil && reflect.DeepEqual(log.entries, []*LogEntry{entry1, entry2})) { 204 | t.Fatalf("Truncating at last commit should work: %v\n\nEntries:\nActual: %v\nExpected: %v", err, log.entries, []*LogEntry{entry1, entry2}) 205 | } 206 | 207 | // Append after truncate 208 | if err := log.appendEntry(entry3); err != nil { 209 | t.Fatalf("Unable to append after truncate: %v", err) 210 | } 211 | 212 | log.close() 213 | 214 | // Recovery the truncated log 215 | log = newLog() 216 | if err := log.open(path); err != nil { 217 | t.Fatalf("Unable to open log: %v", err) 218 | } 219 | // Validate existing log entries. 220 | if len(log.entries) != 3 { 221 | t.Fatalf("Expected 3 entries, got %d", len(log.entries)) 222 | } 223 | if log.entries[0].Index() != 1 || log.entries[0].Term() != 1 { 224 | t.Fatalf("Unexpected entry[0]: %v", log.entries[0]) 225 | } 226 | if log.entries[1].Index() != 2 || log.entries[1].Term() != 1 { 227 | t.Fatalf("Unexpected entry[1]: %v", log.entries[1]) 228 | } 229 | if log.entries[2].Index() != 3 || log.entries[2].Term() != 2 { 230 | t.Fatalf("Unexpected entry[2]: %v", log.entries[2]) 231 | } 232 | } 233 | -------------------------------------------------------------------------------- /peer.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "sync" 5 | "time" 6 | ) 7 | 8 | //------------------------------------------------------------------------------ 9 | // 10 | // Typedefs 11 | // 12 | //------------------------------------------------------------------------------ 13 | 14 | // A peer is a reference to another server involved in the consensus protocol. 15 | type Peer struct { 16 | server *server 17 | Name string `json:"name"` 18 | ConnectionString string `json:"connectionString"` 19 | prevLogIndex uint64 20 | stopChan chan bool 21 | heartbeatInterval time.Duration 22 | lastActivity time.Time 23 | sync.RWMutex 24 | } 25 | 26 | //------------------------------------------------------------------------------ 27 | // 28 | // Constructor 29 | // 30 | //------------------------------------------------------------------------------ 31 | 32 | // Creates a new peer. 33 | func newPeer(server *server, name string, connectionString string, heartbeatInterval time.Duration) *Peer { 34 | return &Peer{ 35 | server: server, 36 | Name: name, 37 | ConnectionString: connectionString, 38 | heartbeatInterval: heartbeatInterval, 39 | } 40 | } 41 | 42 | //------------------------------------------------------------------------------ 43 | // 44 | // Accessors 45 | // 46 | //------------------------------------------------------------------------------ 47 | 48 | // Sets the heartbeat timeout. 49 | func (p *Peer) setHeartbeatInterval(duration time.Duration) { 50 | p.heartbeatInterval = duration 51 | } 52 | 53 | //-------------------------------------- 54 | // Prev log index 55 | //-------------------------------------- 56 | 57 | // Retrieves the previous log index. 58 | func (p *Peer) getPrevLogIndex() uint64 { 59 | p.RLock() 60 | defer p.RUnlock() 61 | return p.prevLogIndex 62 | } 63 | 64 | // Sets the previous log index. 65 | func (p *Peer) setPrevLogIndex(value uint64) { 66 | p.Lock() 67 | defer p.Unlock() 68 | p.prevLogIndex = value 69 | } 70 | 71 | func (p *Peer) setLastActivity(now time.Time) { 72 | p.Lock() 73 | defer p.Unlock() 74 | p.lastActivity = now 75 | } 76 | 77 | //------------------------------------------------------------------------------ 78 | // 79 | // Methods 80 | // 81 | //------------------------------------------------------------------------------ 82 | 83 | //-------------------------------------- 84 | // Heartbeat 85 | //-------------------------------------- 86 | 87 | // Starts the peer heartbeat. 88 | func (p *Peer) startHeartbeat() { 89 | p.stopChan = make(chan bool) 90 | c := make(chan bool) 91 | 92 | p.setLastActivity(time.Now()) 93 | 94 | p.server.routineGroup.Add(1) 95 | go func() { 96 | defer p.server.routineGroup.Done() 97 | p.heartbeat(c) 98 | }() 99 | <-c 100 | } 101 | 102 | // Stops the peer heartbeat. 103 | func (p *Peer) stopHeartbeat(flush bool) { 104 | p.setLastActivity(time.Time{}) 105 | 106 | p.stopChan <- flush 107 | } 108 | 109 | // LastActivity returns the last time any response was received from the peer. 110 | func (p *Peer) LastActivity() time.Time { 111 | p.RLock() 112 | defer p.RUnlock() 113 | return p.lastActivity 114 | } 115 | 116 | //-------------------------------------- 117 | // Copying 118 | //-------------------------------------- 119 | 120 | // Clones the state of the peer. The clone is not attached to a server and 121 | // the heartbeat timer will not exist. 122 | func (p *Peer) clone() *Peer { 123 | p.Lock() 124 | defer p.Unlock() 125 | return &Peer{ 126 | Name: p.Name, 127 | ConnectionString: p.ConnectionString, 128 | prevLogIndex: p.prevLogIndex, 129 | lastActivity: p.lastActivity, 130 | } 131 | } 132 | 133 | //-------------------------------------- 134 | // Heartbeat 135 | //-------------------------------------- 136 | 137 | // Listens to the heartbeat timeout and flushes an AppendEntries RPC. 138 | func (p *Peer) heartbeat(c chan bool) { 139 | stopChan := p.stopChan 140 | 141 | c <- true 142 | 143 | ticker := time.Tick(p.heartbeatInterval) 144 | 145 | debugln("peer.heartbeat: ", p.Name, p.heartbeatInterval) 146 | 147 | for { 148 | select { 149 | case flush := <-stopChan: 150 | if flush { 151 | // before we can safely remove a node 152 | // we must flush the remove command to the node first 153 | p.flush() 154 | debugln("peer.heartbeat.stop.with.flush: ", p.Name) 155 | return 156 | } else { 157 | debugln("peer.heartbeat.stop: ", p.Name) 158 | return 159 | } 160 | 161 | case <-ticker: 162 | start := time.Now() 163 | p.flush() 164 | duration := time.Now().Sub(start) 165 | p.server.DispatchEvent(newEvent(HeartbeatEventType, duration, nil)) 166 | } 167 | } 168 | } 169 | 170 | func (p *Peer) flush() { 171 | debugln("peer.heartbeat.flush: ", p.Name) 172 | prevLogIndex := p.getPrevLogIndex() 173 | term := p.server.currentTerm 174 | 175 | entries, prevLogTerm := p.server.log.getEntriesAfter(prevLogIndex, p.server.maxLogEntriesPerRequest) 176 | 177 | if entries != nil { 178 | p.sendAppendEntriesRequest(newAppendEntriesRequest(term, prevLogIndex, prevLogTerm, p.server.log.CommitIndex(), p.server.name, entries)) 179 | } else { 180 | p.sendSnapshotRequest(newSnapshotRequest(p.server.name, p.server.snapshot)) 181 | } 182 | } 183 | 184 | //-------------------------------------- 185 | // Append Entries 186 | //-------------------------------------- 187 | 188 | // Sends an AppendEntries request to the peer through the transport. 189 | func (p *Peer) sendAppendEntriesRequest(req *AppendEntriesRequest) { 190 | tracef("peer.append.send: %s->%s [prevLog:%v length: %v]\n", 191 | p.server.Name(), p.Name, req.PrevLogIndex, len(req.Entries)) 192 | 193 | resp := p.server.Transporter().SendAppendEntriesRequest(p.server, p, req) 194 | if resp == nil { 195 | p.server.DispatchEvent(newEvent(HeartbeatIntervalEventType, p, nil)) 196 | debugln("peer.append.timeout: ", p.server.Name(), "->", p.Name) 197 | return 198 | } 199 | traceln("peer.append.resp: ", p.server.Name(), "<-", p.Name) 200 | 201 | p.setLastActivity(time.Now()) 202 | // If successful then update the previous log index. 203 | p.Lock() 204 | if resp.Success() { 205 | if len(req.Entries) > 0 { 206 | p.prevLogIndex = req.Entries[len(req.Entries)-1].GetIndex() 207 | 208 | // if peer append a log entry from the current term 209 | // we set append to true 210 | if req.Entries[len(req.Entries)-1].GetTerm() == p.server.currentTerm { 211 | resp.append = true 212 | } 213 | } 214 | traceln("peer.append.resp.success: ", p.Name, "; idx =", p.prevLogIndex) 215 | // If it was unsuccessful then decrement the previous log index and 216 | // we'll try again next time. 217 | } else { 218 | if resp.Term() > p.server.Term() { 219 | // this happens when there is a new leader comes up that this *leader* has not 220 | // known yet. 221 | // this server can know until the new leader send a ae with higher term 222 | // or this server finish processing this response. 223 | debugln("peer.append.resp.not.update: new.leader.found") 224 | } else if resp.Term() == req.Term && resp.CommitIndex() >= p.prevLogIndex { 225 | // we may miss a response from peer 226 | // so maybe the peer has committed the logs we just sent 227 | // but we did not receive the successful reply and did not increase 228 | // the prevLogIndex 229 | 230 | // peer failed to truncate the log and sent a fail reply at this time 231 | // we just need to update peer's prevLog index to commitIndex 232 | 233 | p.prevLogIndex = resp.CommitIndex() 234 | debugln("peer.append.resp.update: ", p.Name, "; idx =", p.prevLogIndex) 235 | 236 | } else if p.prevLogIndex > 0 { 237 | // Decrement the previous log index down until we find a match. Don't 238 | // let it go below where the peer's commit index is though. That's a 239 | // problem. 240 | p.prevLogIndex-- 241 | // if it not enough, we directly decrease to the index of the 242 | if p.prevLogIndex > resp.Index() { 243 | p.prevLogIndex = resp.Index() 244 | } 245 | 246 | debugln("peer.append.resp.decrement: ", p.Name, "; idx =", p.prevLogIndex) 247 | } 248 | } 249 | p.Unlock() 250 | 251 | // Attach the peer to resp, thus server can know where it comes from 252 | resp.peer = p.Name 253 | // Send response to server for processing. 254 | p.server.sendAsync(resp) 255 | } 256 | 257 | // Sends an Snapshot request to the peer through the transport. 258 | func (p *Peer) sendSnapshotRequest(req *SnapshotRequest) { 259 | debugln("peer.snap.send: ", p.Name) 260 | 261 | resp := p.server.Transporter().SendSnapshotRequest(p.server, p, req) 262 | if resp == nil { 263 | debugln("peer.snap.timeout: ", p.Name) 264 | return 265 | } 266 | 267 | debugln("peer.snap.recv: ", p.Name) 268 | 269 | // If successful, the peer should have been to snapshot state 270 | // Send it the snapshot! 271 | p.setLastActivity(time.Now()) 272 | 273 | if resp.Success { 274 | p.sendSnapshotRecoveryRequest() 275 | } else { 276 | debugln("peer.snap.failed: ", p.Name) 277 | return 278 | } 279 | 280 | } 281 | 282 | // Sends an Snapshot Recovery request to the peer through the transport. 283 | func (p *Peer) sendSnapshotRecoveryRequest() { 284 | req := newSnapshotRecoveryRequest(p.server.name, p.server.snapshot) 285 | debugln("peer.snap.recovery.send: ", p.Name) 286 | resp := p.server.Transporter().SendSnapshotRecoveryRequest(p.server, p, req) 287 | 288 | if resp == nil { 289 | debugln("peer.snap.recovery.timeout: ", p.Name) 290 | return 291 | } 292 | 293 | p.setLastActivity(time.Now()) 294 | if resp.Success { 295 | p.prevLogIndex = req.LastIndex 296 | } else { 297 | debugln("peer.snap.recovery.failed: ", p.Name) 298 | return 299 | } 300 | 301 | p.server.sendAsync(resp) 302 | } 303 | 304 | //-------------------------------------- 305 | // Vote Requests 306 | //-------------------------------------- 307 | 308 | // send VoteRequest Request 309 | func (p *Peer) sendVoteRequest(req *RequestVoteRequest, c chan *RequestVoteResponse) { 310 | debugln("peer.vote: ", p.server.Name(), "->", p.Name) 311 | req.peer = p 312 | if resp := p.server.Transporter().SendVoteRequest(p.server, p, req); resp != nil { 313 | debugln("peer.vote.recv: ", p.server.Name(), "<-", p.Name) 314 | p.setLastActivity(time.Now()) 315 | resp.peer = p 316 | c <- resp 317 | } else { 318 | debugln("peer.vote.failed: ", p.server.Name(), "<-", p.Name) 319 | } 320 | } 321 | -------------------------------------------------------------------------------- /protobuf/append_entries_request.pb.go: -------------------------------------------------------------------------------- 1 | // Code generated by protoc-gen-go. 2 | // source: append_entries_request.proto 3 | // DO NOT EDIT! 4 | 5 | /* 6 | Package protobuf is a generated protocol buffer package. 7 | 8 | It is generated from these files: 9 | append_entries_request.proto 10 | append_entries_responses.proto 11 | log_entry.proto 12 | request_vote_request.proto 13 | request_vote_responses.proto 14 | snapshot_recovery_request.proto 15 | snapshot_recovery_response.proto 16 | snapshot_request.proto 17 | snapshot_response.proto 18 | 19 | It has these top-level messages: 20 | AppendEntriesRequest 21 | */ 22 | package protobuf 23 | 24 | import proto "code.google.com/p/goprotobuf/proto" 25 | import math "math" 26 | 27 | // discarding unused import gogoproto "code.google.com/p/gogoprotobuf/gogoproto/gogo.pb" 28 | 29 | // Reference imports to suppress errors if they are not otherwise used. 30 | var _ = proto.Marshal 31 | var _ = math.Inf 32 | 33 | type AppendEntriesRequest struct { 34 | Term *uint64 `protobuf:"varint,1,req" json:"Term,omitempty"` 35 | PrevLogIndex *uint64 `protobuf:"varint,2,req" json:"PrevLogIndex,omitempty"` 36 | PrevLogTerm *uint64 `protobuf:"varint,3,req" json:"PrevLogTerm,omitempty"` 37 | CommitIndex *uint64 `protobuf:"varint,4,req" json:"CommitIndex,omitempty"` 38 | LeaderName *string `protobuf:"bytes,5,req" json:"LeaderName,omitempty"` 39 | Entries []*LogEntry `protobuf:"bytes,6,rep" json:"Entries,omitempty"` 40 | XXX_unrecognized []byte `json:"-"` 41 | } 42 | 43 | func (m *AppendEntriesRequest) Reset() { *m = AppendEntriesRequest{} } 44 | func (m *AppendEntriesRequest) String() string { return proto.CompactTextString(m) } 45 | func (*AppendEntriesRequest) ProtoMessage() {} 46 | 47 | func (m *AppendEntriesRequest) GetTerm() uint64 { 48 | if m != nil && m.Term != nil { 49 | return *m.Term 50 | } 51 | return 0 52 | } 53 | 54 | func (m *AppendEntriesRequest) GetPrevLogIndex() uint64 { 55 | if m != nil && m.PrevLogIndex != nil { 56 | return *m.PrevLogIndex 57 | } 58 | return 0 59 | } 60 | 61 | func (m *AppendEntriesRequest) GetPrevLogTerm() uint64 { 62 | if m != nil && m.PrevLogTerm != nil { 63 | return *m.PrevLogTerm 64 | } 65 | return 0 66 | } 67 | 68 | func (m *AppendEntriesRequest) GetCommitIndex() uint64 { 69 | if m != nil && m.CommitIndex != nil { 70 | return *m.CommitIndex 71 | } 72 | return 0 73 | } 74 | 75 | func (m *AppendEntriesRequest) GetLeaderName() string { 76 | if m != nil && m.LeaderName != nil { 77 | return *m.LeaderName 78 | } 79 | return "" 80 | } 81 | 82 | func (m *AppendEntriesRequest) GetEntries() []*LogEntry { 83 | if m != nil { 84 | return m.Entries 85 | } 86 | return nil 87 | } 88 | 89 | func init() { 90 | } 91 | -------------------------------------------------------------------------------- /protobuf/append_entries_request.proto: -------------------------------------------------------------------------------- 1 | package protobuf; 2 | 3 | import "code.google.com/p/gogoprotobuf/gogoproto/gogo.proto"; 4 | import "log_entry.proto"; 5 | 6 | option (gogoproto.gostring_all) = true; 7 | option (gogoproto.equal_all) = true; 8 | option (gogoproto.verbose_equal_all) = true; 9 | option (gogoproto.goproto_stringer_all) = false; 10 | option (gogoproto.stringer_all) = true; 11 | option (gogoproto.populate_all) = true; 12 | option (gogoproto.testgen_all) = true; 13 | option (gogoproto.benchgen_all) = true; 14 | option (gogoproto.marshaler_all) = true; 15 | option (gogoproto.sizer_all) = true; 16 | option (gogoproto.unmarshaler_all) = true; 17 | 18 | message AppendEntriesRequest { 19 | required uint64 Term=1; 20 | required uint64 PrevLogIndex=2; 21 | required uint64 PrevLogTerm=3; 22 | required uint64 CommitIndex=4; 23 | required string LeaderName=5; 24 | repeated LogEntry Entries=6; 25 | } -------------------------------------------------------------------------------- /protobuf/append_entries_responses.pb.go: -------------------------------------------------------------------------------- 1 | // Code generated by protoc-gen-go. 2 | // source: append_entries_responses.proto 3 | // DO NOT EDIT! 4 | 5 | package protobuf 6 | 7 | import proto "code.google.com/p/goprotobuf/proto" 8 | import math "math" 9 | 10 | // discarding unused import gogoproto "code.google.com/p/gogoprotobuf/gogoproto/gogo.pb" 11 | 12 | // Reference imports to suppress errors if they are not otherwise used. 13 | var _ = proto.Marshal 14 | var _ = math.Inf 15 | 16 | type AppendEntriesResponse struct { 17 | Term *uint64 `protobuf:"varint,1,req" json:"Term,omitempty"` 18 | Index *uint64 `protobuf:"varint,2,req" json:"Index,omitempty"` 19 | CommitIndex *uint64 `protobuf:"varint,3,req" json:"CommitIndex,omitempty"` 20 | Success *bool `protobuf:"varint,4,req" json:"Success,omitempty"` 21 | XXX_unrecognized []byte `json:"-"` 22 | } 23 | 24 | func (m *AppendEntriesResponse) Reset() { *m = AppendEntriesResponse{} } 25 | func (m *AppendEntriesResponse) String() string { return proto.CompactTextString(m) } 26 | func (*AppendEntriesResponse) ProtoMessage() {} 27 | 28 | func (m *AppendEntriesResponse) GetTerm() uint64 { 29 | if m != nil && m.Term != nil { 30 | return *m.Term 31 | } 32 | return 0 33 | } 34 | 35 | func (m *AppendEntriesResponse) GetIndex() uint64 { 36 | if m != nil && m.Index != nil { 37 | return *m.Index 38 | } 39 | return 0 40 | } 41 | 42 | func (m *AppendEntriesResponse) GetCommitIndex() uint64 { 43 | if m != nil && m.CommitIndex != nil { 44 | return *m.CommitIndex 45 | } 46 | return 0 47 | } 48 | 49 | func (m *AppendEntriesResponse) GetSuccess() bool { 50 | if m != nil && m.Success != nil { 51 | return *m.Success 52 | } 53 | return false 54 | } 55 | 56 | func init() { 57 | } 58 | -------------------------------------------------------------------------------- /protobuf/append_entries_responses.proto: -------------------------------------------------------------------------------- 1 | package protobuf; 2 | 3 | import "code.google.com/p/gogoprotobuf/gogoproto/gogo.proto"; 4 | 5 | option (gogoproto.gostring_all) = true; 6 | option (gogoproto.equal_all) = true; 7 | option (gogoproto.verbose_equal_all) = true; 8 | option (gogoproto.goproto_stringer_all) = false; 9 | option (gogoproto.stringer_all) = true; 10 | option (gogoproto.populate_all) = true; 11 | option (gogoproto.testgen_all) = true; 12 | option (gogoproto.benchgen_all) = true; 13 | option (gogoproto.marshaler_all) = true; 14 | option (gogoproto.sizer_all) = true; 15 | option (gogoproto.unmarshaler_all) = true; 16 | 17 | message AppendEntriesResponse { 18 | required uint64 Term=1; 19 | required uint64 Index=2; 20 | required uint64 CommitIndex=3; 21 | required bool Success=4; 22 | } -------------------------------------------------------------------------------- /protobuf/log_entry.pb.go: -------------------------------------------------------------------------------- 1 | // Code generated by protoc-gen-go. 2 | // source: log_entry.proto 3 | // DO NOT EDIT! 4 | 5 | package protobuf 6 | 7 | import proto "code.google.com/p/goprotobuf/proto" 8 | import math "math" 9 | 10 | // discarding unused import gogoproto "code.google.com/p/gogoprotobuf/gogoproto/gogo.pb" 11 | 12 | // Reference imports to suppress errors if they are not otherwise used. 13 | var _ = proto.Marshal 14 | var _ = math.Inf 15 | 16 | type LogEntry struct { 17 | Index *uint64 `protobuf:"varint,1,req" json:"Index,omitempty"` 18 | Term *uint64 `protobuf:"varint,2,req" json:"Term,omitempty"` 19 | CommandName *string `protobuf:"bytes,3,req" json:"CommandName,omitempty"` 20 | Command []byte `protobuf:"bytes,4,opt" json:"Command,omitempty"` 21 | XXX_unrecognized []byte `json:"-"` 22 | } 23 | 24 | func (m *LogEntry) Reset() { *m = LogEntry{} } 25 | func (m *LogEntry) String() string { return proto.CompactTextString(m) } 26 | func (*LogEntry) ProtoMessage() {} 27 | 28 | func (m *LogEntry) GetIndex() uint64 { 29 | if m != nil && m.Index != nil { 30 | return *m.Index 31 | } 32 | return 0 33 | } 34 | 35 | func (m *LogEntry) GetTerm() uint64 { 36 | if m != nil && m.Term != nil { 37 | return *m.Term 38 | } 39 | return 0 40 | } 41 | 42 | func (m *LogEntry) GetCommandName() string { 43 | if m != nil && m.CommandName != nil { 44 | return *m.CommandName 45 | } 46 | return "" 47 | } 48 | 49 | func (m *LogEntry) GetCommand() []byte { 50 | if m != nil { 51 | return m.Command 52 | } 53 | return nil 54 | } 55 | 56 | func init() { 57 | } 58 | -------------------------------------------------------------------------------- /protobuf/log_entry.proto: -------------------------------------------------------------------------------- 1 | package protobuf; 2 | 3 | import "code.google.com/p/gogoprotobuf/gogoproto/gogo.proto"; 4 | 5 | option (gogoproto.gostring_all) = true; 6 | option (gogoproto.equal_all) = true; 7 | option (gogoproto.verbose_equal_all) = true; 8 | option (gogoproto.goproto_stringer_all) = false; 9 | option (gogoproto.stringer_all) = true; 10 | option (gogoproto.populate_all) = true; 11 | option (gogoproto.testgen_all) = true; 12 | option (gogoproto.benchgen_all) = true; 13 | option (gogoproto.marshaler_all) = true; 14 | option (gogoproto.sizer_all) = true; 15 | option (gogoproto.unmarshaler_all) = true; 16 | 17 | message LogEntry { 18 | required uint64 Index=1; 19 | required uint64 Term=2; 20 | required string CommandName=3; 21 | optional bytes Command=4; // for nop-command 22 | } -------------------------------------------------------------------------------- /protobuf/request_vote_request.pb.go: -------------------------------------------------------------------------------- 1 | // Code generated by protoc-gen-go. 2 | // source: request_vote_request.proto 3 | // DO NOT EDIT! 4 | 5 | package protobuf 6 | 7 | import proto "code.google.com/p/goprotobuf/proto" 8 | import math "math" 9 | 10 | // discarding unused import gogoproto "code.google.com/p/gogoprotobuf/gogoproto/gogo.pb" 11 | 12 | // Reference imports to suppress errors if they are not otherwise used. 13 | var _ = proto.Marshal 14 | var _ = math.Inf 15 | 16 | type RequestVoteRequest struct { 17 | Term *uint64 `protobuf:"varint,1,req" json:"Term,omitempty"` 18 | LastLogIndex *uint64 `protobuf:"varint,2,req" json:"LastLogIndex,omitempty"` 19 | LastLogTerm *uint64 `protobuf:"varint,3,req" json:"LastLogTerm,omitempty"` 20 | CandidateName *string `protobuf:"bytes,4,req" json:"CandidateName,omitempty"` 21 | XXX_unrecognized []byte `json:"-"` 22 | } 23 | 24 | func (m *RequestVoteRequest) Reset() { *m = RequestVoteRequest{} } 25 | func (m *RequestVoteRequest) String() string { return proto.CompactTextString(m) } 26 | func (*RequestVoteRequest) ProtoMessage() {} 27 | 28 | func (m *RequestVoteRequest) GetTerm() uint64 { 29 | if m != nil && m.Term != nil { 30 | return *m.Term 31 | } 32 | return 0 33 | } 34 | 35 | func (m *RequestVoteRequest) GetLastLogIndex() uint64 { 36 | if m != nil && m.LastLogIndex != nil { 37 | return *m.LastLogIndex 38 | } 39 | return 0 40 | } 41 | 42 | func (m *RequestVoteRequest) GetLastLogTerm() uint64 { 43 | if m != nil && m.LastLogTerm != nil { 44 | return *m.LastLogTerm 45 | } 46 | return 0 47 | } 48 | 49 | func (m *RequestVoteRequest) GetCandidateName() string { 50 | if m != nil && m.CandidateName != nil { 51 | return *m.CandidateName 52 | } 53 | return "" 54 | } 55 | 56 | func init() { 57 | } 58 | -------------------------------------------------------------------------------- /protobuf/request_vote_request.proto: -------------------------------------------------------------------------------- 1 | package protobuf; 2 | 3 | import "code.google.com/p/gogoprotobuf/gogoproto/gogo.proto"; 4 | 5 | option (gogoproto.gostring_all) = true; 6 | option (gogoproto.equal_all) = true; 7 | option (gogoproto.verbose_equal_all) = true; 8 | option (gogoproto.goproto_stringer_all) = false; 9 | option (gogoproto.stringer_all) = true; 10 | option (gogoproto.populate_all) = true; 11 | option (gogoproto.testgen_all) = true; 12 | option (gogoproto.benchgen_all) = true; 13 | option (gogoproto.marshaler_all) = true; 14 | option (gogoproto.sizer_all) = true; 15 | option (gogoproto.unmarshaler_all) = true; 16 | 17 | message RequestVoteRequest { 18 | required uint64 Term=1; 19 | required uint64 LastLogIndex=2; 20 | required uint64 LastLogTerm=3; 21 | required string CandidateName=4; 22 | } -------------------------------------------------------------------------------- /protobuf/request_vote_responses.pb.go: -------------------------------------------------------------------------------- 1 | // Code generated by protoc-gen-go. 2 | // source: request_vote_responses.proto 3 | // DO NOT EDIT! 4 | 5 | package protobuf 6 | 7 | import proto "code.google.com/p/goprotobuf/proto" 8 | import math "math" 9 | 10 | // discarding unused import gogoproto "code.google.com/p/gogoprotobuf/gogoproto/gogo.pb" 11 | 12 | // Reference imports to suppress errors if they are not otherwise used. 13 | var _ = proto.Marshal 14 | var _ = math.Inf 15 | 16 | type RequestVoteResponse struct { 17 | Term *uint64 `protobuf:"varint,1,req" json:"Term,omitempty"` 18 | VoteGranted *bool `protobuf:"varint,2,req" json:"VoteGranted,omitempty"` 19 | XXX_unrecognized []byte `json:"-"` 20 | } 21 | 22 | func (m *RequestVoteResponse) Reset() { *m = RequestVoteResponse{} } 23 | func (m *RequestVoteResponse) String() string { return proto.CompactTextString(m) } 24 | func (*RequestVoteResponse) ProtoMessage() {} 25 | 26 | func (m *RequestVoteResponse) GetTerm() uint64 { 27 | if m != nil && m.Term != nil { 28 | return *m.Term 29 | } 30 | return 0 31 | } 32 | 33 | func (m *RequestVoteResponse) GetVoteGranted() bool { 34 | if m != nil && m.VoteGranted != nil { 35 | return *m.VoteGranted 36 | } 37 | return false 38 | } 39 | 40 | func init() { 41 | } 42 | -------------------------------------------------------------------------------- /protobuf/request_vote_responses.proto: -------------------------------------------------------------------------------- 1 | package protobuf; 2 | 3 | import "code.google.com/p/gogoprotobuf/gogoproto/gogo.proto"; 4 | 5 | option (gogoproto.gostring_all) = true; 6 | option (gogoproto.equal_all) = true; 7 | option (gogoproto.verbose_equal_all) = true; 8 | option (gogoproto.goproto_stringer_all) = false; 9 | option (gogoproto.stringer_all) = true; 10 | option (gogoproto.populate_all) = true; 11 | option (gogoproto.testgen_all) = true; 12 | option (gogoproto.benchgen_all) = true; 13 | option (gogoproto.marshaler_all) = true; 14 | option (gogoproto.sizer_all) = true; 15 | option (gogoproto.unmarshaler_all) = true; 16 | 17 | message RequestVoteResponse { 18 | required uint64 Term=1; 19 | required bool VoteGranted=2; 20 | } -------------------------------------------------------------------------------- /protobuf/snapshot_recovery_request.pb.go: -------------------------------------------------------------------------------- 1 | // Code generated by protoc-gen-go. 2 | // source: snapshot_recovery_request.proto 3 | // DO NOT EDIT! 4 | 5 | package protobuf 6 | 7 | import proto "code.google.com/p/goprotobuf/proto" 8 | import math "math" 9 | 10 | // discarding unused import gogoproto "code.google.com/p/gogoprotobuf/gogoproto/gogo.pb" 11 | 12 | // Reference imports to suppress errors if they are not otherwise used. 13 | var _ = proto.Marshal 14 | var _ = math.Inf 15 | 16 | type SnapshotRecoveryRequest struct { 17 | LeaderName *string `protobuf:"bytes,1,req" json:"LeaderName,omitempty"` 18 | LastIndex *uint64 `protobuf:"varint,2,req" json:"LastIndex,omitempty"` 19 | LastTerm *uint64 `protobuf:"varint,3,req" json:"LastTerm,omitempty"` 20 | Peers []*SnapshotRecoveryRequest_Peer `protobuf:"bytes,4,rep" json:"Peers,omitempty"` 21 | State []byte `protobuf:"bytes,5,req" json:"State,omitempty"` 22 | XXX_unrecognized []byte `json:"-"` 23 | } 24 | 25 | func (m *SnapshotRecoveryRequest) Reset() { *m = SnapshotRecoveryRequest{} } 26 | func (m *SnapshotRecoveryRequest) String() string { return proto.CompactTextString(m) } 27 | func (*SnapshotRecoveryRequest) ProtoMessage() {} 28 | 29 | func (m *SnapshotRecoveryRequest) GetLeaderName() string { 30 | if m != nil && m.LeaderName != nil { 31 | return *m.LeaderName 32 | } 33 | return "" 34 | } 35 | 36 | func (m *SnapshotRecoveryRequest) GetLastIndex() uint64 { 37 | if m != nil && m.LastIndex != nil { 38 | return *m.LastIndex 39 | } 40 | return 0 41 | } 42 | 43 | func (m *SnapshotRecoveryRequest) GetLastTerm() uint64 { 44 | if m != nil && m.LastTerm != nil { 45 | return *m.LastTerm 46 | } 47 | return 0 48 | } 49 | 50 | func (m *SnapshotRecoveryRequest) GetPeers() []*SnapshotRecoveryRequest_Peer { 51 | if m != nil { 52 | return m.Peers 53 | } 54 | return nil 55 | } 56 | 57 | func (m *SnapshotRecoveryRequest) GetState() []byte { 58 | if m != nil { 59 | return m.State 60 | } 61 | return nil 62 | } 63 | 64 | type SnapshotRecoveryRequest_Peer struct { 65 | Name *string `protobuf:"bytes,1,req" json:"Name,omitempty"` 66 | ConnectionString *string `protobuf:"bytes,2,req" json:"ConnectionString,omitempty"` 67 | XXX_unrecognized []byte `json:"-"` 68 | } 69 | 70 | func (m *SnapshotRecoveryRequest_Peer) Reset() { *m = SnapshotRecoveryRequest_Peer{} } 71 | func (m *SnapshotRecoveryRequest_Peer) String() string { return proto.CompactTextString(m) } 72 | func (*SnapshotRecoveryRequest_Peer) ProtoMessage() {} 73 | 74 | func (m *SnapshotRecoveryRequest_Peer) GetName() string { 75 | if m != nil && m.Name != nil { 76 | return *m.Name 77 | } 78 | return "" 79 | } 80 | 81 | func (m *SnapshotRecoveryRequest_Peer) GetConnectionString() string { 82 | if m != nil && m.ConnectionString != nil { 83 | return *m.ConnectionString 84 | } 85 | return "" 86 | } 87 | 88 | func init() { 89 | } 90 | -------------------------------------------------------------------------------- /protobuf/snapshot_recovery_request.proto: -------------------------------------------------------------------------------- 1 | package protobuf; 2 | 3 | import "code.google.com/p/gogoprotobuf/gogoproto/gogo.proto"; 4 | 5 | option (gogoproto.gostring_all) = true; 6 | option (gogoproto.equal_all) = true; 7 | option (gogoproto.verbose_equal_all) = true; 8 | option (gogoproto.goproto_stringer_all) = false; 9 | option (gogoproto.stringer_all) = true; 10 | option (gogoproto.populate_all) = true; 11 | option (gogoproto.testgen_all) = true; 12 | option (gogoproto.benchgen_all) = true; 13 | option (gogoproto.marshaler_all) = true; 14 | option (gogoproto.sizer_all) = true; 15 | option (gogoproto.unmarshaler_all) = true; 16 | 17 | message SnapshotRecoveryRequest { 18 | required string LeaderName=1; 19 | required uint64 LastIndex=2; 20 | required uint64 LastTerm=3; 21 | 22 | message Peer { 23 | required string Name=1; 24 | required string ConnectionString=2; 25 | } 26 | repeated Peer Peers=4; 27 | 28 | required bytes State=5; 29 | } -------------------------------------------------------------------------------- /protobuf/snapshot_recovery_response.pb.go: -------------------------------------------------------------------------------- 1 | // Code generated by protoc-gen-go. 2 | // source: snapshot_recovery_response.proto 3 | // DO NOT EDIT! 4 | 5 | package protobuf 6 | 7 | import proto "code.google.com/p/goprotobuf/proto" 8 | import math "math" 9 | 10 | // discarding unused import gogoproto "code.google.com/p/gogoprotobuf/gogoproto/gogo.pb" 11 | 12 | // Reference imports to suppress errors if they are not otherwise used. 13 | var _ = proto.Marshal 14 | var _ = math.Inf 15 | 16 | type SnapshotRecoveryResponse struct { 17 | Term *uint64 `protobuf:"varint,1,req" json:"Term,omitempty"` 18 | Success *bool `protobuf:"varint,2,req" json:"Success,omitempty"` 19 | CommitIndex *uint64 `protobuf:"varint,3,req" json:"CommitIndex,omitempty"` 20 | XXX_unrecognized []byte `json:"-"` 21 | } 22 | 23 | func (m *SnapshotRecoveryResponse) Reset() { *m = SnapshotRecoveryResponse{} } 24 | func (m *SnapshotRecoveryResponse) String() string { return proto.CompactTextString(m) } 25 | func (*SnapshotRecoveryResponse) ProtoMessage() {} 26 | 27 | func (m *SnapshotRecoveryResponse) GetTerm() uint64 { 28 | if m != nil && m.Term != nil { 29 | return *m.Term 30 | } 31 | return 0 32 | } 33 | 34 | func (m *SnapshotRecoveryResponse) GetSuccess() bool { 35 | if m != nil && m.Success != nil { 36 | return *m.Success 37 | } 38 | return false 39 | } 40 | 41 | func (m *SnapshotRecoveryResponse) GetCommitIndex() uint64 { 42 | if m != nil && m.CommitIndex != nil { 43 | return *m.CommitIndex 44 | } 45 | return 0 46 | } 47 | 48 | func init() { 49 | } 50 | -------------------------------------------------------------------------------- /protobuf/snapshot_recovery_response.proto: -------------------------------------------------------------------------------- 1 | package protobuf; 2 | 3 | import "code.google.com/p/gogoprotobuf/gogoproto/gogo.proto"; 4 | 5 | option (gogoproto.gostring_all) = true; 6 | option (gogoproto.equal_all) = true; 7 | option (gogoproto.verbose_equal_all) = true; 8 | option (gogoproto.goproto_stringer_all) = false; 9 | option (gogoproto.stringer_all) = true; 10 | option (gogoproto.populate_all) = true; 11 | option (gogoproto.testgen_all) = true; 12 | option (gogoproto.benchgen_all) = true; 13 | option (gogoproto.marshaler_all) = true; 14 | option (gogoproto.sizer_all) = true; 15 | option (gogoproto.unmarshaler_all) = true; 16 | 17 | message SnapshotRecoveryResponse { 18 | required uint64 Term=1; 19 | required bool Success=2; 20 | required uint64 CommitIndex=3; 21 | } -------------------------------------------------------------------------------- /protobuf/snapshot_request.pb.go: -------------------------------------------------------------------------------- 1 | // Code generated by protoc-gen-go. 2 | // source: snapshot_request.proto 3 | // DO NOT EDIT! 4 | 5 | package protobuf 6 | 7 | import proto "code.google.com/p/goprotobuf/proto" 8 | import math "math" 9 | 10 | // discarding unused import gogoproto "code.google.com/p/gogoprotobuf/gogoproto/gogo.pb" 11 | 12 | // Reference imports to suppress errors if they are not otherwise used. 13 | var _ = proto.Marshal 14 | var _ = math.Inf 15 | 16 | type SnapshotRequest struct { 17 | LeaderName *string `protobuf:"bytes,1,req" json:"LeaderName,omitempty"` 18 | LastIndex *uint64 `protobuf:"varint,2,req" json:"LastIndex,omitempty"` 19 | LastTerm *uint64 `protobuf:"varint,3,req" json:"LastTerm,omitempty"` 20 | XXX_unrecognized []byte `json:"-"` 21 | } 22 | 23 | func (m *SnapshotRequest) Reset() { *m = SnapshotRequest{} } 24 | func (m *SnapshotRequest) String() string { return proto.CompactTextString(m) } 25 | func (*SnapshotRequest) ProtoMessage() {} 26 | 27 | func (m *SnapshotRequest) GetLeaderName() string { 28 | if m != nil && m.LeaderName != nil { 29 | return *m.LeaderName 30 | } 31 | return "" 32 | } 33 | 34 | func (m *SnapshotRequest) GetLastIndex() uint64 { 35 | if m != nil && m.LastIndex != nil { 36 | return *m.LastIndex 37 | } 38 | return 0 39 | } 40 | 41 | func (m *SnapshotRequest) GetLastTerm() uint64 { 42 | if m != nil && m.LastTerm != nil { 43 | return *m.LastTerm 44 | } 45 | return 0 46 | } 47 | 48 | func init() { 49 | } 50 | -------------------------------------------------------------------------------- /protobuf/snapshot_request.proto: -------------------------------------------------------------------------------- 1 | package protobuf; 2 | 3 | import "code.google.com/p/gogoprotobuf/gogoproto/gogo.proto"; 4 | 5 | option (gogoproto.gostring_all) = true; 6 | option (gogoproto.equal_all) = true; 7 | option (gogoproto.verbose_equal_all) = true; 8 | option (gogoproto.goproto_stringer_all) = false; 9 | option (gogoproto.stringer_all) = true; 10 | option (gogoproto.populate_all) = true; 11 | option (gogoproto.testgen_all) = true; 12 | option (gogoproto.benchgen_all) = true; 13 | option (gogoproto.marshaler_all) = true; 14 | option (gogoproto.sizer_all) = true; 15 | option (gogoproto.unmarshaler_all) = true; 16 | 17 | message SnapshotRequest { 18 | required string LeaderName=1; 19 | required uint64 LastIndex=2; 20 | required uint64 LastTerm=3; 21 | } -------------------------------------------------------------------------------- /protobuf/snapshot_response.pb.go: -------------------------------------------------------------------------------- 1 | // Code generated by protoc-gen-go. 2 | // source: snapshot_response.proto 3 | // DO NOT EDIT! 4 | 5 | package protobuf 6 | 7 | import proto "code.google.com/p/goprotobuf/proto" 8 | import math "math" 9 | 10 | // discarding unused import gogoproto "code.google.com/p/gogoprotobuf/gogoproto/gogo.pb" 11 | 12 | // Reference imports to suppress errors if they are not otherwise used. 13 | var _ = proto.Marshal 14 | var _ = math.Inf 15 | 16 | type SnapshotResponse struct { 17 | Success *bool `protobuf:"varint,1,req" json:"Success,omitempty"` 18 | XXX_unrecognized []byte `json:"-"` 19 | } 20 | 21 | func (m *SnapshotResponse) Reset() { *m = SnapshotResponse{} } 22 | func (m *SnapshotResponse) String() string { return proto.CompactTextString(m) } 23 | func (*SnapshotResponse) ProtoMessage() {} 24 | 25 | func (m *SnapshotResponse) GetSuccess() bool { 26 | if m != nil && m.Success != nil { 27 | return *m.Success 28 | } 29 | return false 30 | } 31 | 32 | func init() { 33 | } 34 | -------------------------------------------------------------------------------- /protobuf/snapshot_response.proto: -------------------------------------------------------------------------------- 1 | package protobuf; 2 | 3 | import "code.google.com/p/gogoprotobuf/gogoproto/gogo.proto"; 4 | 5 | option (gogoproto.gostring_all) = true; 6 | option (gogoproto.equal_all) = true; 7 | option (gogoproto.verbose_equal_all) = true; 8 | option (gogoproto.goproto_stringer_all) = false; 9 | option (gogoproto.stringer_all) = true; 10 | option (gogoproto.populate_all) = true; 11 | option (gogoproto.testgen_all) = true; 12 | option (gogoproto.benchgen_all) = true; 13 | option (gogoproto.marshaler_all) = true; 14 | option (gogoproto.sizer_all) = true; 15 | option (gogoproto.unmarshaler_all) = true; 16 | 17 | message SnapshotResponse { 18 | required bool Success=1; 19 | } -------------------------------------------------------------------------------- /request_vote.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "io" 5 | "io/ioutil" 6 | 7 | "code.google.com/p/gogoprotobuf/proto" 8 | "github.com/goraft/raft/protobuf" 9 | ) 10 | 11 | // The request sent to a server to vote for a candidate to become a leader. 12 | type RequestVoteRequest struct { 13 | peer *Peer 14 | Term uint64 15 | LastLogIndex uint64 16 | LastLogTerm uint64 17 | CandidateName string 18 | } 19 | 20 | // The response returned from a server after a vote for a candidate to become a leader. 21 | type RequestVoteResponse struct { 22 | peer *Peer 23 | Term uint64 24 | VoteGranted bool 25 | } 26 | 27 | // Creates a new RequestVote request. 28 | func newRequestVoteRequest(term uint64, candidateName string, lastLogIndex uint64, lastLogTerm uint64) *RequestVoteRequest { 29 | return &RequestVoteRequest{ 30 | Term: term, 31 | LastLogIndex: lastLogIndex, 32 | LastLogTerm: lastLogTerm, 33 | CandidateName: candidateName, 34 | } 35 | } 36 | 37 | // Encodes the RequestVoteRequest to a buffer. Returns the number of bytes 38 | // written and any error that may have occurred. 39 | func (req *RequestVoteRequest) Encode(w io.Writer) (int, error) { 40 | pb := &protobuf.RequestVoteRequest{ 41 | Term: proto.Uint64(req.Term), 42 | LastLogIndex: proto.Uint64(req.LastLogIndex), 43 | LastLogTerm: proto.Uint64(req.LastLogTerm), 44 | CandidateName: proto.String(req.CandidateName), 45 | } 46 | p, err := proto.Marshal(pb) 47 | if err != nil { 48 | return -1, err 49 | } 50 | 51 | return w.Write(p) 52 | } 53 | 54 | // Decodes the RequestVoteRequest from a buffer. Returns the number of bytes read and 55 | // any error that occurs. 56 | func (req *RequestVoteRequest) Decode(r io.Reader) (int, error) { 57 | data, err := ioutil.ReadAll(r) 58 | 59 | if err != nil { 60 | return -1, err 61 | } 62 | 63 | totalBytes := len(data) 64 | 65 | pb := &protobuf.RequestVoteRequest{} 66 | if err = proto.Unmarshal(data, pb); err != nil { 67 | return -1, err 68 | } 69 | 70 | req.Term = pb.GetTerm() 71 | req.LastLogIndex = pb.GetLastLogIndex() 72 | req.LastLogTerm = pb.GetLastLogTerm() 73 | req.CandidateName = pb.GetCandidateName() 74 | 75 | return totalBytes, nil 76 | } 77 | 78 | // Creates a new RequestVote response. 79 | func newRequestVoteResponse(term uint64, voteGranted bool) *RequestVoteResponse { 80 | return &RequestVoteResponse{ 81 | Term: term, 82 | VoteGranted: voteGranted, 83 | } 84 | } 85 | 86 | // Encodes the RequestVoteResponse to a buffer. Returns the number of bytes 87 | // written and any error that may have occurred. 88 | func (resp *RequestVoteResponse) Encode(w io.Writer) (int, error) { 89 | pb := &protobuf.RequestVoteResponse{ 90 | Term: proto.Uint64(resp.Term), 91 | VoteGranted: proto.Bool(resp.VoteGranted), 92 | } 93 | 94 | p, err := proto.Marshal(pb) 95 | if err != nil { 96 | return -1, err 97 | } 98 | 99 | return w.Write(p) 100 | } 101 | 102 | // Decodes the RequestVoteResponse from a buffer. Returns the number of bytes read and 103 | // any error that occurs. 104 | func (resp *RequestVoteResponse) Decode(r io.Reader) (int, error) { 105 | data, err := ioutil.ReadAll(r) 106 | 107 | if err != nil { 108 | return 0, err 109 | } 110 | 111 | totalBytes := len(data) 112 | 113 | pb := &protobuf.RequestVoteResponse{} 114 | if err = proto.Unmarshal(data, pb); err != nil { 115 | return -1, err 116 | } 117 | 118 | resp.Term = pb.GetTerm() 119 | resp.VoteGranted = pb.GetVoteGranted() 120 | 121 | return totalBytes, nil 122 | } 123 | -------------------------------------------------------------------------------- /server.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "encoding/json" 5 | "errors" 6 | "fmt" 7 | "hash/crc32" 8 | "io/ioutil" 9 | "os" 10 | "path" 11 | "sort" 12 | "sync" 13 | "time" 14 | ) 15 | 16 | //------------------------------------------------------------------------------ 17 | // 18 | // Constants 19 | // 20 | //------------------------------------------------------------------------------ 21 | 22 | const ( 23 | Stopped = "stopped" 24 | Initialized = "initialized" 25 | Follower = "follower" 26 | Candidate = "candidate" 27 | Leader = "leader" 28 | Snapshotting = "snapshotting" 29 | ) 30 | 31 | const ( 32 | MaxLogEntriesPerRequest = 2000 33 | NumberOfLogEntriesAfterSnapshot = 200 34 | ) 35 | 36 | const ( 37 | // DefaultHeartbeatInterval is the interval that the leader will send 38 | // AppendEntriesRequests to followers to maintain leadership. 39 | DefaultHeartbeatInterval = 50 * time.Millisecond 40 | 41 | DefaultElectionTimeout = 150 * time.Millisecond 42 | ) 43 | 44 | // ElectionTimeoutThresholdPercent specifies the threshold at which the server 45 | // will dispatch warning events that the heartbeat RTT is too close to the 46 | // election timeout. 47 | const ElectionTimeoutThresholdPercent = 0.8 48 | 49 | //------------------------------------------------------------------------------ 50 | // 51 | // Errors 52 | // 53 | //------------------------------------------------------------------------------ 54 | 55 | var NotLeaderError = errors.New("raft.Server: Not current leader") 56 | var DuplicatePeerError = errors.New("raft.Server: Duplicate peer") 57 | var CommandTimeoutError = errors.New("raft: Command timeout") 58 | var StopError = errors.New("raft: Has been stopped") 59 | 60 | //------------------------------------------------------------------------------ 61 | // 62 | // Typedefs 63 | // 64 | //------------------------------------------------------------------------------ 65 | 66 | // A server is involved in the consensus protocol and can act as a follower, 67 | // candidate or a leader. 68 | type Server interface { 69 | Name() string 70 | Context() interface{} 71 | StateMachine() StateMachine 72 | Leader() string 73 | State() string 74 | Path() string 75 | LogPath() string 76 | SnapshotPath(lastIndex uint64, lastTerm uint64) string 77 | Term() uint64 78 | CommitIndex() uint64 79 | VotedFor() string 80 | MemberCount() int 81 | QuorumSize() int 82 | IsLogEmpty() bool 83 | LogEntries() []*LogEntry 84 | LastCommandName() string 85 | GetState() string 86 | ElectionTimeout() time.Duration 87 | SetElectionTimeout(duration time.Duration) 88 | HeartbeatInterval() time.Duration 89 | SetHeartbeatInterval(duration time.Duration) 90 | Transporter() Transporter 91 | SetTransporter(t Transporter) 92 | AppendEntries(req *AppendEntriesRequest) *AppendEntriesResponse 93 | RequestVote(req *RequestVoteRequest) *RequestVoteResponse 94 | RequestSnapshot(req *SnapshotRequest) *SnapshotResponse 95 | SnapshotRecoveryRequest(req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse 96 | AddPeer(name string, connectiongString string) error 97 | RemovePeer(name string) error 98 | Peers() map[string]*Peer 99 | Init() error 100 | Start() error 101 | Stop() 102 | Running() bool 103 | Do(command Command) (interface{}, error) 104 | TakeSnapshot() error 105 | LoadSnapshot() error 106 | AddEventListener(string, EventListener) 107 | FlushCommitIndex() 108 | } 109 | 110 | type server struct { 111 | *eventDispatcher 112 | 113 | name string 114 | path string 115 | state string 116 | transporter Transporter 117 | context interface{} 118 | currentTerm uint64 119 | 120 | votedFor string 121 | log *Log 122 | leader string 123 | peers map[string]*Peer 124 | mutex sync.RWMutex 125 | syncedPeer map[string]bool 126 | 127 | stopped chan bool 128 | c chan *ev 129 | electionTimeout time.Duration 130 | heartbeatInterval time.Duration 131 | 132 | snapshot *Snapshot 133 | 134 | // PendingSnapshot is an unfinished snapshot. 135 | // After the pendingSnapshot is saved to disk, 136 | // it will be set to snapshot and also will be 137 | // set to nil. 138 | pendingSnapshot *Snapshot 139 | 140 | stateMachine StateMachine 141 | maxLogEntriesPerRequest uint64 142 | 143 | connectionString string 144 | 145 | routineGroup sync.WaitGroup 146 | } 147 | 148 | // An internal event to be processed by the server's event loop. 149 | type ev struct { 150 | target interface{} 151 | returnValue interface{} 152 | c chan error 153 | } 154 | 155 | //------------------------------------------------------------------------------ 156 | // 157 | // Constructor 158 | // 159 | //------------------------------------------------------------------------------ 160 | 161 | // Creates a new server with a log at the given path. transporter must 162 | // not be nil. stateMachine can be nil if snapshotting and log 163 | // compaction is to be disabled. context can be anything (including nil) 164 | // and is not used by the raft package except returned by 165 | // Server.Context(). connectionString can be anything. 166 | func NewServer(name string, path string, transporter Transporter, stateMachine StateMachine, ctx interface{}, connectionString string) (Server, error) { 167 | if name == "" { 168 | return nil, errors.New("raft.Server: Name cannot be blank") 169 | } 170 | if transporter == nil { 171 | panic("raft: Transporter required") 172 | } 173 | 174 | s := &server{ 175 | name: name, 176 | path: path, 177 | transporter: transporter, 178 | stateMachine: stateMachine, 179 | context: ctx, 180 | state: Stopped, 181 | peers: make(map[string]*Peer), 182 | log: newLog(), 183 | c: make(chan *ev, 256), 184 | electionTimeout: DefaultElectionTimeout, 185 | heartbeatInterval: DefaultHeartbeatInterval, 186 | maxLogEntriesPerRequest: MaxLogEntriesPerRequest, 187 | connectionString: connectionString, 188 | } 189 | s.eventDispatcher = newEventDispatcher(s) 190 | 191 | // Setup apply function. 192 | s.log.ApplyFunc = func(e *LogEntry, c Command) (interface{}, error) { 193 | // Dispatch commit event. 194 | s.DispatchEvent(newEvent(CommitEventType, e, nil)) 195 | 196 | // Apply command to the state machine. 197 | switch c := c.(type) { 198 | case CommandApply: 199 | return c.Apply(&context{ 200 | server: s, 201 | currentTerm: s.currentTerm, 202 | currentIndex: s.log.internalCurrentIndex(), 203 | commitIndex: s.log.commitIndex, 204 | }) 205 | case deprecatedCommandApply: 206 | return c.Apply(s) 207 | default: 208 | return nil, fmt.Errorf("Command does not implement Apply()") 209 | } 210 | } 211 | 212 | return s, nil 213 | } 214 | 215 | //------------------------------------------------------------------------------ 216 | // 217 | // Accessors 218 | // 219 | //------------------------------------------------------------------------------ 220 | 221 | //-------------------------------------- 222 | // General 223 | //-------------------------------------- 224 | 225 | // Retrieves the name of the server. 226 | func (s *server) Name() string { 227 | return s.name 228 | } 229 | 230 | // Retrieves the storage path for the server. 231 | func (s *server) Path() string { 232 | return s.path 233 | } 234 | 235 | // The name of the current leader. 236 | func (s *server) Leader() string { 237 | return s.leader 238 | } 239 | 240 | // Retrieves a copy of the peer data. 241 | func (s *server) Peers() map[string]*Peer { 242 | s.mutex.Lock() 243 | defer s.mutex.Unlock() 244 | 245 | peers := make(map[string]*Peer) 246 | for name, peer := range s.peers { 247 | peers[name] = peer.clone() 248 | } 249 | return peers 250 | } 251 | 252 | // Retrieves the object that transports requests. 253 | func (s *server) Transporter() Transporter { 254 | s.mutex.RLock() 255 | defer s.mutex.RUnlock() 256 | return s.transporter 257 | } 258 | 259 | func (s *server) SetTransporter(t Transporter) { 260 | s.mutex.Lock() 261 | defer s.mutex.Unlock() 262 | s.transporter = t 263 | } 264 | 265 | // Retrieves the context passed into the constructor. 266 | func (s *server) Context() interface{} { 267 | return s.context 268 | } 269 | 270 | // Retrieves the state machine passed into the constructor. 271 | func (s *server) StateMachine() StateMachine { 272 | return s.stateMachine 273 | } 274 | 275 | // Retrieves the log path for the server. 276 | func (s *server) LogPath() string { 277 | return path.Join(s.path, "log") 278 | } 279 | 280 | // Retrieves the current state of the server. 281 | func (s *server) State() string { 282 | s.mutex.RLock() 283 | defer s.mutex.RUnlock() 284 | return s.state 285 | } 286 | 287 | // Sets the state of the server. 288 | func (s *server) setState(state string) { 289 | s.mutex.Lock() 290 | defer s.mutex.Unlock() 291 | 292 | // Temporarily store previous values. 293 | prevState := s.state 294 | prevLeader := s.leader 295 | 296 | // Update state and leader. 297 | s.state = state 298 | if state == Leader { 299 | s.leader = s.Name() 300 | s.syncedPeer = make(map[string]bool) 301 | } 302 | 303 | // Dispatch state and leader change events. 304 | s.DispatchEvent(newEvent(StateChangeEventType, s.state, prevState)) 305 | 306 | if prevLeader != s.leader { 307 | s.DispatchEvent(newEvent(LeaderChangeEventType, s.leader, prevLeader)) 308 | } 309 | } 310 | 311 | // Retrieves the current term of the server. 312 | func (s *server) Term() uint64 { 313 | s.mutex.RLock() 314 | defer s.mutex.RUnlock() 315 | return s.currentTerm 316 | } 317 | 318 | // Retrieves the current commit index of the server. 319 | func (s *server) CommitIndex() uint64 { 320 | s.log.mutex.RLock() 321 | defer s.log.mutex.RUnlock() 322 | return s.log.commitIndex 323 | } 324 | 325 | // Retrieves the name of the candidate this server voted for in this term. 326 | func (s *server) VotedFor() string { 327 | return s.votedFor 328 | } 329 | 330 | // Retrieves whether the server's log has no entries. 331 | func (s *server) IsLogEmpty() bool { 332 | return s.log.isEmpty() 333 | } 334 | 335 | // A list of all the log entries. This should only be used for debugging purposes. 336 | func (s *server) LogEntries() []*LogEntry { 337 | s.log.mutex.RLock() 338 | defer s.log.mutex.RUnlock() 339 | return s.log.entries 340 | } 341 | 342 | // A reference to the command name of the last entry. 343 | func (s *server) LastCommandName() string { 344 | return s.log.lastCommandName() 345 | } 346 | 347 | // Get the state of the server for debugging 348 | func (s *server) GetState() string { 349 | s.mutex.RLock() 350 | defer s.mutex.RUnlock() 351 | return fmt.Sprintf("Name: %s, State: %s, Term: %v, CommitedIndex: %v ", s.name, s.state, s.currentTerm, s.log.commitIndex) 352 | } 353 | 354 | // Check if the server is promotable 355 | func (s *server) promotable() bool { 356 | return s.log.currentIndex() > 0 357 | } 358 | 359 | //-------------------------------------- 360 | // Membership 361 | //-------------------------------------- 362 | 363 | // Retrieves the number of member servers in the consensus. 364 | func (s *server) MemberCount() int { 365 | s.mutex.RLock() 366 | defer s.mutex.RUnlock() 367 | return len(s.peers) + 1 368 | } 369 | 370 | // Retrieves the number of servers required to make a quorum. 371 | func (s *server) QuorumSize() int { 372 | return (s.MemberCount() / 2) + 1 373 | } 374 | 375 | //-------------------------------------- 376 | // Election timeout 377 | //-------------------------------------- 378 | 379 | // Retrieves the election timeout. 380 | func (s *server) ElectionTimeout() time.Duration { 381 | s.mutex.RLock() 382 | defer s.mutex.RUnlock() 383 | return s.electionTimeout 384 | } 385 | 386 | // Sets the election timeout. 387 | func (s *server) SetElectionTimeout(duration time.Duration) { 388 | s.mutex.Lock() 389 | defer s.mutex.Unlock() 390 | s.electionTimeout = duration 391 | } 392 | 393 | //-------------------------------------- 394 | // Heartbeat timeout 395 | //-------------------------------------- 396 | 397 | // Retrieves the heartbeat timeout. 398 | func (s *server) HeartbeatInterval() time.Duration { 399 | s.mutex.RLock() 400 | defer s.mutex.RUnlock() 401 | return s.heartbeatInterval 402 | } 403 | 404 | // Sets the heartbeat timeout. 405 | func (s *server) SetHeartbeatInterval(duration time.Duration) { 406 | s.mutex.Lock() 407 | defer s.mutex.Unlock() 408 | 409 | s.heartbeatInterval = duration 410 | for _, peer := range s.peers { 411 | peer.setHeartbeatInterval(duration) 412 | } 413 | } 414 | 415 | //------------------------------------------------------------------------------ 416 | // 417 | // Methods 418 | // 419 | //------------------------------------------------------------------------------ 420 | 421 | //-------------------------------------- 422 | // Initialization 423 | //-------------------------------------- 424 | 425 | // Reg the NOPCommand 426 | func init() { 427 | RegisterCommand(&NOPCommand{}) 428 | RegisterCommand(&DefaultJoinCommand{}) 429 | RegisterCommand(&DefaultLeaveCommand{}) 430 | } 431 | 432 | // Start the raft server 433 | // If log entries exist then allow promotion to candidate if no AEs received. 434 | // If no log entries exist then wait for AEs from another node. 435 | // If no log entries exist and a self-join command is issued then 436 | // immediately become leader and commit entry. 437 | func (s *server) Start() error { 438 | // Exit if the server is already running. 439 | if s.Running() { 440 | return fmt.Errorf("raft.Server: Server already running[%v]", s.state) 441 | } 442 | 443 | if err := s.Init(); err != nil { 444 | return err 445 | } 446 | 447 | // stopped needs to be allocated each time server starts 448 | // because it is closed at `Stop`. 449 | s.stopped = make(chan bool) 450 | s.setState(Follower) 451 | 452 | // If no log entries exist then 453 | // 1. wait for AEs from another node 454 | // 2. wait for self-join command 455 | // to set itself promotable 456 | if !s.promotable() { 457 | s.debugln("start as a new raft server") 458 | 459 | // If log entries exist then allow promotion to candidate 460 | // if no AEs received. 461 | } else { 462 | s.debugln("start from previous saved state") 463 | } 464 | 465 | debugln(s.GetState()) 466 | 467 | s.routineGroup.Add(1) 468 | go func() { 469 | defer s.routineGroup.Done() 470 | s.loop() 471 | }() 472 | 473 | return nil 474 | } 475 | 476 | // Init initializes the raft server. 477 | // If there is no previous log file under the given path, Init() will create an empty log file. 478 | // Otherwise, Init() will load in the log entries from the log file. 479 | func (s *server) Init() error { 480 | if s.Running() { 481 | return fmt.Errorf("raft.Server: Server already running[%v]", s.state) 482 | } 483 | 484 | // Server has been initialized or server was stopped after initialized 485 | // If log has been initialized, we know that the server was stopped after 486 | // running. 487 | if s.state == Initialized || s.log.initialized { 488 | s.state = Initialized 489 | return nil 490 | } 491 | 492 | // Create snapshot directory if it does not exist 493 | err := os.Mkdir(path.Join(s.path, "snapshot"), 0700) 494 | if err != nil && !os.IsExist(err) { 495 | s.debugln("raft: Snapshot dir error: ", err) 496 | return fmt.Errorf("raft: Initialization error: %s", err) 497 | } 498 | 499 | if err := s.readConf(); err != nil { 500 | s.debugln("raft: Conf file error: ", err) 501 | return fmt.Errorf("raft: Initialization error: %s", err) 502 | } 503 | 504 | // Initialize the log and load it up. 505 | if err := s.log.open(s.LogPath()); err != nil { 506 | s.debugln("raft: Log error: ", err) 507 | return fmt.Errorf("raft: Initialization error: %s", err) 508 | } 509 | 510 | // Update the term to the last term in the log. 511 | _, s.currentTerm = s.log.lastInfo() 512 | 513 | s.state = Initialized 514 | return nil 515 | } 516 | 517 | // Shuts down the server. 518 | func (s *server) Stop() { 519 | if s.State() == Stopped { 520 | return 521 | } 522 | 523 | close(s.stopped) 524 | 525 | // make sure all goroutines have stopped before we close the log 526 | s.routineGroup.Wait() 527 | 528 | s.log.close() 529 | s.setState(Stopped) 530 | } 531 | 532 | // Checks if the server is currently running. 533 | func (s *server) Running() bool { 534 | s.mutex.RLock() 535 | defer s.mutex.RUnlock() 536 | return (s.state != Stopped && s.state != Initialized) 537 | } 538 | 539 | //-------------------------------------- 540 | // Term 541 | //-------------------------------------- 542 | 543 | // updates the current term for the server. This is only used when a larger 544 | // external term is found. 545 | func (s *server) updateCurrentTerm(term uint64, leaderName string) { 546 | _assert(term > s.currentTerm, 547 | "upadteCurrentTerm: update is called when term is not larger than currentTerm") 548 | 549 | // Store previous values temporarily. 550 | prevTerm := s.currentTerm 551 | prevLeader := s.leader 552 | 553 | // set currentTerm = T, convert to follower (§5.1) 554 | // stop heartbeats before step-down 555 | if s.state == Leader { 556 | for _, peer := range s.peers { 557 | peer.stopHeartbeat(false) 558 | } 559 | } 560 | // update the term and clear vote for 561 | if s.state != Follower { 562 | s.setState(Follower) 563 | } 564 | 565 | s.mutex.Lock() 566 | s.currentTerm = term 567 | s.leader = leaderName 568 | s.votedFor = "" 569 | s.mutex.Unlock() 570 | 571 | // Dispatch change events. 572 | s.DispatchEvent(newEvent(TermChangeEventType, s.currentTerm, prevTerm)) 573 | 574 | if prevLeader != s.leader { 575 | s.DispatchEvent(newEvent(LeaderChangeEventType, s.leader, prevLeader)) 576 | } 577 | } 578 | 579 | //-------------------------------------- 580 | // Event Loop 581 | //-------------------------------------- 582 | 583 | // ________ 584 | // --|Snapshot| timeout 585 | // | -------- ______ 586 | // recover | ^ | | 587 | // snapshot / | |snapshot | | 588 | // higher | | v | recv majority votes 589 | // term | -------- timeout ----------- ----------- 590 | // |-> |Follower| ----------> | Candidate |--------------------> | Leader | 591 | // -------- ----------- ----------- 592 | // ^ higher term/ | higher term | 593 | // | new leader | | 594 | // |_______________________|____________________________________ | 595 | // The main event loop for the server 596 | func (s *server) loop() { 597 | defer s.debugln("server.loop.end") 598 | 599 | state := s.State() 600 | 601 | for state != Stopped { 602 | s.debugln("server.loop.run ", state) 603 | switch state { 604 | case Follower: 605 | s.followerLoop() 606 | case Candidate: 607 | s.candidateLoop() 608 | case Leader: 609 | s.leaderLoop() 610 | case Snapshotting: 611 | s.snapshotLoop() 612 | } 613 | state = s.State() 614 | } 615 | } 616 | 617 | // Sends an event to the event loop to be processed. The function will wait 618 | // until the event is actually processed before returning. 619 | func (s *server) send(value interface{}) (interface{}, error) { 620 | if !s.Running() { 621 | return nil, StopError 622 | } 623 | 624 | event := &ev{target: value, c: make(chan error, 1)} 625 | select { 626 | case s.c <- event: 627 | case <-s.stopped: 628 | return nil, StopError 629 | } 630 | select { 631 | case <-s.stopped: 632 | return nil, StopError 633 | case err := <-event.c: 634 | return event.returnValue, err 635 | } 636 | } 637 | 638 | func (s *server) sendAsync(value interface{}) { 639 | if !s.Running() { 640 | return 641 | } 642 | 643 | event := &ev{target: value, c: make(chan error, 1)} 644 | // try a non-blocking send first 645 | // in most cases, this should not be blocking 646 | // avoid create unnecessary go routines 647 | select { 648 | case s.c <- event: 649 | return 650 | default: 651 | } 652 | 653 | s.routineGroup.Add(1) 654 | go func() { 655 | defer s.routineGroup.Done() 656 | select { 657 | case s.c <- event: 658 | case <-s.stopped: 659 | } 660 | }() 661 | } 662 | 663 | // The event loop that is run when the server is in a Follower state. 664 | // Responds to RPCs from candidates and leaders. 665 | // Converts to candidate if election timeout elapses without either: 666 | // 1.Receiving valid AppendEntries RPC, or 667 | // 2.Granting vote to candidate 668 | func (s *server) followerLoop() { 669 | since := time.Now() 670 | electionTimeout := s.ElectionTimeout() 671 | timeoutChan := afterBetween(s.ElectionTimeout(), s.ElectionTimeout()*2) 672 | 673 | for s.State() == Follower { 674 | var err error 675 | update := false 676 | select { 677 | case <-s.stopped: 678 | s.setState(Stopped) 679 | return 680 | 681 | case e := <-s.c: 682 | switch req := e.target.(type) { 683 | case JoinCommand: 684 | //If no log entries exist and a self-join command is issued 685 | //then immediately become leader and commit entry. 686 | if s.log.currentIndex() == 0 && req.NodeName() == s.Name() { 687 | s.debugln("selfjoin and promote to leader") 688 | s.setState(Leader) 689 | s.processCommand(req, e) 690 | } else { 691 | err = NotLeaderError 692 | } 693 | case *AppendEntriesRequest: 694 | // If heartbeats get too close to the election timeout then send an event. 695 | elapsedTime := time.Now().Sub(since) 696 | if elapsedTime > time.Duration(float64(electionTimeout)*ElectionTimeoutThresholdPercent) { 697 | s.DispatchEvent(newEvent(ElectionTimeoutThresholdEventType, elapsedTime, nil)) 698 | } 699 | e.returnValue, update = s.processAppendEntriesRequest(req) 700 | case *RequestVoteRequest: 701 | e.returnValue, update = s.processRequestVoteRequest(req) 702 | case *SnapshotRequest: 703 | e.returnValue = s.processSnapshotRequest(req) 704 | default: 705 | err = NotLeaderError 706 | } 707 | // Callback to event. 708 | e.c <- err 709 | 710 | case <-timeoutChan: 711 | // only allow synced follower to promote to candidate 712 | if s.promotable() { 713 | s.setState(Candidate) 714 | } else { 715 | update = true 716 | } 717 | } 718 | 719 | // Converts to candidate if election timeout elapses without either: 720 | // 1.Receiving valid AppendEntries RPC, or 721 | // 2.Granting vote to candidate 722 | if update { 723 | since = time.Now() 724 | timeoutChan = afterBetween(s.ElectionTimeout(), s.ElectionTimeout()*2) 725 | } 726 | } 727 | } 728 | 729 | // The event loop that is run when the server is in a Candidate state. 730 | func (s *server) candidateLoop() { 731 | // Clear leader value. 732 | prevLeader := s.leader 733 | s.leader = "" 734 | if prevLeader != s.leader { 735 | s.DispatchEvent(newEvent(LeaderChangeEventType, s.leader, prevLeader)) 736 | } 737 | 738 | lastLogIndex, lastLogTerm := s.log.lastInfo() 739 | doVote := true 740 | votesGranted := 0 741 | var timeoutChan <-chan time.Time 742 | var respChan chan *RequestVoteResponse 743 | 744 | for s.State() == Candidate { 745 | if doVote { 746 | // Increment current term, vote for self. 747 | s.currentTerm++ 748 | s.votedFor = s.name 749 | 750 | // Send RequestVote RPCs to all other servers. 751 | respChan = make(chan *RequestVoteResponse, len(s.peers)) 752 | for _, peer := range s.peers { 753 | s.routineGroup.Add(1) 754 | go func(peer *Peer) { 755 | defer s.routineGroup.Done() 756 | peer.sendVoteRequest(newRequestVoteRequest(s.currentTerm, s.name, lastLogIndex, lastLogTerm), respChan) 757 | }(peer) 758 | } 759 | 760 | // Wait for either: 761 | // * Votes received from majority of servers: become leader 762 | // * AppendEntries RPC received from new leader: step down. 763 | // * Election timeout elapses without election resolution: increment term, start new election 764 | // * Discover higher term: step down (§5.1) 765 | votesGranted = 1 766 | timeoutChan = afterBetween(s.ElectionTimeout(), s.ElectionTimeout()*2) 767 | doVote = false 768 | } 769 | 770 | // If we received enough votes then stop waiting for more votes. 771 | // And return from the candidate loop 772 | if votesGranted == s.QuorumSize() { 773 | s.debugln("server.candidate.recv.enough.votes") 774 | s.setState(Leader) 775 | return 776 | } 777 | 778 | // Collect votes from peers. 779 | select { 780 | case <-s.stopped: 781 | s.setState(Stopped) 782 | return 783 | 784 | case resp := <-respChan: 785 | if success := s.processVoteResponse(resp); success { 786 | s.debugln("server.candidate.vote.granted: ", votesGranted) 787 | votesGranted++ 788 | } 789 | 790 | case e := <-s.c: 791 | var err error 792 | switch req := e.target.(type) { 793 | case Command: 794 | err = NotLeaderError 795 | case *AppendEntriesRequest: 796 | e.returnValue, _ = s.processAppendEntriesRequest(req) 797 | case *RequestVoteRequest: 798 | e.returnValue, _ = s.processRequestVoteRequest(req) 799 | } 800 | 801 | // Callback to event. 802 | e.c <- err 803 | 804 | case <-timeoutChan: 805 | doVote = true 806 | } 807 | } 808 | } 809 | 810 | // The event loop that is run when the server is in a Leader state. 811 | func (s *server) leaderLoop() { 812 | logIndex, _ := s.log.lastInfo() 813 | 814 | // Update the peers prevLogIndex to leader's lastLogIndex and start heartbeat. 815 | s.debugln("leaderLoop.set.PrevIndex to ", logIndex) 816 | for _, peer := range s.peers { 817 | peer.setPrevLogIndex(logIndex) 818 | peer.startHeartbeat() 819 | } 820 | 821 | // Commit a NOP after the server becomes leader. From the Raft paper: 822 | // "Upon election: send initial empty AppendEntries RPCs (heartbeat) to 823 | // each server; repeat during idle periods to prevent election timeouts 824 | // (§5.2)". The heartbeats started above do the "idle" period work. 825 | s.routineGroup.Add(1) 826 | go func() { 827 | defer s.routineGroup.Done() 828 | s.Do(NOPCommand{}) 829 | }() 830 | 831 | // Begin to collect response from followers 832 | for s.State() == Leader { 833 | var err error 834 | select { 835 | case <-s.stopped: 836 | // Stop all peers before stop 837 | for _, peer := range s.peers { 838 | peer.stopHeartbeat(false) 839 | } 840 | s.setState(Stopped) 841 | return 842 | 843 | case e := <-s.c: 844 | switch req := e.target.(type) { 845 | case Command: 846 | s.processCommand(req, e) 847 | continue 848 | case *AppendEntriesRequest: 849 | e.returnValue, _ = s.processAppendEntriesRequest(req) 850 | case *AppendEntriesResponse: 851 | s.processAppendEntriesResponse(req) 852 | case *RequestVoteRequest: 853 | e.returnValue, _ = s.processRequestVoteRequest(req) 854 | } 855 | 856 | // Callback to event. 857 | e.c <- err 858 | } 859 | } 860 | 861 | s.syncedPeer = nil 862 | } 863 | 864 | func (s *server) snapshotLoop() { 865 | for s.State() == Snapshotting { 866 | var err error 867 | select { 868 | case <-s.stopped: 869 | s.setState(Stopped) 870 | return 871 | 872 | case e := <-s.c: 873 | switch req := e.target.(type) { 874 | case Command: 875 | err = NotLeaderError 876 | case *AppendEntriesRequest: 877 | e.returnValue, _ = s.processAppendEntriesRequest(req) 878 | case *RequestVoteRequest: 879 | e.returnValue, _ = s.processRequestVoteRequest(req) 880 | case *SnapshotRecoveryRequest: 881 | e.returnValue = s.processSnapshotRecoveryRequest(req) 882 | } 883 | // Callback to event. 884 | e.c <- err 885 | } 886 | } 887 | } 888 | 889 | //-------------------------------------- 890 | // Commands 891 | //-------------------------------------- 892 | 893 | // Attempts to execute a command and replicate it. The function will return 894 | // when the command has been successfully committed or an error has occurred. 895 | 896 | func (s *server) Do(command Command) (interface{}, error) { 897 | return s.send(command) 898 | } 899 | 900 | // Processes a command. 901 | func (s *server) processCommand(command Command, e *ev) { 902 | s.debugln("server.command.process") 903 | 904 | // Create an entry for the command in the log. 905 | entry, err := s.log.createEntry(s.currentTerm, command, e) 906 | 907 | if err != nil { 908 | s.debugln("server.command.log.entry.error:", err) 909 | e.c <- err 910 | return 911 | } 912 | 913 | if err := s.log.appendEntry(entry); err != nil { 914 | s.debugln("server.command.log.error:", err) 915 | e.c <- err 916 | return 917 | } 918 | 919 | s.syncedPeer[s.Name()] = true 920 | if len(s.peers) == 0 { 921 | commitIndex := s.log.currentIndex() 922 | s.log.setCommitIndex(commitIndex) 923 | s.debugln("commit index ", commitIndex) 924 | } 925 | } 926 | 927 | //-------------------------------------- 928 | // Append Entries 929 | //-------------------------------------- 930 | 931 | // Appends zero or more log entry from the leader to this server. 932 | func (s *server) AppendEntries(req *AppendEntriesRequest) *AppendEntriesResponse { 933 | ret, _ := s.send(req) 934 | resp, _ := ret.(*AppendEntriesResponse) 935 | return resp 936 | } 937 | 938 | // Processes the "append entries" request. 939 | func (s *server) processAppendEntriesRequest(req *AppendEntriesRequest) (*AppendEntriesResponse, bool) { 940 | s.traceln("server.ae.process") 941 | 942 | if req.Term < s.currentTerm { 943 | s.debugln("server.ae.error: stale term") 944 | return newAppendEntriesResponse(s.currentTerm, false, s.log.currentIndex(), s.log.CommitIndex()), false 945 | } 946 | 947 | if req.Term == s.currentTerm { 948 | _assert(s.State() != Leader, "leader.elected.at.same.term.%d\n", s.currentTerm) 949 | 950 | // step-down to follower when it is a candidate 951 | if s.state == Candidate { 952 | // change state to follower 953 | s.setState(Follower) 954 | } 955 | 956 | // discover new leader when candidate 957 | // save leader name when follower 958 | s.leader = req.LeaderName 959 | } else { 960 | // Update term and leader. 961 | s.updateCurrentTerm(req.Term, req.LeaderName) 962 | } 963 | 964 | // Reject if log doesn't contain a matching previous entry. 965 | if err := s.log.truncate(req.PrevLogIndex, req.PrevLogTerm); err != nil { 966 | s.debugln("server.ae.truncate.error: ", err) 967 | return newAppendEntriesResponse(s.currentTerm, false, s.log.currentIndex(), s.log.CommitIndex()), true 968 | } 969 | 970 | // Append entries to the log. 971 | if err := s.log.appendEntries(req.Entries); err != nil { 972 | s.debugln("server.ae.append.error: ", err) 973 | return newAppendEntriesResponse(s.currentTerm, false, s.log.currentIndex(), s.log.CommitIndex()), true 974 | } 975 | 976 | // Commit up to the commit index. 977 | if err := s.log.setCommitIndex(req.CommitIndex); err != nil { 978 | s.debugln("server.ae.commit.error: ", err) 979 | return newAppendEntriesResponse(s.currentTerm, false, s.log.currentIndex(), s.log.CommitIndex()), true 980 | } 981 | 982 | // once the server appended and committed all the log entries from the leader 983 | 984 | return newAppendEntriesResponse(s.currentTerm, true, s.log.currentIndex(), s.log.CommitIndex()), true 985 | } 986 | 987 | // Processes the "append entries" response from the peer. This is only 988 | // processed when the server is a leader. Responses received during other 989 | // states are dropped. 990 | func (s *server) processAppendEntriesResponse(resp *AppendEntriesResponse) { 991 | // If we find a higher term then change to a follower and exit. 992 | if resp.Term() > s.Term() { 993 | s.updateCurrentTerm(resp.Term(), "") 994 | return 995 | } 996 | 997 | // panic response if it's not successful. 998 | if !resp.Success() { 999 | return 1000 | } 1001 | 1002 | // if one peer successfully append a log from the leader term, 1003 | // we add it to the synced list 1004 | if resp.append == true { 1005 | s.syncedPeer[resp.peer] = true 1006 | } 1007 | 1008 | // Increment the commit count to make sure we have a quorum before committing. 1009 | if len(s.syncedPeer) < s.QuorumSize() { 1010 | return 1011 | } 1012 | 1013 | // Determine the committed index that a majority has. 1014 | var indices []uint64 1015 | indices = append(indices, s.log.currentIndex()) 1016 | for _, peer := range s.peers { 1017 | indices = append(indices, peer.getPrevLogIndex()) 1018 | } 1019 | sort.Sort(sort.Reverse(uint64Slice(indices))) 1020 | 1021 | // We can commit up to the index which the majority of the members have appended. 1022 | commitIndex := indices[s.QuorumSize()-1] 1023 | committedIndex := s.log.commitIndex 1024 | 1025 | if commitIndex > committedIndex { 1026 | // leader needs to do a fsync before committing log entries 1027 | s.log.sync() 1028 | s.log.setCommitIndex(commitIndex) 1029 | s.debugln("commit index ", commitIndex) 1030 | } 1031 | } 1032 | 1033 | // processVoteReponse processes a vote request: 1034 | // 1. if the vote is granted for the current term of the candidate, return true 1035 | // 2. if the vote is denied due to smaller term, update the term of this server 1036 | // which will also cause the candidate to step-down, and return false. 1037 | // 3. if the vote is for a smaller term, ignore it and return false. 1038 | func (s *server) processVoteResponse(resp *RequestVoteResponse) bool { 1039 | if resp.VoteGranted && resp.Term == s.currentTerm { 1040 | return true 1041 | } 1042 | 1043 | if resp.Term > s.currentTerm { 1044 | s.debugln("server.candidate.vote.failed") 1045 | s.updateCurrentTerm(resp.Term, "") 1046 | } else { 1047 | s.debugln("server.candidate.vote: denied") 1048 | } 1049 | return false 1050 | } 1051 | 1052 | //-------------------------------------- 1053 | // Request Vote 1054 | //-------------------------------------- 1055 | 1056 | // Requests a vote from a server. A vote can be obtained if the vote's term is 1057 | // at the server's current term and the server has not made a vote yet. A vote 1058 | // can also be obtained if the term is greater than the server's current term. 1059 | func (s *server) RequestVote(req *RequestVoteRequest) *RequestVoteResponse { 1060 | ret, _ := s.send(req) 1061 | resp, _ := ret.(*RequestVoteResponse) 1062 | return resp 1063 | } 1064 | 1065 | // Processes a "request vote" request. 1066 | func (s *server) processRequestVoteRequest(req *RequestVoteRequest) (*RequestVoteResponse, bool) { 1067 | 1068 | // If the request is coming from an old term then reject it. 1069 | if req.Term < s.Term() { 1070 | s.debugln("server.rv.deny.vote: cause stale term") 1071 | return newRequestVoteResponse(s.currentTerm, false), false 1072 | } 1073 | 1074 | // If the term of the request peer is larger than this node, update the term 1075 | // If the term is equal and we've already voted for a different candidate then 1076 | // don't vote for this candidate. 1077 | if req.Term > s.Term() { 1078 | s.updateCurrentTerm(req.Term, "") 1079 | } else if s.votedFor != "" && s.votedFor != req.CandidateName { 1080 | s.debugln("server.deny.vote: cause duplicate vote: ", req.CandidateName, 1081 | " already vote for ", s.votedFor) 1082 | return newRequestVoteResponse(s.currentTerm, false), false 1083 | } 1084 | 1085 | // If the candidate's log is not at least as up-to-date as our last log then don't vote. 1086 | lastIndex, lastTerm := s.log.lastInfo() 1087 | if lastIndex > req.LastLogIndex || lastTerm > req.LastLogTerm { 1088 | s.debugln("server.deny.vote: cause out of date log: ", req.CandidateName, 1089 | "Index :[", lastIndex, "]", " [", req.LastLogIndex, "]", 1090 | "Term :[", lastTerm, "]", " [", req.LastLogTerm, "]") 1091 | return newRequestVoteResponse(s.currentTerm, false), false 1092 | } 1093 | 1094 | // If we made it this far then cast a vote and reset our election time out. 1095 | s.debugln("server.rv.vote: ", s.name, " votes for", req.CandidateName, "at term", req.Term) 1096 | s.votedFor = req.CandidateName 1097 | 1098 | return newRequestVoteResponse(s.currentTerm, true), true 1099 | } 1100 | 1101 | //-------------------------------------- 1102 | // Membership 1103 | //-------------------------------------- 1104 | 1105 | // Adds a peer to the server. 1106 | func (s *server) AddPeer(name string, connectiongString string) error { 1107 | s.debugln("server.peer.add: ", name, len(s.peers)) 1108 | 1109 | // Do not allow peers to be added twice. 1110 | if s.peers[name] != nil { 1111 | return nil 1112 | } 1113 | 1114 | // Skip the Peer if it has the same name as the Server 1115 | if s.name != name { 1116 | peer := newPeer(s, name, connectiongString, s.heartbeatInterval) 1117 | 1118 | if s.State() == Leader { 1119 | peer.startHeartbeat() 1120 | } 1121 | 1122 | s.peers[peer.Name] = peer 1123 | 1124 | s.DispatchEvent(newEvent(AddPeerEventType, name, nil)) 1125 | } 1126 | 1127 | // Write the configuration to file. 1128 | s.writeConf() 1129 | 1130 | return nil 1131 | } 1132 | 1133 | // Removes a peer from the server. 1134 | func (s *server) RemovePeer(name string) error { 1135 | s.debugln("server.peer.remove: ", name, len(s.peers)) 1136 | 1137 | // Skip the Peer if it has the same name as the Server 1138 | if name != s.Name() { 1139 | // Return error if peer doesn't exist. 1140 | peer := s.peers[name] 1141 | if peer == nil { 1142 | return fmt.Errorf("raft: Peer not found: %s", name) 1143 | } 1144 | 1145 | // Stop peer and remove it. 1146 | if s.State() == Leader { 1147 | // We create a go routine here to avoid potential deadlock. 1148 | // We are holding log write lock when reach this line of code. 1149 | // Peer.stopHeartbeat can be blocked without go routine, if the 1150 | // target go routine (which we want to stop) is calling 1151 | // log.getEntriesAfter and waiting for log read lock. 1152 | // So we might be holding log lock and waiting for log lock, 1153 | // which lead to a deadlock. 1154 | // TODO(xiangli) refactor log lock 1155 | s.routineGroup.Add(1) 1156 | go func() { 1157 | defer s.routineGroup.Done() 1158 | peer.stopHeartbeat(true) 1159 | }() 1160 | } 1161 | 1162 | delete(s.peers, name) 1163 | 1164 | s.DispatchEvent(newEvent(RemovePeerEventType, name, nil)) 1165 | } 1166 | 1167 | // Write the configuration to file. 1168 | s.writeConf() 1169 | 1170 | return nil 1171 | } 1172 | 1173 | //-------------------------------------- 1174 | // Log compaction 1175 | //-------------------------------------- 1176 | 1177 | func (s *server) TakeSnapshot() error { 1178 | if s.stateMachine == nil { 1179 | return errors.New("Snapshot: Cannot create snapshot. Missing state machine.") 1180 | } 1181 | 1182 | // Shortcut without lock 1183 | // Exit if the server is currently creating a snapshot. 1184 | if s.pendingSnapshot != nil { 1185 | return errors.New("Snapshot: Last snapshot is not finished.") 1186 | } 1187 | 1188 | // TODO: acquire the lock and no more committed is allowed 1189 | // This will be done after finishing refactoring heartbeat 1190 | s.debugln("take.snapshot") 1191 | 1192 | lastIndex, lastTerm := s.log.commitInfo() 1193 | 1194 | // check if there is log has been committed since the 1195 | // last snapshot. 1196 | if lastIndex == s.log.startIndex { 1197 | return nil 1198 | } 1199 | 1200 | path := s.SnapshotPath(lastIndex, lastTerm) 1201 | // Attach snapshot to pending snapshot and save it to disk. 1202 | s.pendingSnapshot = &Snapshot{lastIndex, lastTerm, nil, nil, path} 1203 | 1204 | state, err := s.stateMachine.Save() 1205 | if err != nil { 1206 | return err 1207 | } 1208 | 1209 | // Clone the list of peers. 1210 | peers := make([]*Peer, 0, len(s.peers)+1) 1211 | for _, peer := range s.peers { 1212 | peers = append(peers, peer.clone()) 1213 | } 1214 | peers = append(peers, &Peer{Name: s.Name(), ConnectionString: s.connectionString}) 1215 | 1216 | // Attach snapshot to pending snapshot and save it to disk. 1217 | s.pendingSnapshot.Peers = peers 1218 | s.pendingSnapshot.State = state 1219 | s.saveSnapshot() 1220 | 1221 | // We keep some log entries after the snapshot. 1222 | // We do not want to send the whole snapshot to the slightly slow machines 1223 | if lastIndex-s.log.startIndex > NumberOfLogEntriesAfterSnapshot { 1224 | compactIndex := lastIndex - NumberOfLogEntriesAfterSnapshot 1225 | compactTerm := s.log.getEntry(compactIndex).Term() 1226 | s.log.compact(compactIndex, compactTerm) 1227 | } 1228 | 1229 | return nil 1230 | } 1231 | 1232 | // Retrieves the log path for the server. 1233 | func (s *server) saveSnapshot() error { 1234 | if s.pendingSnapshot == nil { 1235 | return errors.New("pendingSnapshot.is.nil") 1236 | } 1237 | 1238 | // Write snapshot to disk. 1239 | if err := s.pendingSnapshot.save(); err != nil { 1240 | return err 1241 | } 1242 | 1243 | // Swap the current and last snapshots. 1244 | tmp := s.snapshot 1245 | s.snapshot = s.pendingSnapshot 1246 | 1247 | // Delete the previous snapshot if there is any change 1248 | if tmp != nil && !(tmp.LastIndex == s.snapshot.LastIndex && tmp.LastTerm == s.snapshot.LastTerm) { 1249 | tmp.remove() 1250 | } 1251 | s.pendingSnapshot = nil 1252 | 1253 | return nil 1254 | } 1255 | 1256 | // Retrieves the log path for the server. 1257 | func (s *server) SnapshotPath(lastIndex uint64, lastTerm uint64) string { 1258 | return path.Join(s.path, "snapshot", fmt.Sprintf("%v_%v.ss", lastTerm, lastIndex)) 1259 | } 1260 | 1261 | func (s *server) RequestSnapshot(req *SnapshotRequest) *SnapshotResponse { 1262 | ret, _ := s.send(req) 1263 | resp, _ := ret.(*SnapshotResponse) 1264 | return resp 1265 | } 1266 | 1267 | func (s *server) processSnapshotRequest(req *SnapshotRequest) *SnapshotResponse { 1268 | // If the follower’s log contains an entry at the snapshot’s last index with a term 1269 | // that matches the snapshot’s last term, then the follower already has all the 1270 | // information found in the snapshot and can reply false. 1271 | entry := s.log.getEntry(req.LastIndex) 1272 | 1273 | if entry != nil && entry.Term() == req.LastTerm { 1274 | return newSnapshotResponse(false) 1275 | } 1276 | 1277 | // Update state. 1278 | s.setState(Snapshotting) 1279 | 1280 | return newSnapshotResponse(true) 1281 | } 1282 | 1283 | func (s *server) SnapshotRecoveryRequest(req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse { 1284 | ret, _ := s.send(req) 1285 | resp, _ := ret.(*SnapshotRecoveryResponse) 1286 | return resp 1287 | } 1288 | 1289 | func (s *server) processSnapshotRecoveryRequest(req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse { 1290 | // Recover state sent from request. 1291 | if err := s.stateMachine.Recovery(req.State); err != nil { 1292 | panic("cannot recover from previous state") 1293 | } 1294 | 1295 | // Recover the cluster configuration. 1296 | s.peers = make(map[string]*Peer) 1297 | for _, peer := range req.Peers { 1298 | s.AddPeer(peer.Name, peer.ConnectionString) 1299 | } 1300 | 1301 | // Update log state. 1302 | s.currentTerm = req.LastTerm 1303 | s.log.updateCommitIndex(req.LastIndex) 1304 | 1305 | // Create local snapshot. 1306 | s.pendingSnapshot = &Snapshot{req.LastIndex, req.LastTerm, req.Peers, req.State, s.SnapshotPath(req.LastIndex, req.LastTerm)} 1307 | s.saveSnapshot() 1308 | 1309 | // Clear the previous log entries. 1310 | s.log.compact(req.LastIndex, req.LastTerm) 1311 | 1312 | return newSnapshotRecoveryResponse(req.LastTerm, true, req.LastIndex) 1313 | } 1314 | 1315 | // Load a snapshot at restart 1316 | func (s *server) LoadSnapshot() error { 1317 | // Open snapshot/ directory. 1318 | dir, err := os.OpenFile(path.Join(s.path, "snapshot"), os.O_RDONLY, 0) 1319 | if err != nil { 1320 | s.debugln("cannot.open.snapshot: ", err) 1321 | return err 1322 | } 1323 | 1324 | // Retrieve a list of all snapshots. 1325 | filenames, err := dir.Readdirnames(-1) 1326 | if err != nil { 1327 | dir.Close() 1328 | panic(err) 1329 | } 1330 | dir.Close() 1331 | 1332 | if len(filenames) == 0 { 1333 | s.debugln("no.snapshot.to.load") 1334 | return nil 1335 | } 1336 | 1337 | // Grab the latest snapshot. 1338 | sort.Strings(filenames) 1339 | snapshotPath := path.Join(s.path, "snapshot", filenames[len(filenames)-1]) 1340 | 1341 | // Read snapshot data. 1342 | file, err := os.OpenFile(snapshotPath, os.O_RDONLY, 0) 1343 | if err != nil { 1344 | return err 1345 | } 1346 | defer file.Close() 1347 | 1348 | // Check checksum. 1349 | var checksum uint32 1350 | n, err := fmt.Fscanf(file, "%08x\n", &checksum) 1351 | if err != nil { 1352 | return err 1353 | } else if n != 1 { 1354 | return errors.New("checksum.err: bad.snapshot.file") 1355 | } 1356 | 1357 | // Load remaining snapshot contents. 1358 | b, err := ioutil.ReadAll(file) 1359 | if err != nil { 1360 | return err 1361 | } 1362 | 1363 | // Generate checksum. 1364 | byteChecksum := crc32.ChecksumIEEE(b) 1365 | if uint32(checksum) != byteChecksum { 1366 | s.debugln(checksum, " ", byteChecksum) 1367 | return errors.New("bad snapshot file") 1368 | } 1369 | 1370 | // Decode snapshot. 1371 | if err = json.Unmarshal(b, &s.snapshot); err != nil { 1372 | s.debugln("unmarshal.snapshot.error: ", err) 1373 | return err 1374 | } 1375 | 1376 | // Recover snapshot into state machine. 1377 | if err = s.stateMachine.Recovery(s.snapshot.State); err != nil { 1378 | s.debugln("recovery.snapshot.error: ", err) 1379 | return err 1380 | } 1381 | 1382 | // Recover cluster configuration. 1383 | for _, peer := range s.snapshot.Peers { 1384 | s.AddPeer(peer.Name, peer.ConnectionString) 1385 | } 1386 | 1387 | // Update log state. 1388 | s.log.startTerm = s.snapshot.LastTerm 1389 | s.log.startIndex = s.snapshot.LastIndex 1390 | s.log.updateCommitIndex(s.snapshot.LastIndex) 1391 | 1392 | return err 1393 | } 1394 | 1395 | //-------------------------------------- 1396 | // Config File 1397 | //-------------------------------------- 1398 | 1399 | // Flushes commit index to the disk. 1400 | // So when the raft server restarts, it will commit upto the flushed commitIndex. 1401 | func (s *server) FlushCommitIndex() { 1402 | s.debugln("server.conf.update") 1403 | // Write the configuration to file. 1404 | s.writeConf() 1405 | } 1406 | 1407 | func (s *server) writeConf() { 1408 | 1409 | peers := make([]*Peer, len(s.peers)) 1410 | 1411 | i := 0 1412 | for _, peer := range s.peers { 1413 | peers[i] = peer.clone() 1414 | i++ 1415 | } 1416 | 1417 | r := &Config{ 1418 | CommitIndex: s.log.commitIndex, 1419 | Peers: peers, 1420 | } 1421 | 1422 | b, _ := json.Marshal(r) 1423 | 1424 | confPath := path.Join(s.path, "conf") 1425 | tmpConfPath := path.Join(s.path, "conf.tmp") 1426 | 1427 | err := writeFileSynced(tmpConfPath, b, 0600) 1428 | 1429 | if err != nil { 1430 | panic(err) 1431 | } 1432 | 1433 | os.Rename(tmpConfPath, confPath) 1434 | } 1435 | 1436 | // Read the configuration for the server. 1437 | func (s *server) readConf() error { 1438 | confPath := path.Join(s.path, "conf") 1439 | s.debugln("readConf.open ", confPath) 1440 | 1441 | // open conf file 1442 | b, err := ioutil.ReadFile(confPath) 1443 | 1444 | if err != nil { 1445 | return nil 1446 | } 1447 | 1448 | conf := &Config{} 1449 | 1450 | if err = json.Unmarshal(b, conf); err != nil { 1451 | return err 1452 | } 1453 | 1454 | s.log.updateCommitIndex(conf.CommitIndex) 1455 | 1456 | return nil 1457 | } 1458 | 1459 | //-------------------------------------- 1460 | // Debugging 1461 | //-------------------------------------- 1462 | 1463 | func (s *server) debugln(v ...interface{}) { 1464 | if logLevel > Debug { 1465 | debugf("[%s Term:%d] %s", s.name, s.Term(), fmt.Sprintln(v...)) 1466 | } 1467 | } 1468 | 1469 | func (s *server) traceln(v ...interface{}) { 1470 | if logLevel > Trace { 1471 | tracef("[%s] %s", s.name, fmt.Sprintln(v...)) 1472 | } 1473 | } 1474 | -------------------------------------------------------------------------------- /server_test.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "bytes" 5 | "encoding/json" 6 | "fmt" 7 | "strconv" 8 | "sync" 9 | "testing" 10 | "time" 11 | ) 12 | 13 | //-------------------------------------- 14 | // Request Vote 15 | //-------------------------------------- 16 | 17 | // Ensure that we can request a vote from a server that has not voted. 18 | func TestServerRequestVote(t *testing.T) { 19 | server := newTestServer("1", &testTransporter{}) 20 | 21 | server.Start() 22 | if _, err := server.Do(&DefaultJoinCommand{Name: server.Name()}); err != nil { 23 | t.Fatalf("Server %s unable to join: %v", server.Name(), err) 24 | } 25 | 26 | defer server.Stop() 27 | resp := server.RequestVote(newRequestVoteRequest(1, "foo", 1, 0)) 28 | if resp.Term != 1 || !resp.VoteGranted { 29 | t.Fatalf("Invalid request vote response: %v/%v", resp.Term, resp.VoteGranted) 30 | } 31 | } 32 | 33 | // // Ensure that a vote request is denied if it comes from an old term. 34 | func TestServerRequestVoteDeniedForStaleTerm(t *testing.T) { 35 | s := newTestServer("1", &testTransporter{}) 36 | 37 | s.Start() 38 | if _, err := s.Do(&DefaultJoinCommand{Name: s.Name()}); err != nil { 39 | t.Fatalf("Server %s unable to join: %v", s.Name(), err) 40 | } 41 | 42 | s.(*server).mutex.Lock() 43 | s.(*server).currentTerm = 2 44 | s.(*server).mutex.Unlock() 45 | 46 | defer s.Stop() 47 | resp := s.RequestVote(newRequestVoteRequest(1, "foo", 1, 0)) 48 | if resp.Term != 2 || resp.VoteGranted { 49 | t.Fatalf("Invalid request vote response: %v/%v", resp.Term, resp.VoteGranted) 50 | } 51 | if s.Term() != 2 && s.State() != Follower { 52 | t.Fatalf("Server did not update term and demote: %v / %v", s.Term(), s.State()) 53 | } 54 | } 55 | 56 | // Ensure that a vote request is denied if we've already voted for a different candidate. 57 | func TestServerRequestVoteDeniedIfAlreadyVoted(t *testing.T) { 58 | s := newTestServer("1", &testTransporter{}) 59 | 60 | s.Start() 61 | if _, err := s.Do(&DefaultJoinCommand{Name: s.Name()}); err != nil { 62 | t.Fatalf("Server %s unable to join: %v", s.Name(), err) 63 | } 64 | 65 | s.(*server).mutex.Lock() 66 | s.(*server).currentTerm = 2 67 | s.(*server).mutex.Unlock() 68 | defer s.Stop() 69 | resp := s.RequestVote(newRequestVoteRequest(2, "foo", 1, 0)) 70 | if resp.Term != 2 || !resp.VoteGranted { 71 | t.Fatalf("First vote should not have been denied") 72 | } 73 | resp = s.RequestVote(newRequestVoteRequest(2, "bar", 1, 0)) 74 | if resp.Term != 2 || resp.VoteGranted { 75 | t.Fatalf("Second vote should have been denied") 76 | } 77 | } 78 | 79 | // Ensure that a vote request is approved if vote occurs in a new term. 80 | func TestServerRequestVoteApprovedIfAlreadyVotedInOlderTerm(t *testing.T) { 81 | s := newTestServer("1", &testTransporter{}) 82 | 83 | s.Start() 84 | if _, err := s.Do(&DefaultJoinCommand{Name: s.Name()}); err != nil { 85 | t.Fatalf("Server %s unable to join: %v", s.Name(), err) 86 | } 87 | 88 | time.Sleep(time.Millisecond * 100) 89 | 90 | s.(*server).mutex.Lock() 91 | s.(*server).currentTerm = 2 92 | s.(*server).mutex.Unlock() 93 | defer s.Stop() 94 | resp := s.RequestVote(newRequestVoteRequest(2, "foo", 2, 1)) 95 | if resp.Term != 2 || !resp.VoteGranted || s.VotedFor() != "foo" { 96 | t.Fatalf("First vote should not have been denied") 97 | } 98 | resp = s.RequestVote(newRequestVoteRequest(3, "bar", 2, 1)) 99 | 100 | if resp.Term != 3 || !resp.VoteGranted || s.VotedFor() != "bar" { 101 | t.Fatalf("Second vote should have been approved") 102 | } 103 | } 104 | 105 | // Ensure that a vote request is denied if the log is out of date. 106 | func TestServerRequestVoteDenyIfCandidateLogIsBehind(t *testing.T) { 107 | tmpLog := newLog() 108 | e0, _ := newLogEntry(tmpLog, nil, 1, 1, &testCommand1{Val: "foo", I: 20}) 109 | e1, _ := newLogEntry(tmpLog, nil, 2, 1, &testCommand2{X: 100}) 110 | e2, _ := newLogEntry(tmpLog, nil, 3, 2, &testCommand1{Val: "bar", I: 0}) 111 | s := newTestServerWithLog("1", &testTransporter{}, []*LogEntry{e0, e1, e2}) 112 | 113 | // start as a follower with term 2 and index 3 114 | s.Start() 115 | defer s.Stop() 116 | 117 | // request vote from term 3 with last log entry 2, 2 118 | resp := s.RequestVote(newRequestVoteRequest(3, "foo", 2, 2)) 119 | if resp.Term != 3 || resp.VoteGranted { 120 | t.Fatalf("Stale index vote should have been denied [%v/%v]", resp.Term, resp.VoteGranted) 121 | } 122 | 123 | // request vote from term 2 with last log entry 2, 3 124 | resp = s.RequestVote(newRequestVoteRequest(2, "foo", 3, 2)) 125 | if resp.Term != 3 || resp.VoteGranted { 126 | t.Fatalf("Stale term vote should have been denied [%v/%v]", resp.Term, resp.VoteGranted) 127 | } 128 | 129 | // request vote from term 3 with last log entry 2, 3 130 | resp = s.RequestVote(newRequestVoteRequest(3, "foo", 3, 2)) 131 | if resp.Term != 3 || !resp.VoteGranted { 132 | t.Fatalf("Matching log vote should have been granted") 133 | } 134 | 135 | // request vote from term 3 with last log entry 2, 4 136 | resp = s.RequestVote(newRequestVoteRequest(3, "foo", 4, 2)) 137 | if resp.Term != 3 || !resp.VoteGranted { 138 | t.Fatalf("Ahead-of-log vote should have been granted") 139 | } 140 | } 141 | 142 | func TestProcessVoteResponse(t *testing.T) { 143 | // server Term: 0, status: Leader 144 | // response Term : 1, granted 145 | // Expectation: not success 146 | // Server Term 1 status:Leader 147 | server := &server{} 148 | server.eventDispatcher = newEventDispatcher(server) 149 | server.currentTerm = 0 150 | server.state = Leader 151 | response := &RequestVoteResponse{ 152 | VoteGranted: true, 153 | Term: 1, 154 | } 155 | if success := server.processVoteResponse(response); success { 156 | t.Fatal("Process should fail if the resp's term is larger than server's") 157 | } 158 | if server.state != Follower { 159 | t.Fatal("Server should stepdown") 160 | } 161 | 162 | // server Term: 1, status: Follower 163 | // response Term: 2, granted 164 | // Expectation: not success 165 | response.Term = 2 166 | if success := server.processVoteResponse(response); success { 167 | t.Fatal("Process should fail if the resp's term is larger than server's") 168 | } 169 | if server.state != Follower { 170 | t.Fatal("Server should still be Follower") 171 | } 172 | 173 | server.currentTerm = 2 174 | // server Term: 2, status: Follower 175 | // response Term: 2 176 | // Expectation: success 177 | if success := server.processVoteResponse(response); !success { 178 | t.Fatal("Process should success if the server's term is larger than resp's") 179 | } 180 | 181 | } 182 | 183 | // //-------------------------------------- 184 | // // Promotion 185 | // //-------------------------------------- 186 | 187 | // // Ensure that we can self-promote a server to candidate, obtain votes and become a fearless leader. 188 | func TestServerPromoteSelf(t *testing.T) { 189 | e0, _ := newLogEntry(newLog(), nil, 1, 1, &testCommand1{Val: "foo", I: 20}) 190 | s := newTestServerWithLog("1", &testTransporter{}, []*LogEntry{e0}) 191 | 192 | // start as a follower 193 | s.Start() 194 | defer s.Stop() 195 | 196 | time.Sleep(2 * testElectionTimeout) 197 | 198 | if s.State() != Leader { 199 | t.Fatalf("Server self-promotion failed: %v", s.State()) 200 | } 201 | } 202 | 203 | //Ensure that we can promote a server within a cluster to a leader. 204 | func TestServerPromote(t *testing.T) { 205 | lookup := map[string]Server{} 206 | transporter := &testTransporter{} 207 | transporter.sendVoteRequestFunc = func(s Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse { 208 | return lookup[peer.Name].RequestVote(req) 209 | } 210 | transporter.sendAppendEntriesRequestFunc = func(s Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse { 211 | return lookup[peer.Name].AppendEntries(req) 212 | } 213 | servers := newTestCluster([]string{"1", "2", "3"}, transporter, lookup) 214 | 215 | servers[0].Start() 216 | servers[1].Start() 217 | servers[2].Start() 218 | 219 | time.Sleep(2 * testElectionTimeout) 220 | 221 | if servers[0].State() != Leader && servers[1].State() != Leader && servers[2].State() != Leader { 222 | t.Fatalf("No leader elected: (%s, %s, %s)", servers[0].State(), servers[1].State(), servers[2].State()) 223 | } 224 | for _, s := range servers { 225 | s.Stop() 226 | } 227 | } 228 | 229 | //-------------------------------------- 230 | // Append Entries 231 | //-------------------------------------- 232 | 233 | // Ensure we can append entries to a server. 234 | func TestServerAppendEntries(t *testing.T) { 235 | s := newTestServer("1", &testTransporter{}) 236 | 237 | s.SetHeartbeatInterval(time.Second * 10) 238 | s.Start() 239 | defer s.Stop() 240 | 241 | // Append single entry. 242 | e, _ := newLogEntry(nil, nil, 1, 1, &testCommand1{Val: "foo", I: 10}) 243 | entries := []*LogEntry{e} 244 | resp := s.AppendEntries(newAppendEntriesRequest(1, 0, 0, 0, "ldr", entries)) 245 | if resp.Term() != 1 || !resp.Success() { 246 | t.Fatalf("AppendEntries failed: %v/%v", resp.Term, resp.Success) 247 | } 248 | if index, term := s.(*server).log.commitInfo(); index != 0 || term != 0 { 249 | t.Fatalf("Invalid commit info [IDX=%v, TERM=%v]", index, term) 250 | } 251 | 252 | // Append multiple entries + commit the last one. 253 | e1, _ := newLogEntry(nil, nil, 2, 1, &testCommand1{Val: "bar", I: 20}) 254 | e2, _ := newLogEntry(nil, nil, 3, 1, &testCommand1{Val: "baz", I: 30}) 255 | entries = []*LogEntry{e1, e2} 256 | resp = s.AppendEntries(newAppendEntriesRequest(1, 1, 1, 1, "ldr", entries)) 257 | if resp.Term() != 1 || !resp.Success() { 258 | t.Fatalf("AppendEntries failed: %v/%v", resp.Term, resp.Success) 259 | } 260 | if index, term := s.(*server).log.commitInfo(); index != 1 || term != 1 { 261 | t.Fatalf("Invalid commit info [IDX=%v, TERM=%v]", index, term) 262 | } 263 | 264 | // Send zero entries and commit everything. 265 | resp = s.AppendEntries(newAppendEntriesRequest(2, 3, 1, 3, "ldr", []*LogEntry{})) 266 | if resp.Term() != 2 || !resp.Success() { 267 | t.Fatalf("AppendEntries failed: %v/%v", resp.Term, resp.Success) 268 | } 269 | if index, term := s.(*server).log.commitInfo(); index != 3 || term != 1 { 270 | t.Fatalf("Invalid commit info [IDX=%v, TERM=%v]", index, term) 271 | } 272 | } 273 | 274 | //Ensure that entries with stale terms are rejected. 275 | func TestServerAppendEntriesWithStaleTermsAreRejected(t *testing.T) { 276 | s := newTestServer("1", &testTransporter{}) 277 | 278 | s.Start() 279 | 280 | defer s.Stop() 281 | s.(*server).mutex.Lock() 282 | s.(*server).currentTerm = 2 283 | s.(*server).mutex.Unlock() 284 | 285 | // Append single entry. 286 | e, _ := newLogEntry(nil, nil, 1, 1, &testCommand1{Val: "foo", I: 10}) 287 | entries := []*LogEntry{e} 288 | resp := s.AppendEntries(newAppendEntriesRequest(1, 0, 0, 0, "ldr", entries)) 289 | if resp.Term() != 2 || resp.Success() { 290 | t.Fatalf("AppendEntries should have failed: %v/%v", resp.Term, resp.Success) 291 | } 292 | if index, term := s.(*server).log.commitInfo(); index != 0 || term != 0 { 293 | t.Fatalf("Invalid commit info [IDX=%v, TERM=%v]", index, term) 294 | } 295 | } 296 | 297 | // Ensure that we reject entries if the commit log is different. 298 | func TestServerAppendEntriesRejectedIfAlreadyCommitted(t *testing.T) { 299 | s := newTestServer("1", &testTransporter{}) 300 | s.Start() 301 | defer s.Stop() 302 | 303 | // Append single entry + commit. 304 | e1, _ := newLogEntry(nil, nil, 1, 1, &testCommand1{Val: "foo", I: 10}) 305 | e2, _ := newLogEntry(nil, nil, 2, 1, &testCommand1{Val: "foo", I: 15}) 306 | entries := []*LogEntry{e1, e2} 307 | resp := s.AppendEntries(newAppendEntriesRequest(1, 0, 0, 2, "ldr", entries)) 308 | if resp.Term() != 1 || !resp.Success() { 309 | t.Fatalf("AppendEntries failed: %v/%v", resp.Term, resp.Success) 310 | } 311 | 312 | // Append entry again (post-commit). 313 | e, _ := newLogEntry(nil, nil, 2, 1, &testCommand1{Val: "bar", I: 20}) 314 | entries = []*LogEntry{e} 315 | resp = s.AppendEntries(newAppendEntriesRequest(1, 2, 1, 1, "ldr", entries)) 316 | if resp.Term() != 1 || resp.Success() { 317 | t.Fatalf("AppendEntries should have failed: %v/%v", resp.Term, resp.Success) 318 | } 319 | } 320 | 321 | // Ensure that we uncommitted entries are rolled back if new entries overwrite them. 322 | func TestServerAppendEntriesOverwritesUncommittedEntries(t *testing.T) { 323 | s := newTestServer("1", &testTransporter{}) 324 | s.Start() 325 | defer s.Stop() 326 | 327 | entry1, _ := newLogEntry(s.(*server).log, nil, 1, 1, &testCommand1{Val: "foo", I: 10}) 328 | entry2, _ := newLogEntry(s.(*server).log, nil, 2, 1, &testCommand1{Val: "foo", I: 15}) 329 | entry3, _ := newLogEntry(s.(*server).log, nil, 2, 2, &testCommand1{Val: "bar", I: 20}) 330 | 331 | // Append single entry + commit. 332 | entries := []*LogEntry{entry1, entry2} 333 | resp := s.AppendEntries(newAppendEntriesRequest(1, 0, 0, 1, "ldr", entries)) 334 | if resp.Term() != 1 || !resp.Success() || s.(*server).log.commitIndex != 1 { 335 | t.Fatalf("AppendEntries failed: %v/%v", resp.Term, resp.Success) 336 | } 337 | 338 | for i, entry := range s.(*server).log.entries { 339 | if entry.Term() != entries[i].Term() || entry.Index() != entries[i].Index() || !bytes.Equal(entry.Command(), entries[i].Command()) { 340 | t.Fatalf("AppendEntries failed: %v/%v", resp.Term, resp.Success) 341 | } 342 | } 343 | 344 | // Append entry that overwrites the second (uncommitted) entry. 345 | entries = []*LogEntry{entry3} 346 | resp = s.AppendEntries(newAppendEntriesRequest(2, 1, 1, 2, "ldr", entries)) 347 | if resp.Term() != 2 || !resp.Success() || s.(*server).log.commitIndex != 2 { 348 | t.Fatalf("AppendEntries should have succeeded: %v/%v", resp.Term, resp.Success) 349 | } 350 | 351 | entries = []*LogEntry{entry1, entry3} 352 | for i, entry := range s.(*server).log.entries { 353 | if entry.Term() != entries[i].Term() || entry.Index() != entries[i].Index() || !bytes.Equal(entry.Command(), entries[i].Command()) { 354 | t.Fatalf("AppendEntries failed: %v/%v", resp.Term, resp.Success) 355 | } 356 | } 357 | } 358 | 359 | //-------------------------------------- 360 | // Command Execution 361 | //-------------------------------------- 362 | 363 | // Ensure that a follower cannot execute a command. 364 | func TestServerDenyCommandExecutionWhenFollower(t *testing.T) { 365 | s := newTestServer("1", &testTransporter{}) 366 | s.Start() 367 | defer s.Stop() 368 | var err error 369 | if _, err = s.Do(&testCommand1{Val: "foo", I: 10}); err != NotLeaderError { 370 | t.Fatalf("Expected error: %v, got: %v", NotLeaderError, err) 371 | } 372 | } 373 | 374 | //-------------------------------------- 375 | // Recovery 376 | //-------------------------------------- 377 | 378 | // Ensure that a follower cannot execute a command. 379 | func TestServerRecoverFromPreviousLogAndConf(t *testing.T) { 380 | // Initialize the servers. 381 | var mutex sync.RWMutex 382 | servers := map[string]Server{} 383 | 384 | transporter := &testTransporter{} 385 | transporter.sendVoteRequestFunc = func(s Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse { 386 | mutex.RLock() 387 | target := servers[peer.Name] 388 | mutex.RUnlock() 389 | 390 | b, _ := json.Marshal(req) 391 | clonedReq := &RequestVoteRequest{} 392 | json.Unmarshal(b, clonedReq) 393 | 394 | return target.RequestVote(clonedReq) 395 | } 396 | transporter.sendAppendEntriesRequestFunc = func(s Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse { 397 | mutex.RLock() 398 | target := servers[peer.Name] 399 | mutex.RUnlock() 400 | 401 | b, _ := json.Marshal(req) 402 | clonedReq := &AppendEntriesRequest{} 403 | json.Unmarshal(b, clonedReq) 404 | 405 | return target.AppendEntries(clonedReq) 406 | } 407 | 408 | disTransporter := &testTransporter{} 409 | disTransporter.sendVoteRequestFunc = func(s Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse { 410 | return nil 411 | } 412 | disTransporter.sendAppendEntriesRequestFunc = func(s Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse { 413 | return nil 414 | } 415 | 416 | var names []string 417 | var paths = make(map[string]string) 418 | 419 | n := 5 420 | 421 | // add n servers 422 | for i := 1; i <= n; i++ { 423 | names = append(names, strconv.Itoa(i)) 424 | } 425 | 426 | var leader Server 427 | for _, name := range names { 428 | s := newTestServer(name, transporter) 429 | 430 | mutex.Lock() 431 | servers[name] = s 432 | mutex.Unlock() 433 | paths[name] = s.Path() 434 | 435 | if name == "1" { 436 | leader = s 437 | s.SetHeartbeatInterval(testHeartbeatInterval) 438 | s.Start() 439 | time.Sleep(testHeartbeatInterval) 440 | } else { 441 | s.SetElectionTimeout(testElectionTimeout) 442 | s.SetHeartbeatInterval(testHeartbeatInterval) 443 | s.Start() 444 | time.Sleep(testHeartbeatInterval) 445 | } 446 | if _, err := leader.Do(&DefaultJoinCommand{Name: name}); err != nil { 447 | t.Fatalf("Unable to join server[%s]: %v", name, err) 448 | } 449 | 450 | } 451 | 452 | // commit some commands 453 | for i := 0; i < 10; i++ { 454 | if _, err := leader.Do(&testCommand2{X: 1}); err != nil { 455 | t.Fatalf("cannot commit command: %s", err.Error()) 456 | } 457 | } 458 | 459 | time.Sleep(2 * testHeartbeatInterval) 460 | 461 | for _, name := range names { 462 | s := servers[name] 463 | if s.CommitIndex() != 16 { 464 | t.Fatalf("%s commitIndex is invalid [%d/%d]", name, s.CommitIndex(), 16) 465 | } 466 | s.Stop() 467 | } 468 | 469 | for _, name := range names { 470 | // with old path and disable transportation 471 | s := newTestServerWithPath(name, disTransporter, paths[name]) 472 | servers[name] = s 473 | 474 | s.Start() 475 | 476 | // should only commit to the last join command 477 | if s.CommitIndex() != 6 { 478 | t.Fatalf("%s recover phase 1 commitIndex is invalid [%d/%d]", name, s.CommitIndex(), 6) 479 | } 480 | 481 | // peer conf should be recovered 482 | if len(s.Peers()) != 4 { 483 | t.Fatalf("%s recover phase 1 peer failed! [%d/%d]", name, len(s.Peers()), 4) 484 | } 485 | } 486 | 487 | // let nodes talk to each other 488 | for _, name := range names { 489 | servers[name].SetTransporter(transporter) 490 | } 491 | 492 | time.Sleep(2 * testElectionTimeout) 493 | 494 | // should commit to the previous index + 1(nop command when new leader elected) 495 | for _, name := range names { 496 | s := servers[name] 497 | if s.CommitIndex() != 17 { 498 | t.Fatalf("%s commitIndex is invalid [%d/%d]", name, s.CommitIndex(), 17) 499 | } 500 | s.Stop() 501 | } 502 | } 503 | 504 | //-------------------------------------- 505 | // Membership 506 | //-------------------------------------- 507 | 508 | // Ensure that we can start a single server and append to its log. 509 | func TestServerSingleNode(t *testing.T) { 510 | s := newTestServer("1", &testTransporter{}) 511 | if s.State() != Stopped { 512 | t.Fatalf("Unexpected server state: %v", s.State()) 513 | } 514 | 515 | s.Start() 516 | 517 | time.Sleep(testHeartbeatInterval) 518 | 519 | // Join the server to itself. 520 | if _, err := s.Do(&DefaultJoinCommand{Name: "1"}); err != nil { 521 | t.Fatalf("Unable to join: %v", err) 522 | } 523 | debugln("finish command") 524 | 525 | if s.State() != Leader { 526 | t.Fatalf("Unexpected server state: %v", s.State()) 527 | } 528 | 529 | s.Stop() 530 | 531 | if s.State() != Stopped { 532 | t.Fatalf("Unexpected server state: %v", s.State()) 533 | } 534 | } 535 | 536 | // Ensure that we can start multiple servers and determine a leader. 537 | func TestServerMultiNode(t *testing.T) { 538 | // Initialize the servers. 539 | var mutex sync.RWMutex 540 | servers := map[string]Server{} 541 | 542 | transporter := &testTransporter{} 543 | transporter.sendVoteRequestFunc = func(s Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse { 544 | mutex.RLock() 545 | target := servers[peer.Name] 546 | mutex.RUnlock() 547 | 548 | b, _ := json.Marshal(req) 549 | clonedReq := &RequestVoteRequest{} 550 | json.Unmarshal(b, clonedReq) 551 | 552 | c := make(chan *RequestVoteResponse) 553 | 554 | go func() { 555 | c <- target.RequestVote(clonedReq) 556 | }() 557 | 558 | select { 559 | case resp := <-c: 560 | return resp 561 | case <-time.After(time.Millisecond * 200): 562 | return nil 563 | } 564 | 565 | } 566 | transporter.sendAppendEntriesRequestFunc = func(s Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse { 567 | mutex.RLock() 568 | target := servers[peer.Name] 569 | mutex.RUnlock() 570 | 571 | b, _ := json.Marshal(req) 572 | clonedReq := &AppendEntriesRequest{} 573 | json.Unmarshal(b, clonedReq) 574 | 575 | c := make(chan *AppendEntriesResponse) 576 | 577 | go func() { 578 | c <- target.AppendEntries(clonedReq) 579 | }() 580 | 581 | select { 582 | case resp := <-c: 583 | return resp 584 | case <-time.After(time.Millisecond * 200): 585 | return nil 586 | } 587 | } 588 | 589 | disTransporter := &testTransporter{} 590 | disTransporter.sendVoteRequestFunc = func(s Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse { 591 | return nil 592 | } 593 | disTransporter.sendAppendEntriesRequestFunc = func(s Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse { 594 | return nil 595 | } 596 | 597 | var names []string 598 | 599 | n := 5 600 | 601 | // add n servers 602 | for i := 1; i <= n; i++ { 603 | names = append(names, strconv.Itoa(i)) 604 | } 605 | 606 | var leader Server 607 | for _, name := range names { 608 | s := newTestServer(name, transporter) 609 | defer s.Stop() 610 | 611 | mutex.Lock() 612 | servers[name] = s 613 | mutex.Unlock() 614 | 615 | if name == "1" { 616 | leader = s 617 | s.SetHeartbeatInterval(testHeartbeatInterval) 618 | s.Start() 619 | time.Sleep(testHeartbeatInterval) 620 | } else { 621 | s.SetElectionTimeout(testElectionTimeout) 622 | s.SetHeartbeatInterval(testHeartbeatInterval) 623 | s.Start() 624 | time.Sleep(testHeartbeatInterval) 625 | } 626 | if _, err := leader.Do(&DefaultJoinCommand{Name: name}); err != nil { 627 | t.Fatalf("Unable to join server[%s]: %v", name, err) 628 | } 629 | 630 | } 631 | time.Sleep(2 * testElectionTimeout) 632 | 633 | // Check that two peers exist on leader. 634 | mutex.RLock() 635 | if leader.MemberCount() != n { 636 | t.Fatalf("Expected member count to be %v, got %v", n, leader.MemberCount()) 637 | } 638 | if servers["2"].State() == Leader || servers["3"].State() == Leader { 639 | t.Fatalf("Expected leader should be 1: 2=%v, 3=%v\n", servers["2"].State(), servers["3"].State()) 640 | } 641 | mutex.RUnlock() 642 | 643 | for i := 0; i < 20; i++ { 644 | retry := 0 645 | fmt.Println("Round ", i) 646 | 647 | num := strconv.Itoa(i%(len(servers)) + 1) 648 | num_1 := strconv.Itoa((i+3)%(len(servers)) + 1) 649 | toStop := servers[num] 650 | toStop_1 := servers[num_1] 651 | 652 | // Stop the first server and wait for a re-election. 653 | time.Sleep(2 * testElectionTimeout) 654 | debugln("Disconnect ", toStop.Name()) 655 | debugln("disconnect ", num, " ", num_1) 656 | toStop.SetTransporter(disTransporter) 657 | toStop_1.SetTransporter(disTransporter) 658 | time.Sleep(2 * testElectionTimeout) 659 | // Check that either server 2 or 3 is the leader now. 660 | //mutex.Lock() 661 | 662 | leader := 0 663 | 664 | for key, value := range servers { 665 | debugln("Play begin") 666 | if key != num && key != num_1 { 667 | if value.State() == Leader { 668 | debugln("Found leader") 669 | for i := 0; i < 10; i++ { 670 | debugln("[Test] do ", value.Name()) 671 | if _, err := value.Do(&testCommand2{X: 1}); err != nil { 672 | break 673 | } 674 | debugln("[Test] Done") 675 | } 676 | debugln("Leader is ", value.Name(), " Index ", value.(*server).log.commitIndex) 677 | } 678 | debugln("Not Found leader") 679 | } 680 | } 681 | for { 682 | for key, value := range servers { 683 | if key != num && key != num_1 { 684 | if value.State() == Leader { 685 | leader++ 686 | } 687 | debugln(value.Name(), " ", value.(*server).Term(), " ", value.State()) 688 | } 689 | } 690 | 691 | if leader > 1 { 692 | if retry < 300 { 693 | debugln("retry") 694 | retry++ 695 | leader = 0 696 | time.Sleep(2 * testElectionTimeout) 697 | continue 698 | } 699 | t.Fatalf("wrong leader number %v", leader) 700 | } 701 | if leader == 0 { 702 | if retry < 300 { 703 | retry++ 704 | fmt.Println("retry 0") 705 | leader = 0 706 | time.Sleep(2 * testElectionTimeout) 707 | continue 708 | } 709 | t.Fatalf("wrong leader number %v", leader) 710 | } 711 | if leader == 1 { 712 | break 713 | } 714 | } 715 | 716 | //mutex.Unlock() 717 | 718 | toStop.SetTransporter(transporter) 719 | toStop_1.SetTransporter(transporter) 720 | } 721 | 722 | } 723 | -------------------------------------------------------------------------------- /snapshot.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "encoding/json" 5 | "fmt" 6 | "hash/crc32" 7 | "io" 8 | "io/ioutil" 9 | "os" 10 | 11 | "code.google.com/p/gogoprotobuf/proto" 12 | "github.com/goraft/raft/protobuf" 13 | ) 14 | 15 | // Snapshot represents an in-memory representation of the current state of the system. 16 | type Snapshot struct { 17 | LastIndex uint64 `json:"lastIndex"` 18 | LastTerm uint64 `json:"lastTerm"` 19 | 20 | // Cluster configuration. 21 | Peers []*Peer `json:"peers"` 22 | State []byte `json:"state"` 23 | Path string `json:"path"` 24 | } 25 | 26 | // The request sent to a server to start from the snapshot. 27 | type SnapshotRecoveryRequest struct { 28 | LeaderName string 29 | LastIndex uint64 30 | LastTerm uint64 31 | Peers []*Peer 32 | State []byte 33 | } 34 | 35 | // The response returned from a server appending entries to the log. 36 | type SnapshotRecoveryResponse struct { 37 | Term uint64 38 | Success bool 39 | CommitIndex uint64 40 | } 41 | 42 | // The request sent to a server to start from the snapshot. 43 | type SnapshotRequest struct { 44 | LeaderName string 45 | LastIndex uint64 46 | LastTerm uint64 47 | } 48 | 49 | // The response returned if the follower entered snapshot state 50 | type SnapshotResponse struct { 51 | Success bool `json:"success"` 52 | } 53 | 54 | // save writes the snapshot to file. 55 | func (ss *Snapshot) save() error { 56 | // Open the file for writing. 57 | file, err := os.OpenFile(ss.Path, os.O_CREATE|os.O_WRONLY, 0600) 58 | if err != nil { 59 | return err 60 | } 61 | defer file.Close() 62 | 63 | // Serialize to JSON. 64 | b, err := json.Marshal(ss) 65 | if err != nil { 66 | return err 67 | } 68 | 69 | // Generate checksum and write it to disk. 70 | checksum := crc32.ChecksumIEEE(b) 71 | if _, err = fmt.Fprintf(file, "%08x\n", checksum); err != nil { 72 | return err 73 | } 74 | 75 | // Write the snapshot to disk. 76 | if _, err = file.Write(b); err != nil { 77 | return err 78 | } 79 | 80 | // Ensure that the snapshot has been flushed to disk before continuing. 81 | if err := file.Sync(); err != nil { 82 | return err 83 | } 84 | 85 | return nil 86 | } 87 | 88 | // remove deletes the snapshot file. 89 | func (ss *Snapshot) remove() error { 90 | if err := os.Remove(ss.Path); err != nil { 91 | return err 92 | } 93 | return nil 94 | } 95 | 96 | // Creates a new Snapshot request. 97 | func newSnapshotRecoveryRequest(leaderName string, snapshot *Snapshot) *SnapshotRecoveryRequest { 98 | return &SnapshotRecoveryRequest{ 99 | LeaderName: leaderName, 100 | LastIndex: snapshot.LastIndex, 101 | LastTerm: snapshot.LastTerm, 102 | Peers: snapshot.Peers, 103 | State: snapshot.State, 104 | } 105 | } 106 | 107 | // Encodes the SnapshotRecoveryRequest to a buffer. Returns the number of bytes 108 | // written and any error that may have occurred. 109 | func (req *SnapshotRecoveryRequest) Encode(w io.Writer) (int, error) { 110 | 111 | protoPeers := make([]*protobuf.SnapshotRecoveryRequest_Peer, len(req.Peers)) 112 | 113 | for i, peer := range req.Peers { 114 | protoPeers[i] = &protobuf.SnapshotRecoveryRequest_Peer{ 115 | Name: proto.String(peer.Name), 116 | ConnectionString: proto.String(peer.ConnectionString), 117 | } 118 | } 119 | 120 | pb := &protobuf.SnapshotRecoveryRequest{ 121 | LeaderName: proto.String(req.LeaderName), 122 | LastIndex: proto.Uint64(req.LastIndex), 123 | LastTerm: proto.Uint64(req.LastTerm), 124 | Peers: protoPeers, 125 | State: req.State, 126 | } 127 | p, err := proto.Marshal(pb) 128 | if err != nil { 129 | return -1, err 130 | } 131 | 132 | return w.Write(p) 133 | } 134 | 135 | // Decodes the SnapshotRecoveryRequest from a buffer. Returns the number of bytes read and 136 | // any error that occurs. 137 | func (req *SnapshotRecoveryRequest) Decode(r io.Reader) (int, error) { 138 | data, err := ioutil.ReadAll(r) 139 | 140 | if err != nil { 141 | return 0, err 142 | } 143 | 144 | totalBytes := len(data) 145 | 146 | pb := &protobuf.SnapshotRecoveryRequest{} 147 | if err = proto.Unmarshal(data, pb); err != nil { 148 | return -1, err 149 | } 150 | 151 | req.LeaderName = pb.GetLeaderName() 152 | req.LastIndex = pb.GetLastIndex() 153 | req.LastTerm = pb.GetLastTerm() 154 | req.State = pb.GetState() 155 | 156 | req.Peers = make([]*Peer, len(pb.Peers)) 157 | 158 | for i, peer := range pb.Peers { 159 | req.Peers[i] = &Peer{ 160 | Name: peer.GetName(), 161 | ConnectionString: peer.GetConnectionString(), 162 | } 163 | } 164 | 165 | return totalBytes, nil 166 | } 167 | 168 | // Creates a new Snapshot response. 169 | func newSnapshotRecoveryResponse(term uint64, success bool, commitIndex uint64) *SnapshotRecoveryResponse { 170 | return &SnapshotRecoveryResponse{ 171 | Term: term, 172 | Success: success, 173 | CommitIndex: commitIndex, 174 | } 175 | } 176 | 177 | // Encode writes the response to a writer. 178 | // Returns the number of bytes written and any error that occurs. 179 | func (req *SnapshotRecoveryResponse) Encode(w io.Writer) (int, error) { 180 | pb := &protobuf.SnapshotRecoveryResponse{ 181 | Term: proto.Uint64(req.Term), 182 | Success: proto.Bool(req.Success), 183 | CommitIndex: proto.Uint64(req.CommitIndex), 184 | } 185 | p, err := proto.Marshal(pb) 186 | if err != nil { 187 | return -1, err 188 | } 189 | 190 | return w.Write(p) 191 | } 192 | 193 | // Decodes the SnapshotRecoveryResponse from a buffer. 194 | func (req *SnapshotRecoveryResponse) Decode(r io.Reader) (int, error) { 195 | data, err := ioutil.ReadAll(r) 196 | 197 | if err != nil { 198 | return 0, err 199 | } 200 | 201 | totalBytes := len(data) 202 | 203 | pb := &protobuf.SnapshotRecoveryResponse{} 204 | if err := proto.Unmarshal(data, pb); err != nil { 205 | return -1, err 206 | } 207 | 208 | req.Term = pb.GetTerm() 209 | req.Success = pb.GetSuccess() 210 | req.CommitIndex = pb.GetCommitIndex() 211 | 212 | return totalBytes, nil 213 | } 214 | 215 | // Creates a new Snapshot request. 216 | func newSnapshotRequest(leaderName string, snapshot *Snapshot) *SnapshotRequest { 217 | return &SnapshotRequest{ 218 | LeaderName: leaderName, 219 | LastIndex: snapshot.LastIndex, 220 | LastTerm: snapshot.LastTerm, 221 | } 222 | } 223 | 224 | // Encodes the SnapshotRequest to a buffer. Returns the number of bytes 225 | // written and any error that may have occurred. 226 | func (req *SnapshotRequest) Encode(w io.Writer) (int, error) { 227 | pb := &protobuf.SnapshotRequest{ 228 | LeaderName: proto.String(req.LeaderName), 229 | LastIndex: proto.Uint64(req.LastIndex), 230 | LastTerm: proto.Uint64(req.LastTerm), 231 | } 232 | p, err := proto.Marshal(pb) 233 | if err != nil { 234 | return -1, err 235 | } 236 | 237 | return w.Write(p) 238 | } 239 | 240 | // Decodes the SnapshotRequest from a buffer. Returns the number of bytes read and 241 | // any error that occurs. 242 | func (req *SnapshotRequest) Decode(r io.Reader) (int, error) { 243 | data, err := ioutil.ReadAll(r) 244 | 245 | if err != nil { 246 | return 0, err 247 | } 248 | 249 | totalBytes := len(data) 250 | 251 | pb := &protobuf.SnapshotRequest{} 252 | 253 | if err := proto.Unmarshal(data, pb); err != nil { 254 | return -1, err 255 | } 256 | 257 | req.LeaderName = pb.GetLeaderName() 258 | req.LastIndex = pb.GetLastIndex() 259 | req.LastTerm = pb.GetLastTerm() 260 | 261 | return totalBytes, nil 262 | } 263 | 264 | // Creates a new Snapshot response. 265 | func newSnapshotResponse(success bool) *SnapshotResponse { 266 | return &SnapshotResponse{ 267 | Success: success, 268 | } 269 | } 270 | 271 | // Encodes the SnapshotResponse to a buffer. Returns the number of bytes 272 | // written and any error that may have occurred. 273 | func (resp *SnapshotResponse) Encode(w io.Writer) (int, error) { 274 | pb := &protobuf.SnapshotResponse{ 275 | Success: proto.Bool(resp.Success), 276 | } 277 | p, err := proto.Marshal(pb) 278 | if err != nil { 279 | return -1, err 280 | } 281 | 282 | return w.Write(p) 283 | } 284 | 285 | // Decodes the SnapshotResponse from a buffer. Returns the number of bytes read and 286 | // any error that occurs. 287 | func (resp *SnapshotResponse) Decode(r io.Reader) (int, error) { 288 | data, err := ioutil.ReadAll(r) 289 | 290 | if err != nil { 291 | return 0, err 292 | } 293 | 294 | totalBytes := len(data) 295 | 296 | pb := &protobuf.SnapshotResponse{} 297 | if err := proto.Unmarshal(data, pb); err != nil { 298 | return -1, err 299 | } 300 | 301 | resp.Success = pb.GetSuccess() 302 | 303 | return totalBytes, nil 304 | } 305 | -------------------------------------------------------------------------------- /snapshot_test.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "testing" 5 | "time" 6 | 7 | "github.com/stretchr/testify/assert" 8 | "github.com/stretchr/testify/mock" 9 | ) 10 | 11 | // Ensure that a snapshot occurs when there are existing logs. 12 | func TestSnapshot(t *testing.T) { 13 | runServerWithMockStateMachine(Leader, func(s Server, m *mock.Mock) { 14 | m.On("Save").Return([]byte("foo"), nil) 15 | m.On("Recovery", []byte("foo")).Return(nil) 16 | 17 | s.Do(&testCommand1{}) 18 | err := s.TakeSnapshot() 19 | assert.NoError(t, err) 20 | assert.Equal(t, s.(*server).snapshot.LastIndex, uint64(2)) 21 | 22 | // Repeat to make sure new snapshot gets created. 23 | s.Do(&testCommand1{}) 24 | err = s.TakeSnapshot() 25 | assert.NoError(t, err) 26 | assert.Equal(t, s.(*server).snapshot.LastIndex, uint64(4)) 27 | 28 | // Restart server. 29 | s.Stop() 30 | // Recover from snapshot. 31 | err = s.LoadSnapshot() 32 | assert.NoError(t, err) 33 | s.Start() 34 | }) 35 | } 36 | 37 | // Ensure that a new server can recover from previous snapshot with log 38 | func TestSnapshotRecovery(t *testing.T) { 39 | runServerWithMockStateMachine(Leader, func(s Server, m *mock.Mock) { 40 | m.On("Save").Return([]byte("foo"), nil) 41 | m.On("Recovery", []byte("foo")).Return(nil) 42 | 43 | s.Do(&testCommand1{}) 44 | err := s.TakeSnapshot() 45 | assert.NoError(t, err) 46 | assert.Equal(t, s.(*server).snapshot.LastIndex, uint64(2)) 47 | 48 | // Repeat to make sure new snapshot gets created. 49 | s.Do(&testCommand1{}) 50 | 51 | // Stop the old server 52 | s.Stop() 53 | 54 | // create a new server with previous log and snapshot 55 | newS, err := NewServer("1", s.Path(), &testTransporter{}, s.StateMachine(), nil, "") 56 | // Recover from snapshot. 57 | err = newS.LoadSnapshot() 58 | assert.NoError(t, err) 59 | 60 | newS.Start() 61 | defer newS.Stop() 62 | 63 | // wait for it to become leader 64 | time.Sleep(time.Second) 65 | // ensure server load the previous log 66 | assert.Equal(t, len(newS.LogEntries()), 3, "") 67 | }) 68 | } 69 | 70 | // Ensure that a snapshot request can be sent and received. 71 | func TestSnapshotRequest(t *testing.T) { 72 | runServerWithMockStateMachine(Follower, func(s Server, m *mock.Mock) { 73 | m.On("Recovery", []byte("bar")).Return(nil) 74 | 75 | // Send snapshot request. 76 | resp := s.RequestSnapshot(&SnapshotRequest{LastIndex: 5, LastTerm: 1}) 77 | assert.Equal(t, resp.Success, true) 78 | assert.Equal(t, s.State(), Snapshotting) 79 | 80 | // Send recovery request. 81 | resp2 := s.SnapshotRecoveryRequest(&SnapshotRecoveryRequest{ 82 | LeaderName: "1", 83 | LastIndex: 5, 84 | LastTerm: 2, 85 | Peers: make([]*Peer, 0), 86 | State: []byte("bar"), 87 | }) 88 | assert.Equal(t, resp2.Success, true) 89 | }) 90 | } 91 | 92 | func runServerWithMockStateMachine(state string, fn func(s Server, m *mock.Mock)) { 93 | var m mockStateMachine 94 | s := newTestServer("1", &testTransporter{}) 95 | s.(*server).stateMachine = &m 96 | if err := s.Start(); err != nil { 97 | panic("server start error: " + err.Error()) 98 | } 99 | if state == Leader { 100 | if _, err := s.Do(&DefaultJoinCommand{Name: s.Name()}); err != nil { 101 | panic("unable to join server to self: " + err.Error()) 102 | } 103 | } 104 | defer s.Stop() 105 | fn(s, &m.Mock) 106 | } 107 | -------------------------------------------------------------------------------- /statemachine.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | // StateMachine is the interface for allowing the host application to save and 4 | // recovery the state machine. This makes it possible to make snapshots 5 | // and compact the log. 6 | type StateMachine interface { 7 | Save() ([]byte, error) 8 | Recovery([]byte) error 9 | } 10 | -------------------------------------------------------------------------------- /statemachine_test.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "github.com/stretchr/testify/mock" 5 | ) 6 | 7 | type mockStateMachine struct { 8 | mock.Mock 9 | } 10 | 11 | func (m *mockStateMachine) Save() ([]byte, error) { 12 | args := m.Called() 13 | return args.Get(0).([]byte), args.Error(1) 14 | } 15 | 16 | func (m *mockStateMachine) Recovery(b []byte) error { 17 | args := m.Called(b) 18 | return args.Error(0) 19 | } 20 | -------------------------------------------------------------------------------- /test.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "fmt" 5 | "io/ioutil" 6 | "os" 7 | "time" 8 | ) 9 | 10 | const ( 11 | testHeartbeatInterval = 50 * time.Millisecond 12 | testElectionTimeout = 200 * time.Millisecond 13 | ) 14 | 15 | const ( 16 | testListenerLoggerEnabled = false 17 | ) 18 | 19 | func init() { 20 | RegisterCommand(&testCommand1{}) 21 | RegisterCommand(&testCommand2{}) 22 | } 23 | 24 | //------------------------------------------------------------------------------ 25 | // 26 | // Helpers 27 | // 28 | //------------------------------------------------------------------------------ 29 | 30 | //-------------------------------------- 31 | // Logs 32 | //-------------------------------------- 33 | 34 | func getLogPath() string { 35 | f, _ := ioutil.TempFile("", "raft-log-") 36 | f.Close() 37 | os.Remove(f.Name()) 38 | return f.Name() 39 | } 40 | 41 | func setupLog(entries []*LogEntry) (*Log, string) { 42 | f, _ := ioutil.TempFile("", "raft-log-") 43 | 44 | for _, entry := range entries { 45 | entry.Encode(f) 46 | } 47 | err := f.Close() 48 | 49 | if err != nil { 50 | panic(err) 51 | } 52 | 53 | log := newLog() 54 | log.ApplyFunc = func(e *LogEntry, c Command) (interface{}, error) { 55 | return nil, nil 56 | } 57 | if err := log.open(f.Name()); err != nil { 58 | panic(err) 59 | } 60 | return log, f.Name() 61 | } 62 | 63 | //-------------------------------------- 64 | // Servers 65 | //-------------------------------------- 66 | 67 | func newTestServer(name string, transporter Transporter) Server { 68 | p, _ := ioutil.TempDir("", "raft-server-") 69 | if err := os.MkdirAll(p, 0644); err != nil { 70 | panic(err.Error()) 71 | } 72 | server, _ := NewServer(name, p, transporter, nil, nil, "") 73 | if testListenerLoggerEnabled { 74 | fn := func(e Event) { 75 | server := e.Source().(Server) 76 | warnf("[%s] %s %v -> %v\n", server.Name(), e.Type(), e.PrevValue(), e.Value()) 77 | } 78 | server.AddEventListener(StateChangeEventType, fn) 79 | server.AddEventListener(LeaderChangeEventType, fn) 80 | server.AddEventListener(TermChangeEventType, fn) 81 | } 82 | return server 83 | } 84 | 85 | func newTestServerWithPath(name string, transporter Transporter, p string) Server { 86 | server, _ := NewServer(name, p, transporter, nil, nil, "") 87 | return server 88 | } 89 | 90 | func newTestServerWithLog(name string, transporter Transporter, entries []*LogEntry) Server { 91 | server := newTestServer(name, transporter) 92 | f, err := os.Create(server.LogPath()) 93 | if err != nil { 94 | panic(err) 95 | } 96 | 97 | for _, entry := range entries { 98 | entry.Encode(f) 99 | } 100 | f.Close() 101 | return server 102 | } 103 | 104 | func newTestCluster(names []string, transporter Transporter, lookup map[string]Server) []Server { 105 | servers := []Server{} 106 | e0, _ := newLogEntry(newLog(), nil, 1, 1, &testCommand1{Val: "foo", I: 20}) 107 | 108 | for _, name := range names { 109 | if lookup[name] != nil { 110 | panic(fmt.Sprintf("raft: Duplicate server in test cluster! %v", name)) 111 | } 112 | server := newTestServerWithLog("1", transporter, []*LogEntry{e0}) 113 | server.SetElectionTimeout(testElectionTimeout) 114 | servers = append(servers, server) 115 | lookup[name] = server 116 | } 117 | for _, server := range servers { 118 | server.SetHeartbeatInterval(testHeartbeatInterval) 119 | server.Start() 120 | for _, peer := range servers { 121 | server.AddPeer(peer.Name(), "") 122 | } 123 | } 124 | return servers 125 | } 126 | 127 | //-------------------------------------- 128 | // Transporter 129 | //-------------------------------------- 130 | 131 | type testTransporter struct { 132 | sendVoteRequestFunc func(server Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse 133 | sendAppendEntriesRequestFunc func(server Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse 134 | sendSnapshotRequestFunc func(server Server, peer *Peer, req *SnapshotRequest) *SnapshotResponse 135 | } 136 | 137 | func (t *testTransporter) SendVoteRequest(server Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse { 138 | return t.sendVoteRequestFunc(server, peer, req) 139 | } 140 | 141 | func (t *testTransporter) SendAppendEntriesRequest(server Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse { 142 | return t.sendAppendEntriesRequestFunc(server, peer, req) 143 | } 144 | 145 | func (t *testTransporter) SendSnapshotRequest(server Server, peer *Peer, req *SnapshotRequest) *SnapshotResponse { 146 | return t.sendSnapshotRequestFunc(server, peer, req) 147 | } 148 | 149 | func (t *testTransporter) SendSnapshotRecoveryRequest(server Server, peer *Peer, req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse { 150 | return t.SendSnapshotRecoveryRequest(server, peer, req) 151 | } 152 | 153 | type testStateMachine struct { 154 | saveFunc func() ([]byte, error) 155 | recoveryFunc func([]byte) error 156 | } 157 | 158 | func (sm *testStateMachine) Save() ([]byte, error) { 159 | return sm.saveFunc() 160 | } 161 | 162 | func (sm *testStateMachine) Recovery(state []byte) error { 163 | return sm.recoveryFunc(state) 164 | } 165 | 166 | //-------------------------------------- 167 | // Command1 168 | //-------------------------------------- 169 | 170 | type testCommand1 struct { 171 | Val string `json:"val"` 172 | I int `json:"i"` 173 | } 174 | 175 | func (c *testCommand1) CommandName() string { 176 | return "cmd_1" 177 | } 178 | 179 | func (c *testCommand1) Apply(server Server) (interface{}, error) { 180 | return nil, nil 181 | } 182 | 183 | //-------------------------------------- 184 | // Command2 185 | //-------------------------------------- 186 | 187 | type testCommand2 struct { 188 | X int `json:"x"` 189 | } 190 | 191 | func (c *testCommand2) CommandName() string { 192 | return "cmd_2" 193 | } 194 | 195 | func (c *testCommand2) Apply(server Server) (interface{}, error) { 196 | return nil, nil 197 | } 198 | -------------------------------------------------------------------------------- /transporter.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | //------------------------------------------------------------------------------ 4 | // 5 | // Typedefs 6 | // 7 | //------------------------------------------------------------------------------ 8 | 9 | // Transporter is the interface for allowing the host application to transport 10 | // requests to other nodes. 11 | type Transporter interface { 12 | SendVoteRequest(server Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse 13 | SendAppendEntriesRequest(server Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse 14 | SendSnapshotRequest(server Server, peer *Peer, req *SnapshotRequest) *SnapshotResponse 15 | SendSnapshotRecoveryRequest(server Server, peer *Peer, req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse 16 | } 17 | -------------------------------------------------------------------------------- /util.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "fmt" 5 | "io" 6 | "math/rand" 7 | "os" 8 | "time" 9 | ) 10 | 11 | // uint64Slice implements sort interface 12 | type uint64Slice []uint64 13 | 14 | func (p uint64Slice) Len() int { return len(p) } 15 | func (p uint64Slice) Less(i, j int) bool { return p[i] < p[j] } 16 | func (p uint64Slice) Swap(i, j int) { p[i], p[j] = p[j], p[i] } 17 | 18 | // WriteFile writes data to a file named by filename. 19 | // If the file does not exist, WriteFile creates it with permissions perm; 20 | // otherwise WriteFile truncates it before writing. 21 | // This is copied from ioutil.WriteFile with the addition of a Sync call to 22 | // ensure the data reaches the disk. 23 | func writeFileSynced(filename string, data []byte, perm os.FileMode) error { 24 | f, err := os.OpenFile(filename, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, perm) 25 | if err != nil { 26 | return err 27 | } 28 | defer f.Close() // Idempotent 29 | 30 | n, err := f.Write(data) 31 | if err == nil && n < len(data) { 32 | return io.ErrShortWrite 33 | } else if err != nil { 34 | return err 35 | } 36 | 37 | if err = f.Sync(); err != nil { 38 | return err 39 | } 40 | 41 | return f.Close() 42 | } 43 | 44 | // Waits for a random time between two durations and sends the current time on 45 | // the returned channel. 46 | func afterBetween(min time.Duration, max time.Duration) <-chan time.Time { 47 | rand := rand.New(rand.NewSource(time.Now().UnixNano())) 48 | d, delta := min, (max - min) 49 | if delta > 0 { 50 | d += time.Duration(rand.Int63n(int64(delta))) 51 | } 52 | return time.After(d) 53 | } 54 | 55 | // TODO(xiangli): Remove assertions when we reach version 1.0 56 | 57 | // _assert will panic with a given formatted message if the given condition is false. 58 | func _assert(condition bool, msg string, v ...interface{}) { 59 | if !condition { 60 | panic(fmt.Sprintf("assertion failed: "+msg, v...)) 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /z_test.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | /* 4 | import ( 5 | "testing" 6 | "time" 7 | ) 8 | 9 | func TestGC(t *testing.T) { 10 | <-time.After(500 * time.Millisecond) 11 | panic("Oh god no!") 12 | } 13 | */ 14 | --------------------------------------------------------------------------------