2014-09-03 20:28:25 +00:00
|
|
|
package raft_test
|
|
|
|
|
|
|
|
import (
|
2014-09-24 23:15:21 +00:00
|
|
|
"encoding/binary"
|
2014-09-09 23:01:48 +00:00
|
|
|
"encoding/json"
|
2014-09-26 20:57:33 +00:00
|
|
|
"fmt"
|
2014-09-24 23:15:21 +00:00
|
|
|
"io"
|
2014-09-26 20:57:33 +00:00
|
|
|
"io/ioutil"
|
2014-09-03 20:28:25 +00:00
|
|
|
"log"
|
2014-09-08 21:39:05 +00:00
|
|
|
"net/url"
|
2014-09-03 20:28:25 +00:00
|
|
|
"os"
|
2015-02-06 19:46:50 +00:00
|
|
|
"strings"
|
2014-09-09 23:01:48 +00:00
|
|
|
"sync"
|
2014-09-03 20:28:25 +00:00
|
|
|
"testing"
|
|
|
|
|
|
|
|
"github.com/influxdb/influxdb/raft"
|
|
|
|
)
|
|
|
|
|
2014-09-27 22:24:59 +00:00
|
|
|
// Ensure that opening an already open log returns an error.
|
|
|
|
func TestLog_Open_ErrOpen(t *testing.T) {
|
2015-02-05 01:59:06 +00:00
|
|
|
l := NewInitializedLog(&url.URL{Host: "log0"})
|
|
|
|
defer l.Close()
|
|
|
|
if err := l.Open(tempfile()); err != raft.ErrOpen {
|
2014-09-27 22:24:59 +00:00
|
|
|
t.Fatal("expected error")
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Ensure that a log can be checked for being open.
|
|
|
|
func TestLog_Opened(t *testing.T) {
|
2015-02-05 01:59:06 +00:00
|
|
|
l := NewInitializedLog(&url.URL{Host: "log0"})
|
|
|
|
if l.Opened() != true {
|
2014-09-27 22:24:59 +00:00
|
|
|
t.Fatalf("expected open")
|
|
|
|
}
|
2015-02-05 01:59:06 +00:00
|
|
|
l.Close()
|
|
|
|
if l.Opened() != false {
|
2014-09-27 22:24:59 +00:00
|
|
|
t.Fatalf("expected closed")
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Ensure that reopening an existing log will restore its ID.
|
|
|
|
func TestLog_Reopen(t *testing.T) {
|
2015-02-05 01:59:06 +00:00
|
|
|
l := NewInitializedLog(&url.URL{Host: "log0"})
|
|
|
|
if l.ID() != 1 {
|
2014-09-27 22:24:59 +00:00
|
|
|
t.Fatalf("expected id == 1")
|
|
|
|
}
|
2015-02-05 01:59:06 +00:00
|
|
|
path := l.Path()
|
2014-09-27 22:24:59 +00:00
|
|
|
|
|
|
|
// Close log and make sure id is cleared.
|
2015-02-05 01:59:06 +00:00
|
|
|
l.Log.Close()
|
|
|
|
if l.ID() != 0 {
|
2014-09-27 22:24:59 +00:00
|
|
|
t.Fatalf("expected id == 0")
|
|
|
|
}
|
|
|
|
|
|
|
|
// Re-open and ensure id is restored.
|
2015-02-05 01:59:06 +00:00
|
|
|
if err := l.Open(path); err != nil {
|
2014-09-27 22:24:59 +00:00
|
|
|
t.Fatalf("unexpected error: %s", err)
|
|
|
|
}
|
2015-02-05 01:59:06 +00:00
|
|
|
if id := l.ID(); id != 1 {
|
|
|
|
t.Fatalf("unexpected id: %d", id)
|
2014-09-27 22:24:59 +00:00
|
|
|
}
|
2015-02-05 01:59:06 +00:00
|
|
|
l.Close()
|
2014-09-27 22:24:59 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// Ensure that a single node-cluster can apply a log entry.
|
|
|
|
func TestLog_Apply(t *testing.T) {
|
2015-02-05 01:59:06 +00:00
|
|
|
l := NewInitializedLog(&url.URL{Host: "log0"})
|
|
|
|
defer l.Close()
|
2014-09-27 22:24:59 +00:00
|
|
|
|
|
|
|
// Apply a command.
|
2015-02-05 01:59:06 +00:00
|
|
|
index, err := l.Apply([]byte("foo"))
|
2014-09-27 22:24:59 +00:00
|
|
|
if err != nil {
|
|
|
|
t.Fatalf("unexpected error: %s", err)
|
|
|
|
} else if index != 2 {
|
|
|
|
t.Fatalf("unexpected index: %d", index)
|
|
|
|
}
|
|
|
|
|
2015-02-05 01:59:06 +00:00
|
|
|
// Force apply cycle and then signal wait.
|
|
|
|
go func() { l.Clock.apply() }()
|
|
|
|
|
2014-09-27 22:24:59 +00:00
|
|
|
// Single node clusters should apply to FSM immediately.
|
2015-02-05 01:59:06 +00:00
|
|
|
l.Wait(index)
|
|
|
|
if n := len(l.FSM.Commands); n != 1 {
|
2014-09-27 22:24:59 +00:00
|
|
|
t.Fatalf("unexpected command count: %d", n)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2015-02-05 01:59:06 +00:00
|
|
|
// Ensure that a node has no configuration after it's closed.
|
|
|
|
func TestLog_Config_Closed(t *testing.T) {
|
|
|
|
l := NewInitializedLog(&url.URL{Host: "log0"})
|
|
|
|
defer l.Close()
|
|
|
|
l.Log.Close()
|
|
|
|
if l.Config() != nil {
|
|
|
|
t.Fatal("expected nil config")
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Ensure that log ids in a cluster are set sequentially.
|
|
|
|
func TestCluster_ID_Sequential(t *testing.T) {
|
|
|
|
c := NewCluster()
|
2014-09-27 22:24:59 +00:00
|
|
|
defer c.Close()
|
2015-02-05 01:59:06 +00:00
|
|
|
for i, l := range c.Logs {
|
|
|
|
if l.ID() != uint64(i+1) {
|
|
|
|
t.Fatalf("expected id: %d, got: %d", i+1, l.ID())
|
2014-09-27 22:24:59 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Ensure that cluster starts with one leader and multiple followers.
|
2015-02-05 01:59:06 +00:00
|
|
|
func TestCluster_State(t *testing.T) {
|
|
|
|
c := NewCluster()
|
2014-09-27 22:24:59 +00:00
|
|
|
defer c.Close()
|
2015-02-05 01:59:06 +00:00
|
|
|
if state := c.Logs[0].State(); state != raft.Leader {
|
2014-09-27 22:24:59 +00:00
|
|
|
t.Fatalf("unexpected state(0): %s", state)
|
|
|
|
}
|
2015-02-05 01:59:06 +00:00
|
|
|
if state := c.Logs[1].State(); state != raft.Follower {
|
2014-09-27 22:24:59 +00:00
|
|
|
t.Fatalf("unexpected state(1): %s", state)
|
|
|
|
}
|
2015-02-05 01:59:06 +00:00
|
|
|
if state := c.Logs[2].State(); state != raft.Follower {
|
2014-09-27 22:24:59 +00:00
|
|
|
t.Fatalf("unexpected state(2): %s", state)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Ensure that each node's configuration matches in the cluster.
|
2015-02-05 01:59:06 +00:00
|
|
|
func TestCluster_Config(t *testing.T) {
|
|
|
|
c := NewCluster()
|
2014-09-27 22:24:59 +00:00
|
|
|
defer c.Close()
|
2015-02-05 01:59:06 +00:00
|
|
|
config := jsonify(c.Logs[0].Config())
|
|
|
|
for _, l := range c.Logs[1:] {
|
|
|
|
if b := jsonify(l.Config()); config != b {
|
|
|
|
t.Fatalf("config mismatch(%d):\n\nexp=%s\n\ngot:%s\n\n", l.ID(), config, b)
|
2014-09-27 22:24:59 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2015-02-05 01:59:06 +00:00
|
|
|
// Ensure that a command can be applied to a cluster and distributed appropriately.
|
|
|
|
func TestCluster_Apply(t *testing.T) {
|
|
|
|
c := NewCluster()
|
2014-09-27 22:24:59 +00:00
|
|
|
defer c.Close()
|
|
|
|
|
|
|
|
// Apply a command.
|
2015-02-05 01:59:06 +00:00
|
|
|
leader := c.Logs[0]
|
|
|
|
index, err := leader.Apply([]byte("foo"))
|
2014-09-27 22:24:59 +00:00
|
|
|
if err != nil {
|
|
|
|
t.Fatalf("unexpected error: %s", err)
|
|
|
|
} else if index != 4 {
|
|
|
|
t.Fatalf("unexpected index: %d", index)
|
|
|
|
}
|
2015-02-05 01:59:06 +00:00
|
|
|
c.Logs[1].MustWaitUncommitted(4)
|
|
|
|
c.Logs[2].MustWaitUncommitted(4)
|
2014-09-27 22:24:59 +00:00
|
|
|
|
|
|
|
// Should not apply immediately.
|
2015-02-05 01:59:06 +00:00
|
|
|
if n := len(leader.FSM.Commands); n != 0 {
|
2014-09-27 22:24:59 +00:00
|
|
|
t.Fatalf("unexpected pre-heartbeat command count: %d", n)
|
|
|
|
}
|
|
|
|
|
2015-02-05 01:59:06 +00:00
|
|
|
// Run the heartbeat on the leader and have all logs apply.
|
2014-09-27 22:24:59 +00:00
|
|
|
// Only the leader should have the changes applied.
|
2015-02-05 01:59:06 +00:00
|
|
|
c.Logs[0].Clock.heartbeat()
|
|
|
|
c.Logs[0].Clock.apply()
|
|
|
|
c.Logs[1].Clock.apply()
|
|
|
|
c.Logs[2].Clock.apply()
|
|
|
|
if n := len(c.Logs[0].FSM.Commands); n != 1 {
|
2014-09-27 22:24:59 +00:00
|
|
|
t.Fatalf("unexpected command count(0): %d", n)
|
2014-09-03 20:28:25 +00:00
|
|
|
}
|
2015-02-05 01:59:06 +00:00
|
|
|
if n := len(c.Logs[1].FSM.Commands); n != 0 {
|
2014-09-27 22:24:59 +00:00
|
|
|
t.Fatalf("unexpected command count(1): %d", n)
|
2014-09-07 03:10:06 +00:00
|
|
|
}
|
2015-02-05 01:59:06 +00:00
|
|
|
if n := len(c.Logs[2].FSM.Commands); n != 0 {
|
2014-09-27 22:24:59 +00:00
|
|
|
t.Fatalf("unexpected command count(2): %d", n)
|
|
|
|
}
|
|
|
|
|
2015-02-05 01:59:06 +00:00
|
|
|
// Wait for another heartbeat and all logs should be in sync.
|
|
|
|
c.Logs[0].Clock.heartbeat()
|
|
|
|
c.Logs[1].Clock.apply()
|
|
|
|
c.Logs[2].Clock.apply()
|
|
|
|
if n := len(c.Logs[1].FSM.Commands); n != 1 {
|
2014-09-27 22:24:59 +00:00
|
|
|
t.Fatalf("unexpected command count(1): %d", n)
|
|
|
|
}
|
2015-02-05 01:59:06 +00:00
|
|
|
if n := len(c.Logs[2].FSM.Commands); n != 1 {
|
2014-09-27 22:24:59 +00:00
|
|
|
t.Fatalf("unexpected command count(2): %d", n)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2014-09-27 23:28:03 +00:00
|
|
|
// Ensure that a new leader can be elected.
|
|
|
|
func TestLog_Elect(t *testing.T) {
|
2015-02-05 01:59:06 +00:00
|
|
|
c := NewCluster()
|
2014-09-27 23:28:03 +00:00
|
|
|
defer c.Close()
|
|
|
|
|
|
|
|
// Stop leader.
|
2015-02-05 01:59:06 +00:00
|
|
|
path := c.Logs[0].Path()
|
|
|
|
c.Logs[0].Log.Close()
|
2014-09-27 23:28:03 +00:00
|
|
|
|
2015-02-05 01:59:06 +00:00
|
|
|
// Signal election on node 1. Then heartbeat to establish leadership.
|
|
|
|
c.Logs[1].Clock.now = c.Logs[1].Clock.now.Add(raft.DefaultElectionTimeout)
|
|
|
|
c.Logs[1].Clock.election()
|
|
|
|
c.Logs[1].Clock.heartbeat()
|
2014-09-27 23:28:03 +00:00
|
|
|
|
2015-02-05 01:59:06 +00:00
|
|
|
// Ensure node 1 is elected in the next term.
|
|
|
|
if state := c.Logs[1].State(); state != raft.Leader {
|
|
|
|
t.Fatalf("expected node 1 to move to leader: %s", state)
|
|
|
|
} else if term := c.Logs[1].Term(); term != 2 {
|
|
|
|
t.Fatalf("expected term 2: got %d", term)
|
2014-09-27 23:28:03 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// Restart leader and make sure it rejoins as a follower.
|
2015-02-05 01:59:06 +00:00
|
|
|
if err := c.Logs[0].Open(path); err != nil {
|
2014-09-27 23:28:03 +00:00
|
|
|
t.Fatalf("unexpected open error: %s", err)
|
|
|
|
}
|
|
|
|
|
2015-02-05 01:59:06 +00:00
|
|
|
// Wait for a heartbeat and verify the node 1 is still the leader.
|
|
|
|
c.Logs[1].Clock.heartbeat()
|
|
|
|
if state := c.Logs[1].State(); state != raft.Leader {
|
|
|
|
t.Fatalf("node 1 unexpectedly deposed: %s", state)
|
|
|
|
} else if term := c.Logs[1].Term(); term != 2 {
|
|
|
|
t.Fatalf("expected node 0 to go to term 2: got term %d", term)
|
2014-09-27 23:28:03 +00:00
|
|
|
}
|
|
|
|
|
2014-09-28 17:38:08 +00:00
|
|
|
// Apply a command and ensure it's replicated.
|
2015-02-05 01:59:06 +00:00
|
|
|
index, err := c.Logs[1].Log.Apply([]byte("abc"))
|
2014-09-28 17:38:08 +00:00
|
|
|
if err != nil {
|
|
|
|
t.Fatalf("unexpected apply error: %s", err)
|
|
|
|
}
|
2015-02-05 01:59:06 +00:00
|
|
|
|
|
|
|
c.MustWaitUncommitted(index)
|
|
|
|
c.Logs[1].Clock.heartbeat()
|
|
|
|
c.Logs[1].Clock.heartbeat()
|
|
|
|
c.Logs[0].Clock.apply()
|
|
|
|
c.Logs[1].Clock.apply()
|
|
|
|
c.Logs[2].Clock.apply()
|
|
|
|
if err := c.Logs[0].Wait(index); err != nil {
|
2014-09-28 17:38:08 +00:00
|
|
|
t.Fatalf("unexpected wait error: %s", err)
|
|
|
|
}
|
2014-09-27 23:28:03 +00:00
|
|
|
}
|
|
|
|
|
2014-09-27 22:24:59 +00:00
|
|
|
// Ensure that state can be stringified.
|
|
|
|
func TestState_String(t *testing.T) {
|
|
|
|
var tests = []struct {
|
|
|
|
state raft.State
|
|
|
|
s string
|
|
|
|
}{
|
|
|
|
{raft.Stopped, "stopped"},
|
|
|
|
{raft.Follower, "follower"},
|
|
|
|
{raft.Candidate, "candidate"},
|
|
|
|
{raft.Leader, "leader"},
|
|
|
|
{raft.State(50), "unknown"},
|
|
|
|
}
|
|
|
|
for i, tt := range tests {
|
|
|
|
if tt.state.String() != tt.s {
|
|
|
|
t.Errorf("%d. mismatch: %s != %s", i, tt.state.String(), tt.s)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2015-02-06 19:46:50 +00:00
|
|
|
func BenchmarkLogApply1(b *testing.B) { benchmarkLogApply(b, 1) }
|
|
|
|
func BenchmarkLogApply2(b *testing.B) { benchmarkLogApply(b, 2) }
|
|
|
|
func BenchmarkLogApply3(b *testing.B) { benchmarkLogApply(b, 3) }
|
|
|
|
|
|
|
|
// Benchmarks an n-node cluster connected through an in-memory transport.
|
|
|
|
func benchmarkLogApply(b *testing.B, logN int) {
|
|
|
|
warnf("== BenchmarkLogApply (%d) ====================================", b.N)
|
|
|
|
|
|
|
|
logs := make([]*raft.Log, logN)
|
|
|
|
t := NewTransport()
|
|
|
|
var ptrs []string
|
|
|
|
for i := 0; i < logN; i++ {
|
|
|
|
// Create log.
|
|
|
|
l := raft.NewLog()
|
|
|
|
l.URL = &url.URL{Host: fmt.Sprintf("log%d", i)}
|
|
|
|
l.FSM = &BenchmarkFSM{}
|
|
|
|
l.DebugEnabled = true
|
|
|
|
l.Transport = t
|
|
|
|
t.register(l)
|
|
|
|
|
|
|
|
// Open log.
|
|
|
|
if err := l.Open(tempfile()); err != nil {
|
|
|
|
b.Fatalf("open: %s", err)
|
|
|
|
}
|
|
|
|
|
|
|
|
// Initialize or join.
|
|
|
|
if i == 0 {
|
|
|
|
if err := l.Initialize(); err != nil {
|
|
|
|
b.Fatalf("initialize: %s", err)
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
if err := l.Join(logs[0].URL); err != nil {
|
|
|
|
b.Fatalf("initialize: %s", err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
ptrs = append(ptrs, fmt.Sprintf("%d/%p", i, l))
|
|
|
|
logs[i] = l
|
|
|
|
}
|
|
|
|
warn("LOGS:", strings.Join(ptrs, " "))
|
|
|
|
b.ResetTimer()
|
|
|
|
|
|
|
|
// Apply commands to leader.
|
|
|
|
var index uint64
|
|
|
|
var err error
|
|
|
|
for i := 0; i < b.N; i++ {
|
|
|
|
index, err = logs[0].Apply(make([]byte, 50))
|
|
|
|
if err != nil {
|
|
|
|
b.Fatalf("apply: %s", err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Wait for all logs to catch up.
|
|
|
|
for i, l := range logs {
|
|
|
|
if err := l.Wait(index); err != nil {
|
|
|
|
b.Fatalf("wait(%d): %s", i, err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
b.StopTimer()
|
|
|
|
|
|
|
|
// Verify FSM indicies match.
|
|
|
|
for i, l := range logs {
|
|
|
|
if fsm := l.FSM.(*BenchmarkFSM); index != fsm.index {
|
|
|
|
b.Errorf("fsm index mismatch(%d): exp=%d, got=%d", i, index, fsm.index)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// BenchmarkFSM represents a state machine that records the command count.
|
|
|
|
type BenchmarkFSM struct {
|
|
|
|
index uint64
|
|
|
|
}
|
|
|
|
|
|
|
|
// MustApply updates the index.
|
|
|
|
func (fsm *BenchmarkFSM) MustApply(entry *raft.LogEntry) { fsm.index = entry.Index }
|
|
|
|
|
|
|
|
// Index returns the highest applied index.
|
|
|
|
func (fsm *BenchmarkFSM) Index() (uint64, error) { return fsm.index, nil }
|
|
|
|
|
|
|
|
// Snapshot writes the FSM's index as the snapshot.
|
|
|
|
func (fsm *BenchmarkFSM) Snapshot(w io.Writer) (uint64, error) {
|
|
|
|
return fsm.index, binary.Write(w, binary.BigEndian, fsm.index)
|
|
|
|
}
|
|
|
|
|
|
|
|
// Restore reads the snapshot from the reader.
|
|
|
|
func (fsm *BenchmarkFSM) Restore(r io.Reader) error {
|
|
|
|
return binary.Read(r, binary.BigEndian, &fsm.index)
|
|
|
|
}
|
|
|
|
|
2014-09-27 22:24:59 +00:00
|
|
|
// Cluster represents a collection of nodes that share the same mock clock.
|
|
|
|
type Cluster struct {
|
2015-02-05 01:59:06 +00:00
|
|
|
Logs []*Log
|
2014-09-07 03:10:06 +00:00
|
|
|
}
|
|
|
|
|
2015-02-05 01:59:06 +00:00
|
|
|
// NewCluster creates a new 3 log cluster.
|
|
|
|
func NewCluster() *Cluster {
|
2014-09-27 22:24:59 +00:00
|
|
|
c := &Cluster{}
|
2015-02-05 01:59:06 +00:00
|
|
|
t := NewTransport()
|
|
|
|
|
|
|
|
logN := 3
|
|
|
|
for i := 0; i < logN; i++ {
|
|
|
|
l := NewLog(&url.URL{Host: fmt.Sprintf("log%d", i)})
|
|
|
|
l.Transport = t
|
|
|
|
c.Logs = append(c.Logs, l)
|
|
|
|
t.register(l.Log)
|
|
|
|
warnf("Log %s: %p", l.URL.String(), l.Log)
|
|
|
|
}
|
|
|
|
warn("")
|
|
|
|
|
|
|
|
// Initialize leader.
|
|
|
|
c.Logs[0].MustOpen()
|
|
|
|
c.Logs[0].MustInitialize()
|
|
|
|
|
|
|
|
// Join second node.
|
2015-02-06 19:46:50 +00:00
|
|
|
c.Logs[1].MustOpen()
|
2015-02-05 01:59:06 +00:00
|
|
|
go func() {
|
|
|
|
c.Logs[0].MustWaitUncommitted(2)
|
|
|
|
c.Logs[0].Clock.apply()
|
|
|
|
}()
|
|
|
|
if err := c.Logs[1].Join(c.Logs[0].URL); err != nil {
|
|
|
|
panic("join: " + err.Error())
|
|
|
|
}
|
2015-02-06 19:46:50 +00:00
|
|
|
c.Logs[0].Clock.heartbeat()
|
|
|
|
c.Logs[1].MustWaitUncommitted(2)
|
|
|
|
c.Logs[1].Clock.apply()
|
|
|
|
c.Logs[0].Clock.heartbeat()
|
2015-02-05 01:59:06 +00:00
|
|
|
|
|
|
|
// Join third node.
|
2015-02-06 19:46:50 +00:00
|
|
|
c.Logs[2].MustOpen()
|
2015-02-05 01:59:06 +00:00
|
|
|
go func() {
|
|
|
|
c.Logs[0].MustWaitUncommitted(3)
|
2015-02-06 19:46:50 +00:00
|
|
|
c.Logs[1].MustWaitUncommitted(3)
|
2015-02-05 01:59:06 +00:00
|
|
|
c.Logs[0].Clock.heartbeat()
|
|
|
|
c.Logs[0].Clock.apply()
|
|
|
|
c.Logs[1].Clock.apply()
|
|
|
|
c.Logs[2].Clock.apply()
|
|
|
|
}()
|
|
|
|
if err := c.Logs[2].Log.Join(c.Logs[0].Log.URL); err != nil {
|
|
|
|
panic("join: " + err.Error())
|
|
|
|
}
|
|
|
|
|
|
|
|
// Heartbeart final commit index to all nodes and reapply.
|
|
|
|
c.Logs[0].Clock.heartbeat()
|
|
|
|
c.Logs[1].Clock.apply()
|
|
|
|
c.Logs[2].Clock.apply()
|
2014-09-27 22:24:59 +00:00
|
|
|
|
|
|
|
return c
|
2014-09-03 20:28:25 +00:00
|
|
|
}
|
|
|
|
|
2015-02-05 01:59:06 +00:00
|
|
|
// Close closes all logs in the cluster.
|
2014-09-27 22:24:59 +00:00
|
|
|
func (c *Cluster) Close() {
|
2015-02-05 01:59:06 +00:00
|
|
|
for _, l := range c.Logs {
|
|
|
|
l.Close()
|
2014-09-08 21:39:05 +00:00
|
|
|
}
|
2014-09-27 22:24:59 +00:00
|
|
|
}
|
|
|
|
|
2015-02-05 01:59:06 +00:00
|
|
|
// Leader returns the leader log with the highest term.
|
|
|
|
func (c *Cluster) Leader() *Log {
|
|
|
|
var leader *Log
|
|
|
|
for _, l := range c.Logs {
|
|
|
|
if l.State() == raft.Leader && (leader == nil || leader.Log.Term() < l.Term()) {
|
|
|
|
leader = l
|
|
|
|
}
|
2014-09-03 20:28:25 +00:00
|
|
|
}
|
2015-02-05 01:59:06 +00:00
|
|
|
return leader
|
2014-09-03 20:28:25 +00:00
|
|
|
}
|
|
|
|
|
2015-02-05 01:59:06 +00:00
|
|
|
// WaitUncommitted waits until all logs in the cluster have reached a given uncomiitted index.
|
|
|
|
func (c *Cluster) MustWaitUncommitted(index uint64) {
|
|
|
|
for _, l := range c.Logs {
|
|
|
|
l.MustWaitUncommitted(index)
|
|
|
|
}
|
|
|
|
}
|
2014-09-27 22:24:59 +00:00
|
|
|
|
2015-02-05 01:59:06 +00:00
|
|
|
// flush issues messages to cycle all logs.
|
|
|
|
func (c *Cluster) flush() {
|
|
|
|
for _, l := range c.Logs {
|
|
|
|
l.Clock.heartbeat()
|
|
|
|
l.Clock.apply()
|
2014-09-27 23:28:03 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2015-02-05 01:59:06 +00:00
|
|
|
// Log represents a test log.
|
|
|
|
type Log struct {
|
|
|
|
*raft.Log
|
|
|
|
Clock *Clock
|
|
|
|
FSM *FSM
|
2014-09-03 20:28:25 +00:00
|
|
|
}
|
|
|
|
|
2015-02-05 01:59:06 +00:00
|
|
|
// NewLog returns a new instance of Log.
|
|
|
|
func NewLog(u *url.URL) *Log {
|
|
|
|
l := &Log{Log: raft.NewLog(), Clock: NewClock(), FSM: &FSM{}}
|
|
|
|
l.URL = u
|
|
|
|
l.Log.FSM = l.FSM
|
|
|
|
l.Log.Clock = l.Clock
|
|
|
|
l.Rand = seq()
|
|
|
|
l.DebugEnabled = true
|
2014-09-27 22:24:59 +00:00
|
|
|
if !testing.Verbose() {
|
2015-02-05 01:59:06 +00:00
|
|
|
l.Logger = log.New(ioutil.Discard, "", 0)
|
2014-09-27 22:24:59 +00:00
|
|
|
}
|
2015-02-05 01:59:06 +00:00
|
|
|
return l
|
2014-09-03 20:28:25 +00:00
|
|
|
}
|
2014-09-07 03:10:06 +00:00
|
|
|
|
2015-02-05 01:59:06 +00:00
|
|
|
// NewInitializedLog returns a new initialized Node.
|
|
|
|
func NewInitializedLog(u *url.URL) *Log {
|
|
|
|
l := NewLog(u)
|
|
|
|
l.MustOpen()
|
|
|
|
l.MustInitialize()
|
|
|
|
return l
|
2014-09-27 22:24:59 +00:00
|
|
|
}
|
|
|
|
|
2015-02-05 01:59:06 +00:00
|
|
|
// MustOpen opens the log. Panic on error.
|
|
|
|
func (l *Log) MustOpen() {
|
|
|
|
if err := l.Open(tempfile()); err != nil {
|
2014-09-27 22:24:59 +00:00
|
|
|
panic("open: " + err.Error())
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2015-02-05 01:59:06 +00:00
|
|
|
// MustInitialize initializes the log. Panic on error.
|
|
|
|
func (l *Log) MustInitialize() {
|
2015-02-06 19:46:50 +00:00
|
|
|
go func() {
|
|
|
|
l.MustWaitUncommitted(1)
|
|
|
|
l.Clock.apply()
|
|
|
|
}()
|
2015-02-05 01:59:06 +00:00
|
|
|
if err := l.Initialize(); err != nil {
|
|
|
|
panic("initialize: " + err.Error())
|
2014-09-27 22:24:59 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2015-02-05 01:59:06 +00:00
|
|
|
// Close closes the log and HTTP server.
|
|
|
|
func (l *Log) Close() error {
|
|
|
|
defer os.RemoveAll(l.Log.Path())
|
|
|
|
_ = l.Log.Close()
|
|
|
|
return nil
|
2014-09-27 22:24:59 +00:00
|
|
|
}
|
|
|
|
|
2015-02-05 01:59:06 +00:00
|
|
|
// MustWaitUncommitted waits for at least a given uncommitted index. Panic on error.
|
|
|
|
func (l *Log) MustWaitUncommitted(index uint64) {
|
|
|
|
if err := l.Log.WaitUncommitted(index); err != nil {
|
|
|
|
panic(l.URL.String() + " wait uncommitted: " + err.Error())
|
|
|
|
}
|
2014-09-27 22:24:59 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// FSM represents a simple state machine that records all commands.
|
|
|
|
type FSM struct {
|
2014-09-24 23:15:21 +00:00
|
|
|
MaxIndex uint64
|
|
|
|
Commands [][]byte
|
|
|
|
}
|
|
|
|
|
2015-01-07 00:21:32 +00:00
|
|
|
// MustApply updates the max index and appends the command.
|
|
|
|
func (fsm *FSM) MustApply(entry *raft.LogEntry) {
|
2014-09-24 23:15:21 +00:00
|
|
|
fsm.MaxIndex = entry.Index
|
|
|
|
if entry.Type == raft.LogEntryCommand {
|
|
|
|
fsm.Commands = append(fsm.Commands, entry.Data)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2014-09-27 22:24:59 +00:00
|
|
|
// Index returns the highest applied index.
|
|
|
|
func (fsm *FSM) Index() (uint64, error) { return fsm.MaxIndex, nil }
|
|
|
|
|
|
|
|
// Snapshot begins writing the FSM to a writer.
|
|
|
|
func (fsm *FSM) Snapshot(w io.Writer) (uint64, error) {
|
2014-09-24 23:15:21 +00:00
|
|
|
b, _ := json.Marshal(fsm)
|
|
|
|
binary.Write(w, binary.BigEndian, uint64(len(b)))
|
|
|
|
_, err := w.Write(b)
|
|
|
|
return fsm.MaxIndex, err
|
|
|
|
}
|
2014-09-27 22:24:59 +00:00
|
|
|
|
|
|
|
// Restore reads the snapshot from the reader.
|
|
|
|
func (fsm *FSM) Restore(r io.Reader) error {
|
2014-09-24 23:15:21 +00:00
|
|
|
var sz uint64
|
|
|
|
if err := binary.Read(r, binary.BigEndian, &sz); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
buf := make([]byte, sz)
|
|
|
|
if _, err := io.ReadFull(r, buf); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2015-02-05 01:59:06 +00:00
|
|
|
if err := json.Unmarshal(buf, &fsm); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
return nil
|
2014-09-24 23:15:21 +00:00
|
|
|
}
|
|
|
|
|
2014-09-27 22:24:59 +00:00
|
|
|
// MockFSM represents a state machine that can be mocked out.
|
|
|
|
type MockFSM struct {
|
|
|
|
ApplyFunc func(*raft.LogEntry) error
|
|
|
|
IndexFunc func() (uint64, error)
|
|
|
|
SnapshotFunc func(w io.Writer) (index uint64, err error)
|
|
|
|
RestoreFunc func(r io.Reader) error
|
2014-09-07 03:10:06 +00:00
|
|
|
}
|
|
|
|
|
2014-09-27 22:24:59 +00:00
|
|
|
func (fsm *MockFSM) Apply(e *raft.LogEntry) error { return fsm.ApplyFunc(e) }
|
2014-09-27 23:28:03 +00:00
|
|
|
func (fsm *MockFSM) Index() (uint64, error) { return fsm.IndexFunc() }
|
2014-09-27 22:24:59 +00:00
|
|
|
func (fsm *MockFSM) Snapshot(w io.Writer) (uint64, error) { return fsm.SnapshotFunc(w) }
|
|
|
|
func (fsm *MockFSM) Restore(r io.Reader) error { return fsm.RestoreFunc(r) }
|
2014-09-08 21:39:05 +00:00
|
|
|
|
2014-09-09 23:01:48 +00:00
|
|
|
// seq implements the raft.Log#Rand interface and returns incrementing ints.
|
|
|
|
func seq() func() int64 {
|
|
|
|
var i int64
|
|
|
|
var mu sync.Mutex
|
|
|
|
|
|
|
|
return func() int64 {
|
|
|
|
mu.Lock()
|
|
|
|
defer mu.Unlock()
|
|
|
|
|
|
|
|
i++
|
|
|
|
return i
|
|
|
|
}
|
|
|
|
}
|
2014-09-26 20:57:33 +00:00
|
|
|
|
|
|
|
// tempfile returns the path to a non-existent file in the temp directory.
|
|
|
|
func tempfile() string {
|
|
|
|
f, _ := ioutil.TempFile("", "raft-")
|
|
|
|
path := f.Name()
|
|
|
|
f.Close()
|
|
|
|
os.Remove(path)
|
|
|
|
return path
|
|
|
|
}
|
|
|
|
|
2014-09-27 22:24:59 +00:00
|
|
|
func jsonify(v interface{}) string {
|
|
|
|
b, _ := json.Marshal(v)
|
|
|
|
return string(b)
|
|
|
|
}
|
|
|
|
|
2015-02-05 01:59:06 +00:00
|
|
|
func warn(v ...interface{}) {
|
|
|
|
if testing.Verbose() {
|
|
|
|
fmt.Fprintln(os.Stderr, v...)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func warnf(msg string, v ...interface{}) {
|
|
|
|
if testing.Verbose() {
|
|
|
|
fmt.Fprintf(os.Stderr, msg+"\n", v...)
|
|
|
|
}
|
|
|
|
}
|