Browse Source

bump(github.com/coreos/go-raft): bb7f7ec92e4cb6d98241cea83f55d0e85e624189

Brandon Philips 12 years ago
parent
commit
449cad4658

+ 39 - 17
third_party/github.com/coreos/go-raft/README.md

@@ -1,19 +1,49 @@
-[![Build Status](https://travis-ci.org/benbjohnson/go-raft.png?branch=master)](https://travis-ci.org/benbjohnson/go-raft)
-
 go-raft
 go-raft
 =======
 =======
 
 
+[![Build Status](https://travis-ci.org/goraft/raft.png?branch=master)](https://travis-ci.org/goraft/raft)
+
 ## Overview
 ## Overview
 
 
-This is an Go implementation of the Raft distributed consensus protocol.
+This is a Go implementation of the Raft distributed consensus protocol.
 Raft is a protocol by which a cluster of nodes can maintain a replicated state machine.
 Raft is a protocol by which a cluster of nodes can maintain a replicated state machine.
 The state machine is kept in sync through the use of a replicated log.
 The state machine is kept in sync through the use of a replicated log.
 
 
-For more details on Raft, you can read [In Search of an Understandable Consensus Algorithm](https://ramcloud.stanford.edu/wiki/download/attachments/11370504/raft.pdf) by Diego Ongaro and John Ousterhout.
+For more details on Raft, you can read [In Search of an Understandable Consensus Algorithm][raft-paper] by Diego Ongaro and John Ousterhout.
+
+## Project Status
+
+This library is feature complete but should be considered experimental until it has seen more usage.
+If you have any questions on implementing go-raft in your project please file an issue.
+There is an [active community][community] of developers who can help.
+go-raft is under the MIT license.
+
+[community]: https://github.com/goraft/raft/contributors
+
+### Features
+
+- Leader election
+- Log replication
+- Configuration changes
+- Log compaction
+- Unit tests
+- Fast Protobuf Log Encoding
+- HTTP transport
 
 
+### Projects
+
+These projects are built on go-raft:
+
+- [coreos/etcd](https://github.com/coreos/etcd) - A highly-available key value store for shared configuration and service discovery
+- [benbjohnson/raftd](https://github.com/benbjohnson/raftd) - A reference implementation for using the go-raft library for distributed consensus.
+
+If you have a project that you're using go-raft in, please add it to this README so others can see implementation examples.
 
 
 ## The Raft Protocol
 ## The Raft Protocol
 
 
+This section provides a summary of the Raft protocol from a high level.
+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].
+
 ### Overview
 ### Overview
 
 
 Maintaining state in a single process on a single server is easy.
 Maintaining state in a single process on a single server is easy.
@@ -26,7 +56,7 @@ Servers can crash or the network between two machines can become unavailable or
 A distributed consensus protocol is used for maintaining a consistent state across multiple servers in a cluster.
 A distributed consensus protocol is used for maintaining a consistent state across multiple servers in a cluster.
 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.
 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.
 
 
-An alternative is the [Raft distributed consensus protocol](https://ramcloud.stanford.edu/wiki/download/attachments/11370504/raft.pdf) by Diego Ongaro and John Ousterhout.
+An alternative is the [Raft distributed consensus protocol][raft-paper] by Diego Ongaro and John Ousterhout.
 Raft is a protocol built with understandability as a primary tenant and it centers around two things:
 Raft is a protocol built with understandability as a primary tenant and it centers around two things:
 
 
 1. Leader Election
 1. Leader Election
@@ -53,17 +83,9 @@ By ensuring that this log is replicated identically between all the nodes in the
 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).
 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).
 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.
 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.
 
 
-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](https://ramcloud.stanford.edu/wiki/download/attachments/11370504/raft.pdf)
-
-
-## Project Status
-
-The go-raft library is feature complete but in alpha.
-There is a reference implementation called [raftd](https://github.com/benbjohnson/raftd) that demonstrates how to use the library 
+## History
 
 
-The library will be considered experimental until it has significant production usage.
-I'm writing the library for the purpose of including distributed processing in my behavioral analytics database called [Sky](https://github.com/skydb/sky).
-However, I hope other projects can benefit from having a distributed consensus protocol so the go-raft library is available under MIT license.
+Ben Johnson started this library for use in his behavioral analytics database called [Sky](https://github.com/skydb/sky).
+He put it under the MIT license in the hopes that it would be useful for other projects too.
 
 
-If you have a project that you're using go-raft in, please add it to this README and send a pull request so others can see implementation examples.
-If you have any questions on implementing go-raft in your project, feel free to contact me on [GitHub](https://github.com/benbjohnson), [Twitter](https://twitter.com/benbjohnson) or by e-mail at [ben@skylandlabs.com](mailto:ben@skylandlabs.com).
+[raft-paper]: https://ramcloud.stanford.edu/wiki/download/attachments/11370504/raft.pdf

+ 7 - 0
third_party/github.com/coreos/go-raft/config.go

@@ -0,0 +1,7 @@
+package raft
+
+type Config struct {
+	CommitIndex uint64 `json:"commitIndex"`
+	// TODO decide what we need to store in peer struct
+	Peers []*Peer `json:"peers"`
+}

+ 2 - 2
third_party/github.com/coreos/go-raft/http_transporter.go

@@ -94,7 +94,7 @@ func (t *HTTPTransporter) SendAppendEntriesRequest(server *Server, peer *Peer, r
 		return nil
 		return nil
 	}
 	}
 
 
-	url := fmt.Sprintf("http://%s%s", peer.Name(), t.AppendEntriesPath())
+	url := fmt.Sprintf("http://%s%s", peer.Name, t.AppendEntriesPath())
 	traceln(server.Name(), "POST", url)
 	traceln(server.Name(), "POST", url)
 
 
 	client := &http.Client{Transport: &http.Transport{DisableKeepAlives: t.DisableKeepAlives}}
 	client := &http.Client{Transport: &http.Transport{DisableKeepAlives: t.DisableKeepAlives}}
@@ -122,7 +122,7 @@ func (t *HTTPTransporter) SendVoteRequest(server *Server, peer *Peer, req *Reque
 		return nil
 		return nil
 	}
 	}
 
 
-	url := fmt.Sprintf("http://%s%s", peer.Name(), t.RequestVotePath())
+	url := fmt.Sprintf("http://%s%s", peer.Name, t.RequestVotePath())
 	traceln(server.Name(), "POST", url)
 	traceln(server.Name(), "POST", url)
 
 
 	client := &http.Client{Transport: &http.Transport{DisableKeepAlives: t.DisableKeepAlives}}
 	client := &http.Client{Transport: &http.Transport{DisableKeepAlives: t.DisableKeepAlives}}

+ 3 - 2
third_party/github.com/coreos/go-raft/join_command.go

@@ -9,7 +9,8 @@ type JoinCommand interface {
 
 
 // Join command
 // Join command
 type DefaultJoinCommand struct {
 type DefaultJoinCommand struct {
-	Name string `json:"name"`
+	Name             string `json:"name"`
+	ConnectionString string `json:"connectionString"`
 }
 }
 
 
 // The name of the Join command in the log
 // The name of the Join command in the log
@@ -18,7 +19,7 @@ func (c *DefaultJoinCommand) CommandName() string {
 }
 }
 
 
 func (c *DefaultJoinCommand) Apply(server *Server) (interface{}, error) {
 func (c *DefaultJoinCommand) Apply(server *Server) (interface{}, error) {
-	err := server.AddPeer(c.Name)
+	err := server.AddPeer(c.Name, c.ConnectionString)
 
 
 	return []byte("join"), err
 	return []byte("join"), err
 }
 }

+ 9 - 0
third_party/github.com/coreos/go-raft/log.go

@@ -183,6 +183,15 @@ func (l *Log) open(path string) error {
 
 
 		// Append entry.
 		// Append entry.
 		l.entries = append(l.entries, entry)
 		l.entries = append(l.entries, entry)
+
+		if entry.Index <= l.commitIndex {
+			command, err := newCommand(entry.CommandName, entry.Command)
+			if err != nil {
+				continue
+			}
+			l.ApplyFunc(command)
+		}
+
 		debugln("open.log.append log index ", entry.Index)
 		debugln("open.log.append log index ", entry.Index)
 
 
 		readBytes += int64(n)
 		readBytes += int64(n)

+ 55 - 45
third_party/github.com/coreos/go-raft/peer.go

@@ -14,7 +14,8 @@ import (
 // A peer is a reference to another server involved in the consensus protocol.
 // A peer is a reference to another server involved in the consensus protocol.
 type Peer struct {
 type Peer struct {
 	server           *Server
 	server           *Server
-	name             string
+	Name             string `json:"name"`
+	ConnectionString string `json:"connectionString"`
 	prevLogIndex     uint64
 	prevLogIndex     uint64
 	mutex            sync.RWMutex
 	mutex            sync.RWMutex
 	stopChan         chan bool
 	stopChan         chan bool
@@ -28,10 +29,11 @@ type Peer struct {
 //------------------------------------------------------------------------------
 //------------------------------------------------------------------------------
 
 
 // Creates a new peer.
 // Creates a new peer.
-func newPeer(server *Server, name string, heartbeatTimeout time.Duration) *Peer {
+func newPeer(server *Server, name string, connectionString string, heartbeatTimeout time.Duration) *Peer {
 	return &Peer{
 	return &Peer{
 		server:           server,
 		server:           server,
-		name:             name,
+		Name:             name,
+		ConnectionString: connectionString,
 		heartbeatTimeout: heartbeatTimeout,
 		heartbeatTimeout: heartbeatTimeout,
 	}
 	}
 }
 }
@@ -42,11 +44,6 @@ func newPeer(server *Server, name string, heartbeatTimeout time.Duration) *Peer
 //
 //
 //------------------------------------------------------------------------------
 //------------------------------------------------------------------------------
 
 
-// Retrieves the name of the peer.
-func (p *Peer) Name() string {
-	return p.name
-}
-
 // Sets the heartbeat timeout.
 // Sets the heartbeat timeout.
 func (p *Peer) setHeartbeatTimeout(duration time.Duration) {
 func (p *Peer) setHeartbeatTimeout(duration time.Duration) {
 	p.heartbeatTimeout = duration
 	p.heartbeatTimeout = duration
@@ -89,17 +86,17 @@ func (p *Peer) startHeartbeat() {
 }
 }
 
 
 // Stops the peer heartbeat.
 // Stops the peer heartbeat.
-func (p *Peer) stopHeartbeat() {
+func (p *Peer) stopHeartbeat(flush bool) {
 	// here is a problem
 	// here is a problem
 	// the previous stop is no buffer leader may get blocked
 	// the previous stop is no buffer leader may get blocked
-	// when heartbeat returns at line 132
+	// when heartbeat returns
 	// I make the channel with 1 buffer
 	// I make the channel with 1 buffer
 	// and try to panic here
 	// and try to panic here
 	select {
 	select {
-	case p.stopChan <- true:
+	case p.stopChan <- flush:
 
 
 	default:
 	default:
-		panic("[" + p.server.Name() + "] cannot stop [" + p.Name() + "] heartbeat")
+		panic("[" + p.server.Name() + "] cannot stop [" + p.Name + "] heartbeat")
 	}
 	}
 }
 }
 
 
@@ -113,8 +110,9 @@ func (p *Peer) clone() *Peer {
 	p.mutex.Lock()
 	p.mutex.Lock()
 	defer p.mutex.Unlock()
 	defer p.mutex.Unlock()
 	return &Peer{
 	return &Peer{
-		name:         p.name,
-		prevLogIndex: p.prevLogIndex,
+		Name:             p.Name,
+		ConnectionString: p.ConnectionString,
+		prevLogIndex:     p.prevLogIndex,
 	}
 	}
 }
 }
 
 
@@ -128,46 +126,58 @@ func (p *Peer) heartbeat(c chan bool) {
 
 
 	c <- true
 	c <- true
 
 
-	debugln("peer.heartbeat: ", p.Name(), p.heartbeatTimeout)
+	debugln("peer.heartbeat: ", p.Name, p.heartbeatTimeout)
 
 
 	for {
 	for {
 		select {
 		select {
-		case <-stopChan:
-			debugln("peer.heartbeat.stop: ", p.Name())
-			return
-
-		case <-time.After(p.heartbeatTimeout):
-			debugln("peer.heartbeat.run: ", p.Name())
-			prevLogIndex := p.getPrevLogIndex()
-			entries, prevLogTerm := p.server.log.getEntriesAfter(prevLogIndex, p.server.maxLogEntriesPerRequest)
-
-			if p.server.State() != Leader {
+		case flush := <-stopChan:
+			if !flush {
+				debugln("peer.heartbeat.stop: ", p.Name)
 				return
 				return
-			}
-
-			if entries != nil {
-				p.sendAppendEntriesRequest(newAppendEntriesRequest(p.server.currentTerm, prevLogIndex, prevLogTerm, p.server.log.CommitIndex(), p.server.name, entries))
 			} else {
 			} else {
-				p.sendSnapshotRequest(newSnapshotRequest(p.server.name, p.server.lastSnapshot))
+				// before we can safely remove a node
+				// we must flush the remove command to the node first
+				p.flush()
+				debugln("peer.heartbeat.stop: ", p.Name)
+				return
 			}
 			}
+
+		case <-time.After(p.heartbeatTimeout):
+			p.flush()
 		}
 		}
 	}
 	}
 }
 }
 
 
+func (p *Peer) flush() {
+	debugln("peer.heartbeat.run: ", p.Name)
+	prevLogIndex := p.getPrevLogIndex()
+	entries, prevLogTerm := p.server.log.getEntriesAfter(prevLogIndex, p.server.maxLogEntriesPerRequest)
+
+	if p.server.State() != Leader {
+		return
+	}
+
+	if entries != nil {
+		p.sendAppendEntriesRequest(newAppendEntriesRequest(p.server.currentTerm, prevLogIndex, prevLogTerm, p.server.log.CommitIndex(), p.server.name, entries))
+	} else {
+		p.sendSnapshotRequest(newSnapshotRequest(p.server.name, p.server.lastSnapshot))
+	}
+}
+
 //--------------------------------------
 //--------------------------------------
 // Append Entries
 // Append Entries
 //--------------------------------------
 //--------------------------------------
 
 
 // Sends an AppendEntries request to the peer through the transport.
 // Sends an AppendEntries request to the peer through the transport.
 func (p *Peer) sendAppendEntriesRequest(req *AppendEntriesRequest) {
 func (p *Peer) sendAppendEntriesRequest(req *AppendEntriesRequest) {
-	traceln("peer.flush.send: ", p.server.Name(), "->", p.Name(), " ", len(req.Entries))
+	traceln("peer.flush.send: ", p.server.Name(), "->", p.Name, " ", len(req.Entries))
 
 
 	resp := p.server.Transporter().SendAppendEntriesRequest(p.server, p, req)
 	resp := p.server.Transporter().SendAppendEntriesRequest(p.server, p, req)
 	if resp == nil {
 	if resp == nil {
-		debugln("peer.flush.timeout: ", p.server.Name(), "->", p.Name())
+		debugln("peer.flush.timeout: ", p.server.Name(), "->", p.Name)
 		return
 		return
 	}
 	}
-	traceln("peer.flush.recv: ", p.Name())
+	traceln("peer.flush.recv: ", p.Name)
 
 
 	// If successful then update the previous log index.
 	// If successful then update the previous log index.
 	p.mutex.Lock()
 	p.mutex.Lock()
@@ -181,7 +191,7 @@ func (p *Peer) sendAppendEntriesRequest(req *AppendEntriesRequest) {
 				resp.append = true
 				resp.append = true
 			}
 			}
 		}
 		}
-		traceln("peer.flush.success: ", p.server.Name(), "->", p.Name(), "; idx =", p.prevLogIndex)
+		traceln("peer.flush.success: ", p.server.Name(), "->", p.Name, "; idx =", p.prevLogIndex)
 
 
 		// If it was unsuccessful then decrement the previous log index and
 		// If it was unsuccessful then decrement the previous log index and
 		// we'll try again next time.
 		// we'll try again next time.
@@ -195,7 +205,7 @@ func (p *Peer) sendAppendEntriesRequest(req *AppendEntriesRequest) {
 
 
 			p.prevLogIndex = resp.CommitIndex
 			p.prevLogIndex = resp.CommitIndex
 
 
-			debugln("peer.flush.commitIndex: ", p.server.Name(), "->", p.Name(), " idx =", p.prevLogIndex)
+			debugln("peer.flush.commitIndex: ", p.server.Name(), "->", p.Name, " idx =", p.prevLogIndex)
 		} else if p.prevLogIndex > 0 {
 		} else if p.prevLogIndex > 0 {
 			// Decrement the previous log index down until we find a match. Don't
 			// Decrement the previous log index down until we find a match. Don't
 			// let it go below where the peer's commit index is though. That's a
 			// let it go below where the peer's commit index is though. That's a
@@ -206,35 +216,35 @@ func (p *Peer) sendAppendEntriesRequest(req *AppendEntriesRequest) {
 				p.prevLogIndex = resp.Index
 				p.prevLogIndex = resp.Index
 			}
 			}
 
 
-			debugln("peer.flush.decrement: ", p.server.Name(), "->", p.Name(), " idx =", p.prevLogIndex)
+			debugln("peer.flush.decrement: ", p.server.Name(), "->", p.Name, " idx =", p.prevLogIndex)
 		}
 		}
 	}
 	}
 	p.mutex.Unlock()
 	p.mutex.Unlock()
 
 
 	// Attach the peer to resp, thus server can know where it comes from
 	// Attach the peer to resp, thus server can know where it comes from
-	resp.peer = p.Name()
+	resp.peer = p.Name
 	// Send response to server for processing.
 	// Send response to server for processing.
 	p.server.send(resp)
 	p.server.send(resp)
 }
 }
 
 
 // Sends an Snapshot request to the peer through the transport.
 // Sends an Snapshot request to the peer through the transport.
 func (p *Peer) sendSnapshotRequest(req *SnapshotRequest) {
 func (p *Peer) sendSnapshotRequest(req *SnapshotRequest) {
-	debugln("peer.snap.send: ", p.name)
+	debugln("peer.snap.send: ", p.Name)
 
 
 	resp := p.server.Transporter().SendSnapshotRequest(p.server, p, req)
 	resp := p.server.Transporter().SendSnapshotRequest(p.server, p, req)
 	if resp == nil {
 	if resp == nil {
-		debugln("peer.snap.timeout: ", p.name)
+		debugln("peer.snap.timeout: ", p.Name)
 		return
 		return
 	}
 	}
 
 
-	debugln("peer.snap.recv: ", p.name)
+	debugln("peer.snap.recv: ", p.Name)
 
 
 	// If successful, the peer should have been to snapshot state
 	// If successful, the peer should have been to snapshot state
 	// Send it the snapshot!
 	// Send it the snapshot!
 	if resp.Success {
 	if resp.Success {
 		p.sendSnapshotRecoveryRequest()
 		p.sendSnapshotRecoveryRequest()
 	} else {
 	} else {
-		debugln("peer.snap.failed: ", p.name)
+		debugln("peer.snap.failed: ", p.Name)
 		return
 		return
 	}
 	}
 
 
@@ -243,12 +253,12 @@ func (p *Peer) sendSnapshotRequest(req *SnapshotRequest) {
 // Sends an Snapshot Recovery request to the peer through the transport.
 // Sends an Snapshot Recovery request to the peer through the transport.
 func (p *Peer) sendSnapshotRecoveryRequest() {
 func (p *Peer) sendSnapshotRecoveryRequest() {
 	req := newSnapshotRecoveryRequest(p.server.name, p.server.lastSnapshot)
 	req := newSnapshotRecoveryRequest(p.server.name, p.server.lastSnapshot)
-	debugln("peer.snap.recovery.send: ", p.name)
+	debugln("peer.snap.recovery.send: ", p.Name)
 	resp := p.server.Transporter().SendSnapshotRecoveryRequest(p.server, p, req)
 	resp := p.server.Transporter().SendSnapshotRecoveryRequest(p.server, p, req)
 	if resp.Success {
 	if resp.Success {
 		p.prevLogIndex = req.LastIndex
 		p.prevLogIndex = req.LastIndex
 	} else {
 	} else {
-		debugln("peer.snap.recovery.failed: ", p.name)
+		debugln("peer.snap.recovery.failed: ", p.Name)
 		return
 		return
 	}
 	}
 	// Send response to server for processing.
 	// Send response to server for processing.
@@ -261,10 +271,10 @@ func (p *Peer) sendSnapshotRecoveryRequest() {
 
 
 // send VoteRequest Request
 // send VoteRequest Request
 func (p *Peer) sendVoteRequest(req *RequestVoteRequest, c chan *RequestVoteResponse) {
 func (p *Peer) sendVoteRequest(req *RequestVoteRequest, c chan *RequestVoteResponse) {
-	debugln("peer.vote: ", p.server.Name(), "->", p.Name())
+	debugln("peer.vote: ", p.server.Name(), "->", p.Name)
 	req.peer = p
 	req.peer = p
 	if resp := p.server.Transporter().SendVoteRequest(p.server, p, req); resp != nil {
 	if resp := p.server.Transporter().SendVoteRequest(p.server, p, req); resp != nil {
-		debugln("peer.vote: recv", p.server.Name(), "<-", p.Name())
+		debugln("peer.vote: recv", p.server.Name(), "<-", p.Name)
 		resp.peer = p
 		resp.peer = p
 		c <- resp
 		c <- resp
 	}
 	}

+ 33 - 7
third_party/github.com/coreos/go-raft/protobuf/snapshot_recovery_request.pb.go

@@ -14,12 +14,12 @@ var _ = &json.SyntaxError{}
 var _ = math.Inf
 var _ = math.Inf
 
 
 type ProtoSnapshotRecoveryRequest struct {
 type ProtoSnapshotRecoveryRequest struct {
-	LeaderName       *string  `protobuf:"bytes,1,req" json:"LeaderName,omitempty"`
-	LastIndex        *uint64  `protobuf:"varint,2,req" json:"LastIndex,omitempty"`
-	LastTerm         *uint64  `protobuf:"varint,3,req" json:"LastTerm,omitempty"`
-	Peers            []string `protobuf:"bytes,4,rep" json:"Peers,omitempty"`
-	State            []byte   `protobuf:"bytes,5,req" json:"State,omitempty"`
-	XXX_unrecognized []byte   `json:"-"`
+	LeaderName       *string                                   `protobuf:"bytes,1,req" json:"LeaderName,omitempty"`
+	LastIndex        *uint64                                   `protobuf:"varint,2,req" json:"LastIndex,omitempty"`
+	LastTerm         *uint64                                   `protobuf:"varint,3,req" json:"LastTerm,omitempty"`
+	Peers            []*ProtoSnapshotRecoveryRequest_ProtoPeer `protobuf:"bytes,4,rep" json:"Peers,omitempty"`
+	State            []byte                                    `protobuf:"bytes,5,req" json:"State,omitempty"`
+	XXX_unrecognized []byte                                    `json:"-"`
 }
 }
 
 
 func (m *ProtoSnapshotRecoveryRequest) Reset()         { *m = ProtoSnapshotRecoveryRequest{} }
 func (m *ProtoSnapshotRecoveryRequest) Reset()         { *m = ProtoSnapshotRecoveryRequest{} }
@@ -47,7 +47,7 @@ func (m *ProtoSnapshotRecoveryRequest) GetLastTerm() uint64 {
 	return 0
 	return 0
 }
 }
 
 
-func (m *ProtoSnapshotRecoveryRequest) GetPeers() []string {
+func (m *ProtoSnapshotRecoveryRequest) GetPeers() []*ProtoSnapshotRecoveryRequest_ProtoPeer {
 	if m != nil {
 	if m != nil {
 		return m.Peers
 		return m.Peers
 	}
 	}
@@ -61,5 +61,31 @@ func (m *ProtoSnapshotRecoveryRequest) GetState() []byte {
 	return nil
 	return nil
 }
 }
 
 
+type ProtoSnapshotRecoveryRequest_ProtoPeer struct {
+	Name             *string `protobuf:"bytes,1,req" json:"Name,omitempty"`
+	ConnectionString *string `protobuf:"bytes,2,req" json:"ConnectionString,omitempty"`
+	XXX_unrecognized []byte  `json:"-"`
+}
+
+func (m *ProtoSnapshotRecoveryRequest_ProtoPeer) Reset() {
+	*m = ProtoSnapshotRecoveryRequest_ProtoPeer{}
+}
+func (m *ProtoSnapshotRecoveryRequest_ProtoPeer) String() string { return proto.CompactTextString(m) }
+func (*ProtoSnapshotRecoveryRequest_ProtoPeer) ProtoMessage()    {}
+
+func (m *ProtoSnapshotRecoveryRequest_ProtoPeer) GetName() string {
+	if m != nil && m.Name != nil {
+		return *m.Name
+	}
+	return ""
+}
+
+func (m *ProtoSnapshotRecoveryRequest_ProtoPeer) GetConnectionString() string {
+	if m != nil && m.ConnectionString != nil {
+		return *m.ConnectionString
+	}
+	return ""
+}
+
 func init() {
 func init() {
 }
 }

+ 8 - 2
third_party/github.com/coreos/go-raft/protobuf/snapshot_recovery_request.proto

@@ -3,7 +3,13 @@ package protobuf;
 message ProtoSnapshotRecoveryRequest {
 message ProtoSnapshotRecoveryRequest {
 	required string  LeaderName=1;
 	required string  LeaderName=1;
 	required uint64  LastIndex=2;
 	required uint64  LastIndex=2;
-	required uint64  LastTerm=3;      
-	repeated string  Peers=4;      
+	required uint64  LastTerm=3;   
+
+	message ProtoPeer {
+		required string Name=1;
+		required string ConnectionString=2;
+	}  
+	repeated ProtoPeer  Peers=4;  
+
 	required bytes   State=5;
 	required bytes   State=5;
 }
 }

+ 152 - 139
third_party/github.com/coreos/go-raft/server.go

@@ -5,7 +5,6 @@ import (
 	"errors"
 	"errors"
 	"fmt"
 	"fmt"
 	"hash/crc32"
 	"hash/crc32"
-	"io"
 	"io/ioutil"
 	"io/ioutil"
 	"os"
 	"os"
 	"path"
 	"path"
@@ -81,8 +80,6 @@ type Server struct {
 	lastSnapshot            *Snapshot
 	lastSnapshot            *Snapshot
 	stateMachine            StateMachine
 	stateMachine            StateMachine
 	maxLogEntriesPerRequest uint64
 	maxLogEntriesPerRequest uint64
-
-	confFile *os.File
 }
 }
 
 
 // An event to be processed by the server's event loop.
 // An event to be processed by the server's event loop.
@@ -272,11 +269,15 @@ func (s *Server) QuorumSize() int {
 
 
 // Retrieves the election timeout.
 // Retrieves the election timeout.
 func (s *Server) ElectionTimeout() time.Duration {
 func (s *Server) ElectionTimeout() time.Duration {
+	s.mutex.RLock()
+	defer s.mutex.RUnlock()
 	return s.electionTimeout
 	return s.electionTimeout
 }
 }
 
 
 // Sets the election timeout.
 // Sets the election timeout.
 func (s *Server) SetElectionTimeout(duration time.Duration) {
 func (s *Server) SetElectionTimeout(duration time.Duration) {
+	s.mutex.Lock()
+	defer s.mutex.Unlock()
 	s.electionTimeout = duration
 	s.electionTimeout = duration
 }
 }
 
 
@@ -286,6 +287,8 @@ func (s *Server) SetElectionTimeout(duration time.Duration) {
 
 
 // Retrieves the heartbeat timeout.
 // Retrieves the heartbeat timeout.
 func (s *Server) HeartbeatTimeout() time.Duration {
 func (s *Server) HeartbeatTimeout() time.Duration {
+	s.mutex.RLock()
+	defer s.mutex.RUnlock()
 	return s.heartbeatTimeout
 	return s.heartbeatTimeout
 }
 }
 
 
@@ -332,14 +335,14 @@ func (s *Server) Start() error {
 	// Create snapshot directory if not exist
 	// Create snapshot directory if not exist
 	os.Mkdir(path.Join(s.path, "snapshot"), 0700)
 	os.Mkdir(path.Join(s.path, "snapshot"), 0700)
 
 
-	// Initialize the log and load it up.
-	if err := s.log.open(s.LogPath()); err != nil {
-		s.debugln("raft: Log error: ", err)
+	if err := s.readConf(); err != nil {
+		s.debugln("raft: Conf file error: ", err)
 		return fmt.Errorf("raft: Initialization error: %s", err)
 		return fmt.Errorf("raft: Initialization error: %s", err)
 	}
 	}
 
 
-	if err := s.readConf(); err != nil {
-		s.debugln("raft: Conf file error: ", err)
+	// Initialize the log and load it up.
+	if err := s.log.open(s.LogPath()); err != nil {
+		s.debugln("raft: Log error: ", err)
 		return fmt.Errorf("raft: Initialization error: %s", err)
 		return fmt.Errorf("raft: Initialization error: %s", err)
 	}
 	}
 
 
@@ -368,59 +371,12 @@ func (s *Server) Start() error {
 	return nil
 	return nil
 }
 }
 
 
-// Read the configuration for the server.
-func (s *Server) readConf() error {
-	var err error
-	confPath := path.Join(s.path, "conf")
-	s.debugln("readConf.open ", confPath)
-	// open conf file
-	s.confFile, err = os.OpenFile(confPath, os.O_RDWR, 0600)
-
-	if err != nil {
-		if os.IsNotExist(err) {
-			s.confFile, err = os.OpenFile(confPath, os.O_WRONLY|os.O_CREATE, 0600)
-			debugln("readConf.create ", confPath)
-			if err != nil {
-				return err
-			}
-		}
-		return err
-	}
-
-	peerNames := make([]string, 0)
-
-	for {
-		var peerName string
-		_, err = fmt.Fscanf(s.confFile, "%s\n", &peerName)
-
-		if err != nil {
-			if err == io.EOF {
-				s.debugln("server.peer.conf: finish")
-				break
-			}
-			return err
-		}
-		s.debugln("server.peer.conf.read: ", peerName)
-
-		peerNames = append(peerNames, peerName)
-	}
-
-	s.confFile.Truncate(0)
-	s.confFile.Seek(0, os.SEEK_SET)
-
-	for _, peerName := range peerNames {
-		s.AddPeer(peerName)
-	}
-
-	return nil
-}
-
 // Shuts down the server.
 // Shuts down the server.
 func (s *Server) Stop() {
 func (s *Server) Stop() {
 	s.send(&stopValue)
 	s.send(&stopValue)
 	s.mutex.Lock()
 	s.mutex.Lock()
+	defer s.mutex.Unlock()
 	s.log.close()
 	s.log.close()
-	s.mutex.Unlock()
 }
 }
 
 
 // Checks if the server is currently running.
 // Checks if the server is currently running.
@@ -532,24 +488,27 @@ func (s *Server) followerLoop() {
 		case e := <-s.c:
 		case e := <-s.c:
 			if e.target == &stopValue {
 			if e.target == &stopValue {
 				s.setState(Stopped)
 				s.setState(Stopped)
-			} else if command, ok := e.target.(JoinCommand); ok {
-				//If no log entries exist and a self-join command is issued
-				//then immediately become leader and commit entry.
-				if s.log.currentIndex() == 0 && command.NodeName() == s.Name() {
-					s.debugln("selfjoin and promote to leader")
-					s.setState(Leader)
-					s.processCommand(command, e)
-				} else {
+			} else {
+				switch req := e.target.(type) {
+				case JoinCommand:
+					//If no log entries exist and a self-join command is issued
+					//then immediately become leader and commit entry.
+					if s.log.currentIndex() == 0 && req.NodeName() == s.Name() {
+						s.debugln("selfjoin and promote to leader")
+						s.setState(Leader)
+						s.processCommand(req, e)
+					} else {
+						err = NotLeaderError
+					}
+				case *AppendEntriesRequest:
+					e.returnValue, update = s.processAppendEntriesRequest(req)
+				case *RequestVoteRequest:
+					e.returnValue, update = s.processRequestVoteRequest(req)
+				case *SnapshotRequest:
+					e.returnValue = s.processSnapshotRequest(req)
+				default:
 					err = NotLeaderError
 					err = NotLeaderError
 				}
 				}
-			} else if req, ok := e.target.(*AppendEntriesRequest); ok {
-				e.returnValue, update = s.processAppendEntriesRequest(req)
-			} else if req, ok := e.target.(*RequestVoteRequest); ok {
-				e.returnValue, update = s.processRequestVoteRequest(req)
-			} else if req, ok := e.target.(*SnapshotRequest); ok {
-				e.returnValue = s.processSnapshotRequest(req)
-			} else {
-				err = NotLeaderError
 			}
 			}
 
 
 			// Callback to event.
 			// Callback to event.
@@ -629,14 +588,16 @@ func (s *Server) candidateLoop() {
 				var err error
 				var err error
 				if e.target == &stopValue {
 				if e.target == &stopValue {
 					s.setState(Stopped)
 					s.setState(Stopped)
-				} else if _, ok := e.target.(Command); ok {
-					err = NotLeaderError
-				} else if req, ok := e.target.(*AppendEntriesRequest); ok {
-					e.returnValue, _ = s.processAppendEntriesRequest(req)
-				} else if req, ok := e.target.(*RequestVoteRequest); ok {
-					e.returnValue, _ = s.processRequestVoteRequest(req)
+				} else {
+					switch req := e.target.(type) {
+					case Command:
+						err = NotLeaderError
+					case *AppendEntriesRequest:
+						e.returnValue, _ = s.processAppendEntriesRequest(req)
+					case *RequestVoteRequest:
+						e.returnValue, _ = s.processRequestVoteRequest(req)
+					}
 				}
 				}
-
 				// Callback to event.
 				// Callback to event.
 				e.c <- err
 				e.c <- err
 
 
@@ -660,7 +621,7 @@ func (s *Server) candidateLoop() {
 	}
 	}
 }
 }
 
 
-// The event loop that is run when the server is in a Candidate state.
+// The event loop that is run when the server is in a Leader state.
 func (s *Server) leaderLoop() {
 func (s *Server) leaderLoop() {
 	s.setState(Leader)
 	s.setState(Leader)
 	s.syncedPeer = make(map[string]bool)
 	s.syncedPeer = make(map[string]bool)
@@ -682,15 +643,18 @@ func (s *Server) leaderLoop() {
 		case e := <-s.c:
 		case e := <-s.c:
 			if e.target == &stopValue {
 			if e.target == &stopValue {
 				s.setState(Stopped)
 				s.setState(Stopped)
-			} else if command, ok := e.target.(Command); ok {
-				s.processCommand(command, e)
-				continue
-			} else if req, ok := e.target.(*AppendEntriesRequest); ok {
-				e.returnValue, _ = s.processAppendEntriesRequest(req)
-			} else if resp, ok := e.target.(*AppendEntriesResponse); ok {
-				s.processAppendEntriesResponse(resp)
-			} else if req, ok := e.target.(*RequestVoteRequest); ok {
-				e.returnValue, _ = s.processRequestVoteRequest(req)
+			} else {
+				switch req := e.target.(type) {
+				case Command:
+					s.processCommand(req, e)
+					continue
+				case *AppendEntriesRequest:
+					e.returnValue, _ = s.processAppendEntriesRequest(req)
+				case *AppendEntriesResponse:
+					s.processAppendEntriesResponse(req)
+				case *RequestVoteRequest:
+					e.returnValue, _ = s.processRequestVoteRequest(req)
+				}
 			}
 			}
 
 
 			// Callback to event.
 			// Callback to event.
@@ -705,7 +669,7 @@ func (s *Server) leaderLoop() {
 
 
 	// Stop all peers.
 	// Stop all peers.
 	for _, peer := range s.peers {
 	for _, peer := range s.peers {
-		peer.stopHeartbeat()
+		peer.stopHeartbeat(false)
 	}
 	}
 	s.syncedPeer = nil
 	s.syncedPeer = nil
 }
 }
@@ -720,16 +684,18 @@ func (s *Server) snapshotLoop() {
 
 
 		if e.target == &stopValue {
 		if e.target == &stopValue {
 			s.setState(Stopped)
 			s.setState(Stopped)
-		} else if _, ok := e.target.(Command); ok {
-			err = NotLeaderError
-		} else if req, ok := e.target.(*AppendEntriesRequest); ok {
-			e.returnValue, _ = s.processAppendEntriesRequest(req)
-		} else if req, ok := e.target.(*RequestVoteRequest); ok {
-			e.returnValue, _ = s.processRequestVoteRequest(req)
-		} else if req, ok := e.target.(*SnapshotRecoveryRequest); ok {
-			e.returnValue = s.processSnapshotRecoveryRequest(req)
+		} else {
+			switch req := e.target.(type) {
+			case Command:
+				err = NotLeaderError
+			case *AppendEntriesRequest:
+				e.returnValue, _ = s.processAppendEntriesRequest(req)
+			case *RequestVoteRequest:
+				e.returnValue, _ = s.processRequestVoteRequest(req)
+			case *SnapshotRecoveryRequest:
+				e.returnValue = s.processSnapshotRecoveryRequest(req)
+			}
 		}
 		}
-
 		// Callback to event.
 		// Callback to event.
 		e.c <- err
 		e.c <- err
 
 
@@ -959,31 +925,29 @@ func (s *Server) processRequestVoteRequest(req *RequestVoteRequest) (*RequestVot
 //--------------------------------------
 //--------------------------------------
 
 
 // Adds a peer to the server.
 // Adds a peer to the server.
-func (s *Server) AddPeer(name string) error {
+func (s *Server) AddPeer(name string, connectiongString string) error {
 	s.debugln("server.peer.add: ", name, len(s.peers))
 	s.debugln("server.peer.add: ", name, len(s.peers))
-
+	defer s.writeConf()
 	// Do not allow peers to be added twice.
 	// Do not allow peers to be added twice.
 	if s.peers[name] != nil {
 	if s.peers[name] != nil {
 		return nil
 		return nil
 	}
 	}
 
 
-	// Only add the peer if it doesn't have the same name.
-	if s.name != name {
-		// when loading snapshot s.confFile should be nil
-		if s.confFile != nil {
-			_, err := fmt.Fprintln(s.confFile, name)
-			s.debugln("server.peer.conf.write: ", name)
-			if err != nil {
-				return err
-			}
-		}
-		peer := newPeer(s, name, s.heartbeatTimeout)
-		if s.State() == Leader {
-			peer.startHeartbeat()
-		}
-		s.peers[peer.name] = peer
+	// Skip the Peer if it has the same name as the Server
+	if s.name == name {
+		return nil
+	}
+
+	peer := newPeer(s, name, connectiongString, s.heartbeatTimeout)
+
+	if s.State() == Leader {
+		peer.startHeartbeat()
 	}
 	}
 
 
+	s.peers[peer.Name] = peer
+
+	s.debugln("server.peer.conf.write: ", name)
+
 	return nil
 	return nil
 }
 }
 
 
@@ -991,8 +955,12 @@ func (s *Server) AddPeer(name string) error {
 func (s *Server) RemovePeer(name string) error {
 func (s *Server) RemovePeer(name string) error {
 	s.debugln("server.peer.remove: ", name, len(s.peers))
 	s.debugln("server.peer.remove: ", name, len(s.peers))
 
 
-	// Ignore removal of the server itself.
-	if s.name == name {
+	defer s.writeConf()
+
+	if name == s.Name() {
+		// when the removed node restart, it should be able
+		// to know it has been removed before. So we need
+		// to update knownCommitIndex
 		return nil
 		return nil
 	}
 	}
 	// Return error if peer doesn't exist.
 	// Return error if peer doesn't exist.
@@ -1001,23 +969,13 @@ func (s *Server) RemovePeer(name string) error {
 		return fmt.Errorf("raft: Peer not found: %s", name)
 		return fmt.Errorf("raft: Peer not found: %s", name)
 	}
 	}
 
 
-	// TODO: Flush entries to the peer first.
-
 	// Stop peer and remove it.
 	// Stop peer and remove it.
-	peer.stopHeartbeat()
+	if s.State() == Leader {
+		peer.stopHeartbeat(true)
+	}
 
 
 	delete(s.peers, name)
 	delete(s.peers, name)
 
 
-	s.confFile.Truncate(0)
-	s.confFile.Seek(0, os.SEEK_SET)
-
-	for peer := range s.peers {
-		_, err := fmt.Fprintln(s.confFile, peer)
-		if err != nil {
-			return err
-		}
-	}
-
 	return nil
 	return nil
 }
 }
 
 
@@ -1054,14 +1012,13 @@ func (s *Server) TakeSnapshot() error {
 		state = []byte{0}
 		state = []byte{0}
 	}
 	}
 
 
-	var peerNames []string
+	var peers []*Peer
 
 
 	for _, peer := range s.peers {
 	for _, peer := range s.peers {
-		peerNames = append(peerNames, peer.Name())
+		peers = append(peers, peer.clone())
 	}
 	}
-	peerNames = append(peerNames, s.Name())
 
 
-	s.currentSnapshot = &Snapshot{lastIndex, lastTerm, peerNames, state, path}
+	s.currentSnapshot = &Snapshot{lastIndex, lastTerm, peers, state, path}
 
 
 	s.saveSnapshot()
 	s.saveSnapshot()
 
 
@@ -1144,8 +1101,8 @@ func (s *Server) processSnapshotRecoveryRequest(req *SnapshotRecoveryRequest) *S
 	s.peers = make(map[string]*Peer)
 	s.peers = make(map[string]*Peer)
 
 
 	// recovery the cluster configuration
 	// recovery the cluster configuration
-	for _, peerName := range req.Peers {
-		s.AddPeer(peerName)
+	for _, peer := range req.Peers {
+		s.AddPeer(peer.Name, peer.ConnectionString)
 	}
 	}
 
 
 	//update term and index
 	//update term and index
@@ -1237,8 +1194,8 @@ func (s *Server) LoadSnapshot() error {
 		return err
 		return err
 	}
 	}
 
 
-	for _, peerName := range s.lastSnapshot.Peers {
-		s.AddPeer(peerName)
+	for _, peer := range s.lastSnapshot.Peers {
+		s.AddPeer(peer.Name, peer.ConnectionString)
 	}
 	}
 
 
 	s.log.startTerm = s.lastSnapshot.LastTerm
 	s.log.startTerm = s.lastSnapshot.LastTerm
@@ -1248,6 +1205,62 @@ func (s *Server) LoadSnapshot() error {
 	return err
 	return err
 }
 }
 
 
+//--------------------------------------
+// Config File
+//--------------------------------------
+
+func (s *Server) writeConf() {
+
+	peers := make([]*Peer, len(s.peers))
+
+	i := 0
+	for _, peer := range s.peers {
+		peers[i] = peer.clone()
+		i++
+	}
+
+	r := &Config{
+		CommitIndex: s.log.commitIndex,
+		Peers:       peers,
+	}
+
+	b, _ := json.Marshal(r)
+
+	confPath := path.Join(s.path, "conf")
+	tmpConfPath := path.Join(s.path, "conf.tmp")
+
+	err := ioutil.WriteFile(tmpConfPath, b, 0600)
+
+	if err != nil {
+		panic(err)
+	}
+
+	os.Rename(tmpConfPath, confPath)
+}
+
+// Read the configuration for the server.
+func (s *Server) readConf() error {
+	confPath := path.Join(s.path, "conf")
+	s.debugln("readConf.open ", confPath)
+
+	// open conf file
+	b, err := ioutil.ReadFile(confPath)
+
+	if err != nil {
+		return nil
+	}
+
+	conf := &Config{}
+
+	if err = json.Unmarshal(b, conf); err != nil {
+		return err
+	}
+
+	s.log.commitIndex = conf.CommitIndex
+
+	return nil
+}
+
 //--------------------------------------
 //--------------------------------------
 // Debugging
 // Debugging
 //--------------------------------------
 //--------------------------------------

+ 122 - 4
third_party/github.com/coreos/go-raft/server_test.go

@@ -164,10 +164,10 @@ func TestServerPromote(t *testing.T) {
 	lookup := map[string]*Server{}
 	lookup := map[string]*Server{}
 	transporter := &testTransporter{}
 	transporter := &testTransporter{}
 	transporter.sendVoteRequestFunc = func(server *Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse {
 	transporter.sendVoteRequestFunc = func(server *Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse {
-		return lookup[peer.Name()].RequestVote(req)
+		return lookup[peer.Name].RequestVote(req)
 	}
 	}
 	transporter.sendAppendEntriesRequestFunc = func(server *Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse {
 	transporter.sendAppendEntriesRequestFunc = func(server *Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse {
-		return lookup[peer.Name()].AppendEntries(req)
+		return lookup[peer.Name].AppendEntries(req)
 	}
 	}
 	servers := newTestCluster([]string{"1", "2", "3"}, transporter, lookup)
 	servers := newTestCluster([]string{"1", "2", "3"}, transporter, lookup)
 
 
@@ -316,6 +316,124 @@ func TestServerDenyCommandExecutionWhenFollower(t *testing.T) {
 	}
 	}
 }
 }
 
 
+//--------------------------------------
+// Recovery
+//--------------------------------------
+
+// Ensure that a follower cannot execute a command.
+func TestServerRecoverFromPreviousLogAndConf(t *testing.T) {
+	// Initialize the servers.
+	var mutex sync.RWMutex
+	servers := map[string]*Server{}
+
+	transporter := &testTransporter{}
+	transporter.sendVoteRequestFunc = func(server *Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse {
+		mutex.RLock()
+		s := servers[peer.Name]
+		mutex.RUnlock()
+		return s.RequestVote(req)
+	}
+	transporter.sendAppendEntriesRequestFunc = func(server *Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse {
+		mutex.RLock()
+		s := servers[peer.Name]
+		mutex.RUnlock()
+		return s.AppendEntries(req)
+	}
+
+	disTransporter := &testTransporter{}
+	disTransporter.sendVoteRequestFunc = func(server *Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse {
+		return nil
+	}
+	disTransporter.sendAppendEntriesRequestFunc = func(server *Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse {
+		return nil
+	}
+
+	var names []string
+	var paths = make(map[string]string)
+
+	n := 5
+
+	// add n servers
+	for i := 1; i <= n; i++ {
+		names = append(names, strconv.Itoa(i))
+	}
+
+	var leader *Server
+	for _, name := range names {
+		server := newTestServer(name, transporter)
+
+		servers[name] = server
+		paths[name] = server.Path()
+
+		if name == "1" {
+			leader = server
+			server.SetHeartbeatTimeout(testHeartbeatTimeout)
+			server.Start()
+			time.Sleep(testHeartbeatTimeout)
+		} else {
+			server.SetElectionTimeout(testElectionTimeout)
+			server.SetHeartbeatTimeout(testHeartbeatTimeout)
+			server.Start()
+			time.Sleep(testHeartbeatTimeout)
+		}
+		if _, err := leader.Do(&DefaultJoinCommand{Name: name}); err != nil {
+			t.Fatalf("Unable to join server[%s]: %v", name, err)
+		}
+
+	}
+
+	// commit some commands
+	for i := 0; i < 10; i++ {
+		if _, err := leader.Do(&testCommand2{X: 1}); err != nil {
+			t.Fatalf("cannot commit command:", err.Error())
+		}
+	}
+
+	time.Sleep(2 * testHeartbeatTimeout)
+
+	for _, name := range names {
+		server := servers[name]
+		if server.CommitIndex() != 16 {
+			t.Fatalf("%s commitIndex is invalid [%d/%d]", name, server.CommitIndex(), 16)
+		}
+		server.Stop()
+	}
+
+	for _, name := range names {
+		// with old path and disable transportation
+		server := newTestServerWithPath(name, disTransporter, paths[name])
+		servers[name] = server
+
+		server.Start()
+
+		// should only commit to the last join command
+		if server.CommitIndex() != 6 {
+			t.Fatalf("%s recover phase 1 commitIndex is invalid [%d/%d]", name, server.CommitIndex(), 6)
+		}
+
+		// peer conf should be recovered
+		if len(server.Peers()) != 4 {
+			t.Fatalf("%s recover phase 1 peer failed! [%d/%d]", name, len(server.Peers()), 4)
+		}
+	}
+
+	// let nodes talk to each other
+	for _, name := range names {
+		servers[name].SetTransporter(transporter)
+	}
+
+	time.Sleep(2 * testElectionTimeout)
+
+	// should commit to the previous index + 1(nop command when new leader elected)
+	for _, name := range names {
+		server := servers[name]
+		if server.CommitIndex() != 17 {
+			t.Fatalf("%s commitIndex is invalid [%d/%d]", name, server.CommitIndex(), 16)
+		}
+		server.Stop()
+	}
+}
+
 //--------------------------------------
 //--------------------------------------
 // Membership
 // Membership
 //--------------------------------------
 //--------------------------------------
@@ -357,13 +475,13 @@ func TestServerMultiNode(t *testing.T) {
 	transporter := &testTransporter{}
 	transporter := &testTransporter{}
 	transporter.sendVoteRequestFunc = func(server *Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse {
 	transporter.sendVoteRequestFunc = func(server *Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse {
 		mutex.RLock()
 		mutex.RLock()
-		s := servers[peer.name]
+		s := servers[peer.Name]
 		mutex.RUnlock()
 		mutex.RUnlock()
 		return s.RequestVote(req)
 		return s.RequestVote(req)
 	}
 	}
 	transporter.sendAppendEntriesRequestFunc = func(server *Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse {
 	transporter.sendAppendEntriesRequestFunc = func(server *Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse {
 		mutex.RLock()
 		mutex.RLock()
-		s := servers[peer.name]
+		s := servers[peer.Name]
 		mutex.RUnlock()
 		mutex.RUnlock()
 		return s.AppendEntries(req)
 		return s.AppendEntries(req)
 	}
 	}

+ 3 - 3
third_party/github.com/coreos/go-raft/snapshot.go

@@ -21,9 +21,9 @@ type Snapshot struct {
 	LastIndex uint64 `json:"lastIndex"`
 	LastIndex uint64 `json:"lastIndex"`
 	LastTerm  uint64 `json:"lastTerm"`
 	LastTerm  uint64 `json:"lastTerm"`
 	// cluster configuration.
 	// cluster configuration.
-	Peers []string `json: "peers"`
-	State []byte   `json: "state"`
-	Path  string   `json: "path"`
+	Peers []*Peer `json: "peers"`
+	State []byte  `json: "state"`
+	Path  string  `json: "path"`
 }
 }
 
 
 // Save the snapshot to a file
 // Save the snapshot to a file

+ 22 - 4
third_party/github.com/coreos/go-raft/snapshot_recovery_request.go

@@ -12,7 +12,7 @@ type SnapshotRecoveryRequest struct {
 	LeaderName string
 	LeaderName string
 	LastIndex  uint64
 	LastIndex  uint64
 	LastTerm   uint64
 	LastTerm   uint64
-	Peers      []string
+	Peers      []*Peer
 	State      []byte
 	State      []byte
 }
 }
 
 
@@ -36,11 +36,21 @@ func newSnapshotRecoveryRequest(leaderName string, snapshot *Snapshot) *Snapshot
 // Encodes the SnapshotRecoveryRequest to a buffer. Returns the number of bytes
 // Encodes the SnapshotRecoveryRequest to a buffer. Returns the number of bytes
 // written and any error that may have occurred.
 // written and any error that may have occurred.
 func (req *SnapshotRecoveryRequest) encode(w io.Writer) (int, error) {
 func (req *SnapshotRecoveryRequest) encode(w io.Writer) (int, error) {
+
+	protoPeers := make([]*protobuf.ProtoSnapshotRecoveryRequest_ProtoPeer, len(req.Peers))
+
+	for i, peer := range req.Peers {
+		protoPeers[i] = &protobuf.ProtoSnapshotRecoveryRequest_ProtoPeer{
+			Name:             proto.String(peer.Name),
+			ConnectionString: proto.String(peer.ConnectionString),
+		}
+	}
+
 	pb := &protobuf.ProtoSnapshotRecoveryRequest{
 	pb := &protobuf.ProtoSnapshotRecoveryRequest{
 		LeaderName: proto.String(req.LeaderName),
 		LeaderName: proto.String(req.LeaderName),
 		LastIndex:  proto.Uint64(req.LastIndex),
 		LastIndex:  proto.Uint64(req.LastIndex),
 		LastTerm:   proto.Uint64(req.LastTerm),
 		LastTerm:   proto.Uint64(req.LastTerm),
-		Peers:      req.Peers,
+		Peers:      protoPeers,
 		State:      req.State,
 		State:      req.State,
 	}
 	}
 	p, err := proto.Marshal(pb)
 	p, err := proto.Marshal(pb)
@@ -62,7 +72,7 @@ func (req *SnapshotRecoveryRequest) decode(r io.Reader) (int, error) {
 
 
 	totalBytes := len(data)
 	totalBytes := len(data)
 
 
-	pb := &protobuf.ProtoSnapshotRequest{}
+	pb := &protobuf.ProtoSnapshotRecoveryRequest{}
 	if err = proto.Unmarshal(data, pb); err != nil {
 	if err = proto.Unmarshal(data, pb); err != nil {
 		return -1, err
 		return -1, err
 	}
 	}
@@ -70,8 +80,16 @@ func (req *SnapshotRecoveryRequest) decode(r io.Reader) (int, error) {
 	req.LeaderName = pb.GetLeaderName()
 	req.LeaderName = pb.GetLeaderName()
 	req.LastIndex = pb.GetLastIndex()
 	req.LastIndex = pb.GetLastIndex()
 	req.LastTerm = pb.GetLastTerm()
 	req.LastTerm = pb.GetLastTerm()
-	req.Peers = req.Peers
 	req.State = req.State
 	req.State = req.State
 
 
+	req.Peers = make([]*Peer, len(pb.Peers))
+
+	for i, peer := range pb.Peers {
+		req.Peers[i] = &Peer{
+			Name:             peer.GetName(),
+			ConnectionString: peer.GetConnectionString(),
+		}
+	}
+
 	return totalBytes, nil
 	return totalBytes, nil
 }
 }

+ 6 - 1
third_party/github.com/coreos/go-raft/test.go

@@ -69,6 +69,11 @@ func newTestServer(name string, transporter Transporter) *Server {
 	return server
 	return server
 }
 }
 
 
+func newTestServerWithPath(name string, transporter Transporter, p string) *Server {
+	server, _ := NewServer(name, p, transporter, nil, nil)
+	return server
+}
+
 func newTestServerWithLog(name string, transporter Transporter, entries []*LogEntry) *Server {
 func newTestServerWithLog(name string, transporter Transporter, entries []*LogEntry) *Server {
 	server := newTestServer(name, transporter)
 	server := newTestServer(name, transporter)
 	f, err := os.Create(server.LogPath())
 	f, err := os.Create(server.LogPath())
@@ -100,7 +105,7 @@ func newTestCluster(names []string, transporter Transporter, lookup map[string]*
 		server.SetHeartbeatTimeout(testHeartbeatTimeout)
 		server.SetHeartbeatTimeout(testHeartbeatTimeout)
 		server.Start()
 		server.Start()
 		for _, peer := range servers {
 		for _, peer := range servers {
-			server.AddPeer(peer.Name())
+			server.AddPeer(peer.Name(), "")
 		}
 		}
 	}
 	}
 	return servers
 	return servers