Refactor simulation testing framework.

pull/903/head
Ben Johnson 2014-09-24 17:15:21 -06:00
parent 425bd0aa2e
commit 3a7ef7684f
4 changed files with 251 additions and 145 deletions

View File

@ -7,6 +7,7 @@ TODO
- [ ] Candidate loop
- [ ] Save current term to disk.
- [ ] Callback
- [ ] Leave / RemovePeer
## Completed

View File

@ -161,6 +161,13 @@ func (l *Log) State() State {
return l.state
}
// Term returns the current term.
func (l *Log) Term() uint64 {
l.mu.Lock()
defer l.mu.Unlock()
return l.term
}
// Config returns a the log's current configuration.
func (l *Log) Config() *Config {
l.mu.Lock()

View File

@ -2,7 +2,9 @@ package raft_test
import (
"bytes"
"encoding/binary"
"encoding/json"
"io"
"log"
"net/url"
"os"
@ -241,6 +243,40 @@ func TestConfig_MarshalJSON(t *testing.T) {
}
}
// TestFSM represents a fake state machine that simple records all commands.
type TestFSM struct {
Log *raft.Log `json:"-"`
MaxIndex uint64
Commands [][]byte
}
func (fsm *TestFSM) Apply(entry *raft.LogEntry) error {
fsm.MaxIndex = entry.Index
if entry.Type == raft.LogEntryCommand {
fsm.Commands = append(fsm.Commands, entry.Data)
}
return nil
}
func (fsm *TestFSM) Index() (uint64, error) { return fsm.MaxIndex, nil }
func (fsm *TestFSM) Snapshot(w io.Writer) (uint64, error) {
b, _ := json.Marshal(fsm)
binary.Write(w, binary.BigEndian, uint64(len(b)))
_, err := w.Write(b)
return fsm.MaxIndex, err
}
func (fsm *TestFSM) Restore(r io.Reader) error {
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
}
return json.Unmarshal(buf, &fsm)
}
// BufferCloser represents a bytes.Buffer that provides a no-op close.
type BufferCloser struct {
*bytes.Buffer

View File

@ -2,10 +2,7 @@ package raft_test
import (
"bytes"
"encoding/binary"
"encoding/json"
"fmt"
"io"
"io/ioutil"
"log"
"math/rand"
@ -24,95 +21,43 @@ func init() {
log.SetFlags(0)
}
// Ensure that a single node can process commands applied to the log.
func Test_Simulate_SingleNode(t *testing.T) {
check(t, func(commands [][]byte) bool {
fmt.Print(".")
const (
// MinSimulationInterval is the minimum time an AddTimeEvent can move the clock.
MinSimulationInterval = 1 * time.Millisecond
var fsm TestFSM
l := raft.NewLog()
l.FSM = &fsm
l.URL, _ = url.Parse("//node")
l.Clock = raft.NewMockClock()
l.Rand = seq()
fsm.Log = l
if err := l.Open(tempfile()); err != nil {
log.Fatal("open: ", err)
}
defer os.RemoveAll(l.Path())
defer l.Close()
// Initialize log.
go func() { l.Clock.Add(150 * time.Millisecond) }()
if err := l.Initialize(); err != nil {
t.Fatalf("initialize: %s", err)
}
if err := l.Wait(1); err != nil {
t.Fatalf("wait: %s", err)
}
// Verify the configuration is set.
if b, _ := json.Marshal(l.Config()); string(b) != `{"clusterID":1,"nodes":[{"id":1,"url":"//node"}],"index":1,"maxNodeID":1}` {
t.Fatalf("unexpected config: %s", b)
}
// Execute a series of commands.
var index uint64
for _, command := range commands {
var err error
index, err = l.Apply(command)
if err != nil {
t.Fatalf("apply: %s", err)
}
}
go func() { l.Clock.Add(2 * l.HeartbeatTimeout) }()
l.Wait(index)
// Verify the commands were executed against the FSM, in order.
for i, command := range commands {
if data := fsm.Commands[i]; !bytes.Equal(command, data) {
t.Fatalf("%d. command:\n\nexp: %x\n\ngot: %x\n\n", i, command, data)
}
}
go func() { l.Clock.Add(l.HeartbeatTimeout) }()
return true
})
}
// MaxSimulationInterval is the maximum time an AddTimeEvent can move the clock.
MaxSimulationInterval = 1 * time.Second
)
// Ensure that a cluster of multiple nodes can maintain consensus.
func Test_Simulate_MultiNode(t *testing.T) {
func Test_Simulate(t *testing.T) {
var checkN int
check(t, func(s *Simulation) bool {
fmt.Print(".")
fmt.Printf("%04d ", checkN)
defer s.Close()
// Retrieve leader.
leader := s.Nodes[0]
// Initialize the cluster.
if err := s.Initialize(); err != nil {
t.Fatalf("initialize: %s", err)
}
// Validate log identifiers.
for i, n := range s.Nodes {
if id := n.Log.ID(); uint64(i+1) != id {
t.Fatalf("unexpected log id: exp=%d, got=%d", i+1, id)
// Execute events against the leader.
for _, e := range s.Events {
// Print a character to the terminal to indicate type.
fmt.Print(string(e.Indicator()))
// Apply the event to the simulation.
if err := e.Apply(s); err != nil {
t.Fatal(err)
}
time.Sleep(1 * time.Millisecond)
// Verify the simulation is in a correct state.
if err := s.Verify(); err != nil {
t.Fatal(err)
}
}
// Apply commands to the leader.
for i, command := range s.Commands {
if _, err := leader.Log.Apply(command); err != nil {
t.Fatalf("%d. apply: %s", i, err)
}
}
// Allow entries to be sent to the followers.
time.Sleep(10 * time.Millisecond)
// Wait for one heartbeat to retrieve current index.
// Wait for another heartbeat to send commit index.
s.Clock.Add(raft.DefaultHeartbeatTimeout)
@ -140,87 +85,104 @@ func Test_Simulate_MultiNode(t *testing.T) {
}
}
fmt.Printf(" n=%d, cmd=%d\n", len(s.Nodes), len(s.Commands))
checkN++
return true
})
}
// TestFSM represents a fake state machine that simple records all commands.
type TestFSM struct {
Log *raft.Log `json:"-"`
MaxIndex uint64
Commands [][]byte
}
func (fsm *TestFSM) Apply(entry *raft.LogEntry) error {
fsm.MaxIndex = entry.Index
if entry.Type == raft.LogEntryCommand {
fsm.Commands = append(fsm.Commands, entry.Data)
}
return nil
}
func (fsm *TestFSM) Index() (uint64, error) { return fsm.MaxIndex, nil }
func (fsm *TestFSM) Snapshot(w io.Writer) (uint64, error) {
b, _ := json.Marshal(fsm)
binary.Write(w, binary.BigEndian, uint64(len(b)))
_, err := w.Write(b)
return fsm.MaxIndex, err
}
func (fsm *TestFSM) Restore(r io.Reader) error {
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
}
return json.Unmarshal(buf, &fsm)
}
// Simulation represents a collection of nodes for simulating a raft cluster.
type Simulation struct {
Nodes []*SimulationNode
Events []SimulationEvent
Clock raft.Clock
Commands [][]byte
Error error // out-of-band error
}
// SetError sets an out-of-band error on the simulation.
// This can occur when an event will cause an error after time has elasped.
// Only the first error is stored.
func (s *Simulation) SetError(err error) {
if s.Error == nil {
s.Error = err
}
}
// Verify checks if the simulation is in the correct state at the given time.
func (s *Simulation) Verify() error {
// Check for out-of-band errors.
if s.Error != nil {
return s.Error
}
// TODO(simulation): Ensure one leader per term.
return nil
}
// Leader returns the node which is the leader of the latest term.
func (s *Simulation) Leader() *SimulationNode {
var node *SimulationNode
for _, n := range s.Nodes {
// Ignore joining nodes since they'll be locked.
if n.Status == Joining {
continue
}
if n.Log.State() == raft.Leader && (node == nil || node.Log.Term() < n.Log.Term()) {
node = n
}
}
return node
}
// Generate implements the testing/quick Generator interface.
func (s *Simulation) Generate(rand *rand.Rand, size int) reflect.Value {
s = &Simulation{
Clock: raft.NewMockClock(),
Clock: raft.NewMockClock(),
Events: make([]SimulationEvent, size),
Nodes: []*SimulationNode{
MustGenerateValue(reflect.TypeOf((*SimulationNode)(nil)), rand).Interface().(*SimulationNode),
},
}
// Generate commands.
s.Commands = GenerateValue(reflect.TypeOf(s.Commands), rand).([][]byte)
// Create between 1 and 9 nodes.
nodeN := 2 // rand.Intn(8) + 1
for i := 0; i < nodeN; i++ {
var n SimulationNode
n.Clock = s.Clock
n.Generate(rand, size)
s.Nodes = append(s.Nodes, &n)
// Generate events.
for i := range s.Events {
n := rand.Intn(100)
if n >= 0 && n < 70 {
s.Events[i] = (*AddTimeEvent)(nil).Generate(rand, size).Interface().(SimulationEvent)
} else if n >= 70 && n < 90 {
s.Events[i] = (*ApplyEvent)(nil).Generate(rand, size).Interface().(SimulationEvent)
} else if n >= 90 && n < 100 {
s.Events[i] = (*AddPeerEvent)(nil).Generate(rand, size).Interface().(SimulationEvent)
} else {
panic("unreachable")
}
}
return reflect.ValueOf(s)
}
// Initialize initializes the first node's log and joins other nodes to the first.
func (s *Simulation) Initialize() error {
// Initialize the log of the first node.
go func() { s.Clock.Add(raft.DefaultHeartbeatTimeout) }()
if err := s.Nodes[0].Log.Initialize(); err != nil {
return fmt.Errorf("node(0): initialize: %s", err)
n := s.Nodes[0]
// Open initial log.
n.Log.Clock = s.Clock
if err := n.Log.Open(tempfile()); err != nil {
return fmt.Errorf("open: %s", err)
}
// All other nodes should join the first node.
for i, n := range s.Nodes[1:] {
go func() { s.Clock.Add(raft.DefaultHeartbeatTimeout) }()
if err := n.Log.Join(s.Nodes[0].Log.URL); err != nil {
return fmt.Errorf("node(%d): join: %s", i, err)
}
n.Log.Wait(uint64(i))
// Start HTTP server and set log URL.
n.HTTPServer = httptest.NewServer(raft.NewHTTPHandler(n.Log))
n.Log.URL, _ = url.Parse(n.HTTPServer.URL)
// Initialize the log of the first node.
go func() { s.Clock.Add(raft.DefaultHeartbeatTimeout) }()
if err := n.Log.Initialize(); err != nil {
return fmt.Errorf("initialize: %s", err)
}
n.Status = Running
return nil
}
@ -234,8 +196,18 @@ func (s *Simulation) Close() error {
return nil
}
type NodeStatus int
const (
Stopped NodeStatus = iota
Joining
Running
Leaving
)
// SimulationNode represents a single node in the simulation.
type SimulationNode struct {
Status NodeStatus
FSM *TestFSM
Log *raft.Log
Clock raft.Clock
@ -244,6 +216,7 @@ type SimulationNode struct {
// Generate implements the testing/quick Generator interface.
func (n *SimulationNode) Generate(rand *rand.Rand, size int) reflect.Value {
n = &SimulationNode{Status: Stopped}
n.FSM = &TestFSM{}
// Create raft log.
@ -253,15 +226,6 @@ func (n *SimulationNode) Generate(rand *rand.Rand, size int) reflect.Value {
n.Log.Rand = seq()
n.FSM.Log = n.Log
// Start HTTP server and set log URL.
n.HTTPServer = httptest.NewServer(raft.NewHTTPHandler(n.Log))
n.Log.URL, _ = url.Parse(n.HTTPServer.URL)
// Open log.
if err := n.Log.Open(tempfile()); err != nil {
log.Fatalf("open: %s", err)
}
return reflect.ValueOf(n)
}
@ -274,12 +238,110 @@ func (n *SimulationNode) Close() error {
return nil
}
func GenerateValue(t reflect.Type, rand *rand.Rand) interface{} {
func MustGenerateValue(t reflect.Type, rand *rand.Rand) reflect.Value {
v, ok := quick.Value(t, rand)
if !ok {
panic("testing/quick value error")
}
return v.Interface()
return v
}
// SimulationEvent represents an action that occurs during the simulation
type SimulationEvent interface {
simulationEvent()
Indicator() rune
Apply(*Simulation) error
}
func (_ *AddTimeEvent) simulationEvent() {}
func (_ *ApplyEvent) simulationEvent() {}
func (_ *AddPeerEvent) simulationEvent() {}
// AddTimeEvent represents a simulation event where time is added to the clock.
type AddTimeEvent struct {
Duration time.Duration
}
func (e *AddTimeEvent) Indicator() rune { return '★' }
func (e *AddTimeEvent) Apply(s *Simulation) error {
s.Clock.Add(e.Duration)
return nil
}
// Generate implements the testing/quick Generator interface.
func (e *AddTimeEvent) Generate(rand *rand.Rand, size int) reflect.Value {
e = &AddTimeEvent{}
e.Duration = time.Duration(rand.Int63n(int64(MaxSimulationInterval-MinSimulationInterval))) + MinSimulationInterval
return reflect.ValueOf(e)
}
// ApplyEvent represents a simulation event where a command is applied.
type ApplyEvent struct {
Data []byte
}
func (e *ApplyEvent) Indicator() rune { return '.' }
func (e *ApplyEvent) Apply(s *Simulation) error {
// Write to leader.
leader := s.Leader()
if _, err := leader.Log.Apply(e.Data); err != nil {
return fmt.Errorf("apply: %s", err)
}
leader.Log.Flush()
// Add to simulation's list of commands.
s.Commands = append(s.Commands, e.Data)
return nil
}
// Generate implements the testing/quick Generator interface.
func (e *ApplyEvent) Generate(rand *rand.Rand, size int) reflect.Value {
e = &ApplyEvent{}
e.Data = MustGenerateValue(reflect.TypeOf(e.Data), rand).Interface().([]byte)
return reflect.ValueOf(e)
}
// AddPeerEvent represents a simulation event where a peer is created.
type AddPeerEvent struct {
Node *SimulationNode
}
func (e *AddPeerEvent) Indicator() rune { return '+' }
func (e *AddPeerEvent) Apply(s *Simulation) error {
// Open log.
e.Node.Log.Clock = s.Clock
if err := e.Node.Log.Open(tempfile()); err != nil {
return fmt.Errorf("open: %s", err)
}
// Start HTTP server and set log URL.
e.Node.HTTPServer = httptest.NewServer(raft.NewHTTPHandler(e.Node.Log))
e.Node.Log.URL, _ = url.Parse(e.Node.HTTPServer.URL)
// Join to the leader.
leaderURL := s.Leader().Log.URL
e.Node.Status = Joining
go func() {
if err := e.Node.Log.Join(leaderURL); err != nil {
s.SetError(fmt.Errorf("node: join: %s", err))
}
e.Node.Status = Running
}()
time.Sleep(10 * time.Millisecond)
// Add to list of simulation nodes.
s.Nodes = append(s.Nodes, e.Node)
return nil
}
// Generate implements the testing/quick Generator interface.
func (e *AddPeerEvent) Generate(rand *rand.Rand, size int) reflect.Value {
e = &AddPeerEvent{}
e.Node = MustGenerateValue(reflect.TypeOf(e.Node), rand).Interface().(*SimulationNode)
return reflect.ValueOf(e)
}
// tempfile returns the path to a non-existent file in the temp directory.