Skip to content

Commit

Permalink
Merge pull request #11005 from tbg/interactiontest
Browse files Browse the repository at this point in the history
raft/rafttest: introduce datadriven testing
  • Loading branch information
tbg authored Aug 12, 2019
2 parents 56ad881 + e8090e5 commit 029401a
Show file tree
Hide file tree
Showing 28 changed files with 1,648 additions and 158 deletions.
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@ module go.etcd.io/etcd

require (
github.com/bgentry/speakeasy v0.1.0
github.com/cockroachdb/datadriven v0.0.0-20190531201743-edce55837238
github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa
github.com/coreos/go-semver v0.2.0
github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7
github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf
Expand Down
4 changes: 2 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -9,8 +9,8 @@ github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+Ce
github.com/bgentry/speakeasy v0.1.0 h1:ByYyxL9InA1OWqxJqqp2A5pYHUrCiAL6K3J+LKSsQkY=
github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs=
github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw=
github.com/cockroachdb/datadriven v0.0.0-20190531201743-edce55837238 h1:uNljlOxtOHrPnRoPPx+JanqjAGZpNiqAGVBfGskd/pg=
github.com/cockroachdb/datadriven v0.0.0-20190531201743-edce55837238/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8=
github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa h1:OaNxuTZr7kxeODyLWsRMC+OD03aFUH+mW6r2d+MWa5Y=
github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8=
github.com/coreos/go-semver v0.2.0 h1:3Jm3tLmsgAYcjC+4Up7hJrFBPr+n7rAqYeSw/SZazuY=
github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk=
github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7 h1:u9SHYsPQNyt5tgDm3YN7+9dYrpK96E5wFilTFWIDZOM=
Expand Down
31 changes: 31 additions & 0 deletions raft/interaction_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
// Copyright 2019 The etcd Authors
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package raft_test

import (
"testing"

"github.com/cockroachdb/datadriven"
"go.etcd.io/etcd/raft/rafttest"
)

func TestInteraction(t *testing.T) {
datadriven.Walk(t, "testdata", func(t *testing.T, path string) {
env := rafttest.NewInteractionEnv(nil)
datadriven.RunTest(t, path, func(d *datadriven.TestData) string {
return env.Handle(t, *d)
})
})
}
13 changes: 11 additions & 2 deletions raft/raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"fmt"
"math"
"math/rand"
"sort"
"strings"
"sync"
"time"
Expand Down Expand Up @@ -529,7 +530,6 @@ func (r *raft) bcastAppend() {
if id == r.id {
return
}

r.sendAppend(id)
})
}
Expand Down Expand Up @@ -795,7 +795,16 @@ func (r *raft) campaign(t CampaignType) {
}
return
}
for id := range r.prs.Voters.IDs() {
var ids []uint64
{
idMap := r.prs.Voters.IDs()
ids = make([]uint64, 0, len(idMap))
for id := range idMap {
ids = append(ids, id)
}
sort.Slice(ids, func(i, j int) bool { return ids[i] < ids[j] })
}
for _, id := range ids {
if id == r.id {
continue
}
Expand Down
120 changes: 0 additions & 120 deletions raft/raft_snap_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ import (
"testing"

pb "go.etcd.io/etcd/raft/raftpb"
"go.etcd.io/etcd/raft/tracker"
)

var (
Expand Down Expand Up @@ -112,125 +111,6 @@ func TestSnapshotSucceed(t *testing.T) {
}
}

// TestSnapshotSucceedViaAppResp regression tests the situation in which a snap-
// shot is sent to a follower at the most recent index (i.e. the snapshot index
// is the leader's last index is the committed index). In that situation, a bug
// in the past left the follower in probing status until the next log entry was
// committed.
func TestSnapshotSucceedViaAppResp(t *testing.T) {
s1 := NewMemoryStorage()
// Create a single-node leader.
n1 := newTestRaft(1, []uint64{1}, 10, 1, s1)
n1.becomeCandidate()
n1.becomeLeader()
// We need to add a second empty entry so that we can truncate the first
// one away.
n1.Step(pb.Message{Type: pb.MsgProp, Entries: []pb.Entry{{}}})

rd := newReady(n1, &SoftState{}, pb.HardState{})
s1.Append(rd.Entries)
s1.SetHardState(rd.HardState)

if exp, ci := s1.lastIndex(), n1.raftLog.committed; ci != exp {
t.Fatalf("unexpected committed index %d, wanted %d: %+v", ci, exp, s1)
}

// Force a log truncation.
if err := s1.Compact(1); err != nil {
t.Fatal(err)
}

// Add a follower to the group. Do this in a clandestine way for simplicity.
// Also set up a snapshot that will be sent to the follower.
n1.applyConfChange(pb.ConfChange{NodeID: 2, Type: pb.ConfChangeAddNode}.AsV2())
s1.snapshot = pb.Snapshot{
Metadata: pb.SnapshotMetadata{
ConfState: pb.ConfState{Voters: []uint64{1, 2}},
Index: s1.lastIndex(),
Term: s1.ents[len(s1.ents)-1].Term,
},
}

noMessage := pb.MessageType(-1)
mustSend := func(from, to *raft, typ pb.MessageType) pb.Message {
t.Helper()
for i, msg := range from.msgs {
if msg.From != from.id || msg.To != to.id || msg.Type != typ {
continue
}
t.Log(DescribeMessage(msg, func([]byte) string { return "" }))
if len(msg.Entries) > 0 {
t.Log(DescribeEntries(msg.Entries, func(b []byte) string { return string(b) }))
}
if err := to.Step(msg); err != nil {
t.Fatalf("%v: %s", msg, err)
}
from.msgs = append(from.msgs[:i], from.msgs[i+1:]...)
return msg
}
if typ == noMessage {
if len(from.msgs) == 0 {
return pb.Message{}
}
t.Fatalf("expected no more messages, but got %d->%d %v", from.id, to.id, from.msgs)
}
t.Fatalf("message %d->%d %s not found in %v", from.id, to.id, typ, from.msgs)
return pb.Message{} // unreachable
}

// Create the follower that will receive the snapshot.
s2 := NewMemoryStorage()
n2 := newTestRaft(2, []uint64{1, 2}, 10, 1, s2)

// Let the leader probe the follower.
if !n1.maybeSendAppend(2, true /* sendIfEmpty */) {
t.Fatalf("expected message to be sent")
}
if msg := mustSend(n1, n2, pb.MsgApp); len(msg.Entries) > 0 {
// For this test to work, the leader must not have anything to append
// to the follower right now.
t.Fatalf("unexpectedly appending entries %v", msg.Entries)
}

// Follower rejects the append (because it doesn't have any log entries)
if msg := mustSend(n2, n1, pb.MsgAppResp); !msg.Reject {
t.Fatalf("expected a rejection with zero hint, got reject=%t hint=%d", msg.Reject, msg.RejectHint)
}

const expIdx = 2
// Leader sends snapshot due to RejectHint of zero (we set up the raft log
// to start at index 2).
if msg := mustSend(n1, n2, pb.MsgSnap); msg.Snapshot.Metadata.Index != expIdx {
t.Fatalf("expected snapshot at index %d, got %d", expIdx, msg.Snapshot.Metadata.Index)
}

// n2 reacts to snapshot with MsgAppResp.
if msg := mustSend(n2, n1, pb.MsgAppResp); msg.Index != expIdx {
t.Fatalf("expected AppResp at index %d, got %d", expIdx, msg.Index)
}

// Leader sends MsgApp to communicate commit index.
if msg := mustSend(n1, n2, pb.MsgApp); msg.Commit != expIdx {
t.Fatalf("expected commit index %d, got %d", expIdx, msg.Commit)
}

// Follower responds.
mustSend(n2, n1, pb.MsgAppResp)

// Leader has correct state for follower.
pr := n1.prs.Progress[2]
if pr.State != tracker.StateReplicate {
t.Fatalf("unexpected state %v", pr)
}
if pr.Match != expIdx || pr.Next != expIdx+1 {
t.Fatalf("expected match = %d, next = %d; got match = %d and next = %d", expIdx, expIdx+1, pr.Match, pr.Next)
}

// Leader and follower are done.
mustSend(n1, n2, noMessage)
mustSend(n2, n1, noMessage)
}

func TestSnapshotAbort(t *testing.T) {
storage := NewMemoryStorage()
sm := newTestRaft(1, []uint64{1, 2}, 10, 1, storage)
Expand Down
65 changes: 65 additions & 0 deletions raft/raftpb/confchange.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,8 @@ package raftpb

import (
"fmt"
"strconv"
"strings"

"github.com/gogo/protobuf/proto"
)
Expand Down Expand Up @@ -103,3 +105,66 @@ func (c *ConfChangeV2) LeaveJoint() bool {
cpy.Context = nil
return proto.Equal(&cpy, &ConfChangeV2{})
}

// ConfChangesFromString parses a Space-delimited sequence of operations into a
// slice of ConfChangeSingle. The supported operations are:
// - vn: make n a voter,
// - ln: make n a learner,
// - rn: remove n, and
// - un: update n.
func ConfChangesFromString(s string) ([]ConfChangeSingle, error) {
var ccs []ConfChangeSingle
toks := strings.Split(strings.TrimSpace(s), " ")
if toks[0] == "" {
toks = nil
}
for _, tok := range toks {
if len(tok) < 2 {
return nil, fmt.Errorf("unknown token %s", tok)
}
var cc ConfChangeSingle
switch tok[0] {
case 'v':
cc.Type = ConfChangeAddNode
case 'l':
cc.Type = ConfChangeAddLearnerNode
case 'r':
cc.Type = ConfChangeRemoveNode
case 'u':
cc.Type = ConfChangeUpdateNode
default:
return nil, fmt.Errorf("unknown input: %s", tok)
}
id, err := strconv.ParseUint(tok[1:], 10, 64)
if err != nil {
return nil, err
}
cc.NodeID = id
ccs = append(ccs, cc)
}
return ccs, nil
}

// ConfChangesToString is the inverse to ConfChangesFromString.
func ConfChangesToString(ccs []ConfChangeSingle) string {
var buf strings.Builder
for i, cc := range ccs {
if i > 0 {
buf.WriteByte(' ')
}
switch cc.Type {
case ConfChangeAddNode:
buf.WriteByte('v')
case ConfChangeAddLearnerNode:
buf.WriteByte('l')
case ConfChangeRemoveNode:
buf.WriteByte('r')
case ConfChangeUpdateNode:
buf.WriteByte('u')
default:
buf.WriteString("unknown")
}
fmt.Fprintf(&buf, "%d", cc.NodeID)
}
return buf.String()
}
90 changes: 90 additions & 0 deletions raft/rafttest/interaction_env.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,90 @@
// Copyright 2019 The etcd Authors
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package rafttest

import (
"fmt"
"math"
"strings"

"go.etcd.io/etcd/raft"
pb "go.etcd.io/etcd/raft/raftpb"
)

// InteractionOpts groups the options for an InteractionEnv.
type InteractionOpts struct {
OnConfig func(*raft.Config)
}

// A Node is a member of a raft group tested via an InteractionEnv.
type Node struct {
*raft.RawNode
Storage

Config *raft.Config
History []pb.Snapshot
}

// InteractionEnv facilitates testing of complex interactions between the
// members of a raft group.
type InteractionEnv struct {
Options *InteractionOpts
Nodes []Node
Messages []pb.Message // in-flight messages

Output *RedirectLogger
}

// NewInteractionEnv initializes an InteractionEnv. opts may be nil.
func NewInteractionEnv(opts *InteractionOpts) *InteractionEnv {
if opts == nil {
opts = &InteractionOpts{}
}
return &InteractionEnv{
Options: opts,
Output: &RedirectLogger{
Builder: &strings.Builder{},
},
}
}

// Storage is the interface used by InteractionEnv. It is comprised of raft's
// Storage interface plus access to operations that maintain the log and drive
// the Ready handling loop.
type Storage interface {
raft.Storage
SetHardState(state pb.HardState) error
ApplySnapshot(pb.Snapshot) error
Compact(newFirstIndex uint64) error
Append([]pb.Entry) error
}

// defaultRaftConfig sets up a *raft.Config with reasonable testing defaults.
// In particular, no limits are set.
func defaultRaftConfig(id uint64, applied uint64, s raft.Storage) *raft.Config {
return &raft.Config{
ID: id,
Applied: applied,
ElectionTick: 3,
HeartbeatTick: 1,
Storage: s,
MaxSizePerMsg: math.MaxUint64,
MaxInflightMsgs: math.MaxInt32,
}
}

func defaultEntryFormatter(b []byte) string {
return fmt.Sprintf("%q", b)
}
Loading

0 comments on commit 029401a

Please sign in to comment.