Fix race conditions.

pull/820/head
Ben Johnson 2013-05-05 13:48:15 -06:00
parent ef423e8ad5
commit 776a2ccff9
3 changed files with 18 additions and 3 deletions

View File

@ -123,7 +123,9 @@ func (t *ElectionTimer) Reset() {
// through to the election timer's external channel.
if internalTimer != nil {
if v, ok := <-internalTimer.C; ok {
t.mutex.Lock()
t.C <- v
t.mutex.Unlock()
}
}
}()

View File

@ -296,7 +296,7 @@ loop:
for _, _peer := range s.peers {
peer := _peer
go func() {
peer.internalFlush()
peer.flush()
}()
}
}

View File

@ -2,6 +2,7 @@ package raft
import (
"reflect"
"sync"
"testing"
)
@ -285,15 +286,22 @@ func TestServerMultiNode(t *testing.T) {
warn("== begin multi-node ==")
// Initialize the servers.
var mutex sync.Mutex
names := []string{"1", "2", "3"}
servers := map[string]*Server{}
for _, name := range names {
server := newTestServer(name)
server.DoHandler = func(server *Server, peer *Peer, command Command) error {
return servers[peer.name].Do(command)
mutex.Lock()
s := servers[peer.name]
mutex.Unlock()
return s.Do(command)
}
server.AppendEntriesHandler = func(server *Server, peer *Peer, req *AppendEntriesRequest) (*AppendEntriesResponse, error) {
return servers[peer.name].AppendEntries(req)
mutex.Lock()
s := servers[peer.name]
mutex.Unlock()
return s.AppendEntries(req)
}
if err := server.Start(); err != nil {
t.Fatalf("Unable to start server[%s]: %v", name, err)
@ -301,10 +309,15 @@ func TestServerMultiNode(t *testing.T) {
if err := server.Join("1"); err != nil {
t.Fatalf("Unable to join server[%s]: %v", name, err)
}
mutex.Lock()
servers[name] = server
mutex.Unlock()
}
// Check that two peers exist on leader.
mutex.Lock()
defer mutex.Unlock()
leader := servers["1"]
if leader.MemberCount() != 3 {
t.Fatalf("Expected member count to be 3, got %v", leader.MemberCount())