raft: add a batch of interaction-driven conf change tests
Verifiy the behavior in various v1 and v2 conf change operations. This also includes various fixups, notably it adds protection against transitioning in and out of new configs when this is not permissible. There are more threads to pull, but those are left for future commits.
This commit is contained in:

committed by
Gyuho Lee

parent
d177b7f6b4
commit
078caccce5
@ -22,6 +22,9 @@ import (
|
||||
)
|
||||
|
||||
func TestInteraction(t *testing.T) {
|
||||
// NB: if this test fails, run `go test ./raft -rewrite` and inspect the
|
||||
// diff. Only commit the changes if you understand what caused them and if
|
||||
// they are desired.
|
||||
datadriven.Walk(t, "testdata", func(t *testing.T, path string) {
|
||||
env := rafttest.NewInteractionEnv(nil)
|
||||
datadriven.RunTest(t, path, func(d *datadriven.TestData) string {
|
||||
|
34
raft/raft.go
34
raft/raft.go
@ -1036,10 +1036,36 @@ func stepLeader(r *raft, m pb.Message) error {
|
||||
|
||||
for i := range m.Entries {
|
||||
e := &m.Entries[i]
|
||||
if e.Type == pb.EntryConfChange || e.Type == pb.EntryConfChangeV2 {
|
||||
if r.pendingConfIndex > r.raftLog.applied {
|
||||
r.logger.Infof("%x propose conf %s ignored since pending unapplied configuration [index %d, applied %d]",
|
||||
r.id, e, r.pendingConfIndex, r.raftLog.applied)
|
||||
var cc pb.ConfChangeI
|
||||
if e.Type == pb.EntryConfChange {
|
||||
var ccc pb.ConfChange
|
||||
if err := ccc.Unmarshal(e.Data); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
cc = ccc
|
||||
} else if e.Type == pb.EntryConfChangeV2 {
|
||||
var ccc pb.ConfChangeV2
|
||||
if err := ccc.Unmarshal(e.Data); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
cc = ccc
|
||||
}
|
||||
if cc != nil {
|
||||
alreadyPending := r.pendingConfIndex > r.raftLog.applied
|
||||
alreadyJoint := len(r.prs.Config.Voters[1]) > 0
|
||||
wantsLeaveJoint := len(cc.AsV2().Changes) == 0
|
||||
|
||||
var refused string
|
||||
if alreadyPending {
|
||||
refused = fmt.Sprintf("possible unapplied conf change at index %d (applied to %d)", r.pendingConfIndex, r.raftLog.applied)
|
||||
} else if alreadyJoint && !wantsLeaveJoint {
|
||||
refused = "must transition out of joint config first"
|
||||
} else if !alreadyJoint && wantsLeaveJoint {
|
||||
refused = "not in joint state; refusing empty conf change"
|
||||
}
|
||||
|
||||
if refused != "" {
|
||||
r.logger.Infof("%x ignoring conf change %v at config %s: %s", r.id, cc, r.prs.Config, refused)
|
||||
m.Entries[i] = pb.Entry{Type: pb.EntryNormal}
|
||||
} else {
|
||||
r.pendingConfIndex = r.raftLog.lastIndex() + uint64(i) + 1
|
||||
|
@ -15,8 +15,8 @@
|
||||
package rafttest
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"strconv"
|
||||
"testing"
|
||||
|
||||
"github.com/cockroachdb/datadriven"
|
||||
@ -25,34 +25,70 @@ import (
|
||||
)
|
||||
|
||||
func (env *InteractionEnv) handleDeliverMsgs(t *testing.T, d datadriven.TestData) error {
|
||||
if len(env.Messages) == 0 {
|
||||
return errors.New("no messages to deliver")
|
||||
var rs []Recipient
|
||||
for _, arg := range d.CmdArgs {
|
||||
if len(arg.Vals) == 0 {
|
||||
id, err := strconv.ParseUint(arg.Key, 10, 64)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
rs = append(rs, Recipient{ID: id})
|
||||
}
|
||||
for i := range arg.Vals {
|
||||
switch arg.Key {
|
||||
case "drop":
|
||||
var id uint64
|
||||
arg.Scan(t, i, &id)
|
||||
var found bool
|
||||
for _, r := range rs {
|
||||
if r.ID == id {
|
||||
found = true
|
||||
}
|
||||
}
|
||||
if found {
|
||||
t.Fatalf("can't both deliver and drop msgs to %d", id)
|
||||
}
|
||||
rs = append(rs, Recipient{ID: id, Drop: true})
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
msgs := env.Messages
|
||||
env.Messages = nil
|
||||
|
||||
return env.DeliverMsgs(msgs)
|
||||
}
|
||||
|
||||
// DeliverMsgs delivers the supplied messages typically taken from env.Messages.
|
||||
func (env *InteractionEnv) DeliverMsgs(msgs []raftpb.Message) error {
|
||||
for _, msg := range msgs {
|
||||
toIdx := int(msg.To - 1)
|
||||
var drop bool
|
||||
if toIdx >= len(env.Nodes) {
|
||||
// Drop messages for peers that don't exist yet.
|
||||
drop = true
|
||||
env.Output.WriteString("dropped: ")
|
||||
}
|
||||
fmt.Fprintln(env.Output, raft.DescribeMessage(msg, defaultEntryFormatter))
|
||||
if drop {
|
||||
continue
|
||||
}
|
||||
if err := env.Nodes[toIdx].Step(msg); err != nil {
|
||||
env.Output.WriteString(err.Error())
|
||||
continue
|
||||
}
|
||||
if n := env.DeliverMsgs(rs...); n == 0 {
|
||||
env.Output.WriteString("no messages\n")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type Recipient struct {
|
||||
ID uint64
|
||||
Drop bool
|
||||
}
|
||||
|
||||
// DeliverMsgs goes through env.Messages and, depending on the Drop flag,
|
||||
// delivers or drops messages to the specified Recipients. Returns the
|
||||
// number of messages handled (i.e. delivered or dropped). A handled message
|
||||
// is removed from env.Messages.
|
||||
func (env *InteractionEnv) DeliverMsgs(rs ...Recipient) int {
|
||||
var n int
|
||||
for _, r := range rs {
|
||||
var msgs []raftpb.Message
|
||||
msgs, env.Messages = splitMsgs(env.Messages, r.ID)
|
||||
n += len(msgs)
|
||||
for _, msg := range msgs {
|
||||
if r.Drop {
|
||||
fmt.Fprint(env.Output, "dropped: ")
|
||||
}
|
||||
fmt.Fprintln(env.Output, raft.DescribeMessage(msg, defaultEntryFormatter))
|
||||
if r.Drop {
|
||||
// NB: it's allowed to drop messages to nodes that haven't been instantiated yet,
|
||||
// we haven't used msg.To yet.
|
||||
continue
|
||||
}
|
||||
toIdx := int(msg.To - 1)
|
||||
if err := env.Nodes[toIdx].Step(msg); err != nil {
|
||||
env.Output.WriteString(err.Error())
|
||||
}
|
||||
}
|
||||
}
|
||||
return n
|
||||
}
|
||||
|
@ -19,7 +19,6 @@ import (
|
||||
|
||||
"github.com/cockroachdb/datadriven"
|
||||
"go.etcd.io/etcd/raft"
|
||||
"go.etcd.io/etcd/raft/quorum"
|
||||
"go.etcd.io/etcd/raft/raftpb"
|
||||
)
|
||||
|
||||
@ -50,6 +49,7 @@ func (env *InteractionEnv) ProcessReady(idx int) error {
|
||||
}
|
||||
for _, ent := range rd.CommittedEntries {
|
||||
var update []byte
|
||||
var cs *raftpb.ConfState
|
||||
switch ent.Type {
|
||||
case raftpb.EntryConfChange:
|
||||
var cc raftpb.ConfChange
|
||||
@ -57,13 +57,13 @@ func (env *InteractionEnv) ProcessReady(idx int) error {
|
||||
return err
|
||||
}
|
||||
update = cc.Context
|
||||
rn.ApplyConfChange(cc)
|
||||
cs = rn.ApplyConfChange(cc)
|
||||
case raftpb.EntryConfChangeV2:
|
||||
var cc raftpb.ConfChangeV2
|
||||
if err := cc.Unmarshal(ent.Data); err != nil {
|
||||
return err
|
||||
}
|
||||
rn.ApplyConfChange(cc)
|
||||
cs = rn.ApplyConfChange(cc)
|
||||
update = cc.Context
|
||||
default:
|
||||
update = ent.Data
|
||||
@ -78,13 +78,11 @@ func (env *InteractionEnv) ProcessReady(idx int) error {
|
||||
snap.Data = append(snap.Data, update...)
|
||||
snap.Metadata.Index = ent.Index
|
||||
snap.Metadata.Term = ent.Term
|
||||
cfg := rn.Status().Config
|
||||
snap.Metadata.ConfState = raftpb.ConfState{
|
||||
Voters: cfg.Voters[0].Slice(),
|
||||
VotersOutgoing: cfg.Voters[1].Slice(),
|
||||
Learners: quorum.MajorityConfig(cfg.Learners).Slice(),
|
||||
LearnersNext: quorum.MajorityConfig(cfg.LearnersNext).Slice(),
|
||||
if cs == nil {
|
||||
sl := env.Nodes[idx].History
|
||||
cs = &sl[len(sl)-1].Metadata.ConfState
|
||||
}
|
||||
snap.Metadata.ConfState = *cs
|
||||
env.Nodes[idx].History = append(env.Nodes[idx].History, snap)
|
||||
}
|
||||
for _, msg := range rd.Messages {
|
||||
|
@ -65,12 +65,13 @@ func (env *InteractionEnv) Stabilize(idxs ...int) error {
|
||||
withIndent(func() { env.ProcessReady(idx) })
|
||||
}
|
||||
}
|
||||
var msgs []raftpb.Message
|
||||
for _, rn := range nodes {
|
||||
msgs, env.Messages = splitMsgs(env.Messages, rn.Status().ID)
|
||||
if len(msgs) > 0 {
|
||||
fmt.Fprintf(env.Output, "> delivering messages\n")
|
||||
withIndent(func() { env.DeliverMsgs(msgs) })
|
||||
id := rn.Status().ID
|
||||
// NB: we grab the messages just to see whether to print the header.
|
||||
// DeliverMsgs will do it again.
|
||||
if msgs, _ := splitMsgs(env.Messages, id); len(msgs) > 0 {
|
||||
fmt.Fprintf(env.Output, "> %d receiving messages\n", id)
|
||||
withIndent(func() { env.DeliverMsgs(Recipient{ID: id}) })
|
||||
done = false
|
||||
}
|
||||
}
|
||||
@ -81,6 +82,7 @@ func (env *InteractionEnv) Stabilize(idxs ...int) error {
|
||||
}
|
||||
|
||||
func splitMsgs(msgs []raftpb.Message, to uint64) (toMsgs []raftpb.Message, rmdr []raftpb.Message) {
|
||||
// NB: this method does not reorder messages.
|
||||
for _, msg := range msgs {
|
||||
if msg.To == to {
|
||||
toMsgs = append(toMsgs, msg)
|
||||
|
18
raft/testdata/campaign.txt
vendored
18
raft/testdata/campaign.txt
vendored
@ -31,12 +31,12 @@ stabilize
|
||||
Messages:
|
||||
1->2 MsgVote Term:1 Log:1/2
|
||||
1->3 MsgVote Term:1 Log:1/2
|
||||
> delivering messages
|
||||
> 2 receiving messages
|
||||
1->2 MsgVote Term:1 Log:1/2
|
||||
INFO 2 [term: 0] received a MsgVote message with higher term from 1 [term: 1]
|
||||
INFO 2 became follower at term 1
|
||||
INFO 2 [logterm: 1, index: 2, vote: 0] cast MsgVote for 1 [logterm: 1, index: 2] at term 1
|
||||
> delivering messages
|
||||
> 3 receiving messages
|
||||
1->3 MsgVote Term:1 Log:1/2
|
||||
INFO 3 [term: 0] received a MsgVote message with higher term from 1 [term: 1]
|
||||
INFO 3 became follower at term 1
|
||||
@ -51,7 +51,7 @@ stabilize
|
||||
HardState Term:1 Vote:1 Commit:2
|
||||
Messages:
|
||||
3->1 MsgVoteResp Term:1 Log:0/0
|
||||
> delivering messages
|
||||
> 1 receiving messages
|
||||
2->1 MsgVoteResp Term:1 Log:0/0
|
||||
INFO 1 received MsgVoteResp from 2 at term 1
|
||||
INFO 1 has received 2 MsgVoteResp votes and 0 vote rejections
|
||||
@ -65,9 +65,9 @@ stabilize
|
||||
Messages:
|
||||
1->2 MsgApp Term:1 Log:1/2 Commit:2 Entries:[1/3 EntryNormal ""]
|
||||
1->3 MsgApp Term:1 Log:1/2 Commit:2 Entries:[1/3 EntryNormal ""]
|
||||
> delivering messages
|
||||
> 2 receiving messages
|
||||
1->2 MsgApp Term:1 Log:1/2 Commit:2 Entries:[1/3 EntryNormal ""]
|
||||
> delivering messages
|
||||
> 3 receiving messages
|
||||
1->3 MsgApp Term:1 Log:1/2 Commit:2 Entries:[1/3 EntryNormal ""]
|
||||
> 2 handling Ready
|
||||
Ready MustSync=true:
|
||||
@ -83,7 +83,7 @@ stabilize
|
||||
1/3 EntryNormal ""
|
||||
Messages:
|
||||
3->1 MsgAppResp Term:1 Log:0/3
|
||||
> delivering messages
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/3
|
||||
3->1 MsgAppResp Term:1 Log:0/3
|
||||
> 1 handling Ready
|
||||
@ -94,9 +94,9 @@ stabilize
|
||||
Messages:
|
||||
1->2 MsgApp Term:1 Log:1/3 Commit:3
|
||||
1->3 MsgApp Term:1 Log:1/3 Commit:3
|
||||
> delivering messages
|
||||
> 2 receiving messages
|
||||
1->2 MsgApp Term:1 Log:1/3 Commit:3
|
||||
> delivering messages
|
||||
> 3 receiving messages
|
||||
1->3 MsgApp Term:1 Log:1/3 Commit:3
|
||||
> 2 handling Ready
|
||||
Ready MustSync=false:
|
||||
@ -112,6 +112,6 @@ stabilize
|
||||
1/3 EntryNormal ""
|
||||
Messages:
|
||||
3->1 MsgAppResp Term:1 Log:0/3
|
||||
> delivering messages
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/3
|
||||
3->1 MsgAppResp Term:1 Log:0/3
|
||||
|
152
raft/testdata/campaign_learner_must_vote.txt
vendored
Normal file
152
raft/testdata/campaign_learner_must_vote.txt
vendored
Normal file
@ -0,0 +1,152 @@
|
||||
# Regression test that verifies that learners can vote. This holds only in the
|
||||
# sense that if a learner is asked to vote, a candidate believes that they are a
|
||||
# voter based on its current config, which may be more recent than that of the
|
||||
# learner. If learners which are actually voters but don't know it yet don't
|
||||
# vote in that situation, the raft group may end up unavailable despite a quorum
|
||||
# of voters (as of the latest config) being available.
|
||||
#
|
||||
# See:
|
||||
# https://github.com/etcd-io/etcd/pull/10998
|
||||
|
||||
# Turn output off during boilerplate.
|
||||
log-level none
|
||||
----
|
||||
ok
|
||||
|
||||
# Bootstrap three nodes.
|
||||
add-nodes 3 voters=(1,2) learners=(3) index=2
|
||||
----
|
||||
ok
|
||||
|
||||
# n1 gets to be leader.
|
||||
campaign 1
|
||||
----
|
||||
ok
|
||||
|
||||
stabilize
|
||||
----
|
||||
ok (quiet)
|
||||
|
||||
# Propose a conf change on n1 that promotes n3 to voter.
|
||||
propose-conf-change 1
|
||||
v3
|
||||
----
|
||||
ok
|
||||
|
||||
# Commit and fully apply said conf change. n1 and n2 now consider n3 a voter.
|
||||
stabilize 1 2
|
||||
----
|
||||
ok (quiet)
|
||||
|
||||
# Drop all inflight messages to 3. We don't want it to be caught up when it is
|
||||
# asked to vote.
|
||||
deliver-msgs drop=(3)
|
||||
----
|
||||
ok (quiet)
|
||||
|
||||
# We now pretend that n1 is dead, and n2 is trying to become leader.
|
||||
|
||||
log-level debug
|
||||
----
|
||||
ok
|
||||
|
||||
campaign 2
|
||||
----
|
||||
INFO 2 is starting a new election at term 1
|
||||
INFO 2 became candidate at term 2
|
||||
INFO 2 received MsgVoteResp from 2 at term 2
|
||||
INFO 2 [logterm: 1, index: 4] sent MsgVote request to 1 at term 2
|
||||
INFO 2 [logterm: 1, index: 4] sent MsgVote request to 3 at term 2
|
||||
|
||||
# Send out the MsgVote requests.
|
||||
process-ready 2
|
||||
----
|
||||
Ready MustSync=true:
|
||||
Lead:0 State:StateCandidate
|
||||
HardState Term:2 Vote:2 Commit:4
|
||||
Messages:
|
||||
2->1 MsgVote Term:2 Log:1/4
|
||||
2->3 MsgVote Term:2 Log:1/4
|
||||
|
||||
# n2 is now campaigning while n1 is down (does not respond). The latest config
|
||||
# has n3 as a voter, but n3 doesn't even have the corresponding conf change in
|
||||
# its log. Still, it casts a vote for n2 which can in turn become leader and
|
||||
# catches up n3.
|
||||
stabilize 3
|
||||
----
|
||||
> 3 receiving messages
|
||||
2->3 MsgVote Term:2 Log:1/4
|
||||
INFO 3 [term: 1] received a MsgVote message with higher term from 2 [term: 2]
|
||||
INFO 3 became follower at term 2
|
||||
INFO 3 [logterm: 1, index: 3, vote: 0] cast MsgVote for 2 [logterm: 1, index: 4] at term 2
|
||||
> 3 handling Ready
|
||||
Ready MustSync=true:
|
||||
Lead:0 State:StateFollower
|
||||
HardState Term:2 Vote:2 Commit:3
|
||||
Messages:
|
||||
3->2 MsgVoteResp Term:2 Log:0/0
|
||||
|
||||
stabilize 2 3
|
||||
----
|
||||
> 2 receiving messages
|
||||
3->2 MsgVoteResp Term:2 Log:0/0
|
||||
INFO 2 received MsgVoteResp from 3 at term 2
|
||||
INFO 2 has received 2 MsgVoteResp votes and 0 vote rejections
|
||||
INFO 2 became leader at term 2
|
||||
> 2 handling Ready
|
||||
Ready MustSync=true:
|
||||
Lead:2 State:StateLeader
|
||||
Entries:
|
||||
2/5 EntryNormal ""
|
||||
Messages:
|
||||
2->1 MsgApp Term:2 Log:1/4 Commit:4 Entries:[2/5 EntryNormal ""]
|
||||
2->3 MsgApp Term:2 Log:1/4 Commit:4 Entries:[2/5 EntryNormal ""]
|
||||
> 3 receiving messages
|
||||
2->3 MsgApp Term:2 Log:1/4 Commit:4 Entries:[2/5 EntryNormal ""]
|
||||
DEBUG 3 [logterm: 0, index: 4] rejected MsgApp [logterm: 1, index: 4] from 2
|
||||
> 3 handling Ready
|
||||
Ready MustSync=false:
|
||||
Lead:2 State:StateFollower
|
||||
Messages:
|
||||
3->2 MsgAppResp Term:2 Log:0/4 Rejected (Hint: 3)
|
||||
> 2 receiving messages
|
||||
3->2 MsgAppResp Term:2 Log:0/4 Rejected (Hint: 3)
|
||||
DEBUG 2 received MsgAppResp(MsgApp was rejected, lastindex: 3) from 3 for index 4
|
||||
DEBUG 2 decreased progress of 3 to [StateProbe match=0 next=4]
|
||||
> 2 handling Ready
|
||||
Ready MustSync=false:
|
||||
Messages:
|
||||
2->3 MsgApp Term:2 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v3, 2/5 EntryNormal ""]
|
||||
> 3 receiving messages
|
||||
2->3 MsgApp Term:2 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v3, 2/5 EntryNormal ""]
|
||||
> 3 handling Ready
|
||||
INFO 3 switched to configuration voters=(1 2 3)
|
||||
Ready MustSync=true:
|
||||
HardState Term:2 Vote:2 Commit:4
|
||||
Entries:
|
||||
1/4 EntryConfChangeV2 v3
|
||||
2/5 EntryNormal ""
|
||||
CommittedEntries:
|
||||
1/4 EntryConfChangeV2 v3
|
||||
Messages:
|
||||
3->2 MsgAppResp Term:2 Log:0/5
|
||||
> 2 receiving messages
|
||||
3->2 MsgAppResp Term:2 Log:0/5
|
||||
> 2 handling Ready
|
||||
Ready MustSync=false:
|
||||
HardState Term:2 Vote:2 Commit:5
|
||||
CommittedEntries:
|
||||
2/5 EntryNormal ""
|
||||
Messages:
|
||||
2->3 MsgApp Term:2 Log:2/5 Commit:5
|
||||
> 3 receiving messages
|
||||
2->3 MsgApp Term:2 Log:2/5 Commit:5
|
||||
> 3 handling Ready
|
||||
Ready MustSync=false:
|
||||
HardState Term:2 Vote:2 Commit:5
|
||||
CommittedEntries:
|
||||
2/5 EntryNormal ""
|
||||
Messages:
|
||||
3->2 MsgAppResp Term:2 Log:0/5
|
||||
> 2 receiving messages
|
||||
3->2 MsgAppResp Term:2 Log:0/5
|
97
raft/testdata/confchange_v1_add_single.txt
vendored
Normal file
97
raft/testdata/confchange_v1_add_single.txt
vendored
Normal file
@ -0,0 +1,97 @@
|
||||
# Run a V1 membership change that adds a single voter.
|
||||
|
||||
# Bootstrap n1.
|
||||
add-nodes 1 voters=(1) index=2
|
||||
----
|
||||
INFO 1 switched to configuration voters=(1)
|
||||
INFO 1 became follower at term 0
|
||||
INFO newRaft 1 [peers: [1], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1]
|
||||
|
||||
campaign 1
|
||||
----
|
||||
INFO 1 is starting a new election at term 0
|
||||
INFO 1 became candidate at term 1
|
||||
INFO 1 received MsgVoteResp from 1 at term 1
|
||||
INFO 1 became leader at term 1
|
||||
|
||||
# Add v2 (with an auto transition).
|
||||
propose-conf-change 1 v1=true
|
||||
v2
|
||||
----
|
||||
ok
|
||||
|
||||
# Pull n2 out of thin air.
|
||||
add-nodes 1
|
||||
----
|
||||
INFO 2 switched to configuration voters=()
|
||||
INFO 2 became follower at term 0
|
||||
INFO newRaft 2 [peers: [], term: 0, commit: 0, applied: 0, lastindex: 0, lastterm: 0]
|
||||
|
||||
# n1 commits the conf change using itself as commit quorum, immediately transitions into
|
||||
# the final config, and catches up n2. Note that it's using an EntryConfChange, not an
|
||||
# EntryConfChangeV2, so this is compatible with nodes that don't know about V2 conf changes.
|
||||
stabilize
|
||||
----
|
||||
> 1 handling Ready
|
||||
INFO 1 switched to configuration voters=(1 2)
|
||||
Ready MustSync=true:
|
||||
Lead:1 State:StateLeader
|
||||
HardState Term:1 Vote:1 Commit:4
|
||||
Entries:
|
||||
1/3 EntryNormal ""
|
||||
1/4 EntryConfChange v2
|
||||
CommittedEntries:
|
||||
1/3 EntryNormal ""
|
||||
1/4 EntryConfChange v2
|
||||
> 1 handling Ready
|
||||
Ready MustSync=false:
|
||||
Messages:
|
||||
1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChange v2]
|
||||
> 2 receiving messages
|
||||
1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChange v2]
|
||||
INFO 2 [term: 0] received a MsgApp message with higher term from 1 [term: 1]
|
||||
INFO 2 became follower at term 1
|
||||
DEBUG 2 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1
|
||||
> 2 handling Ready
|
||||
Ready MustSync=true:
|
||||
Lead:1 State:StateFollower
|
||||
HardState Term:1 Commit:0
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0)
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0)
|
||||
DEBUG 1 received MsgAppResp(MsgApp was rejected, lastindex: 0) from 2 for index 3
|
||||
DEBUG 1 decreased progress of 2 to [StateProbe match=0 next=1]
|
||||
DEBUG 1 [firstindex: 3, commit: 4] sent snapshot[index: 4, term: 1] to 2 [StateProbe match=0 next=1]
|
||||
DEBUG 1 paused sending replication messages to 2 [StateSnapshot match=0 next=1 paused pendingSnap=4]
|
||||
> 1 handling Ready
|
||||
Ready MustSync=false:
|
||||
Messages:
|
||||
1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false
|
||||
> 2 receiving messages
|
||||
1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false
|
||||
INFO log [committed=0, applied=0, unstable.offset=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1]
|
||||
INFO 2 switched to configuration voters=(1 2)
|
||||
INFO 2 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1]
|
||||
INFO 2 [commit: 4] restored snapshot [index: 4, term: 1]
|
||||
> 2 handling Ready
|
||||
Ready MustSync=false:
|
||||
HardState Term:1 Commit:4
|
||||
Snapshot Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/4
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/4
|
||||
DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 2 [StateSnapshot match=4 next=5 paused pendingSnap=4]
|
||||
> 1 handling Ready
|
||||
Ready MustSync=false:
|
||||
Messages:
|
||||
1->2 MsgApp Term:1 Log:1/4 Commit:4
|
||||
> 2 receiving messages
|
||||
1->2 MsgApp Term:1 Log:1/4 Commit:4
|
||||
> 2 handling Ready
|
||||
Ready MustSync=false:
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/4
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/4
|
102
raft/testdata/confchange_v1_remove_leader.txt
vendored
Normal file
102
raft/testdata/confchange_v1_remove_leader.txt
vendored
Normal file
@ -0,0 +1,102 @@
|
||||
# We'll turn this back on after the boilerplate.
|
||||
log-level none
|
||||
----
|
||||
ok
|
||||
|
||||
# Run a V1 membership change that removes the leader.
|
||||
# Bootstrap n1, n2, n3.
|
||||
add-nodes 3 voters=(1,2,3) index=2
|
||||
----
|
||||
ok
|
||||
|
||||
campaign 1
|
||||
----
|
||||
ok
|
||||
|
||||
stabilize
|
||||
----
|
||||
ok (quiet)
|
||||
|
||||
log-level debug
|
||||
----
|
||||
ok
|
||||
|
||||
# Remove n1.
|
||||
propose-conf-change 1 v1=true
|
||||
r1
|
||||
----
|
||||
ok
|
||||
|
||||
stabilize
|
||||
----
|
||||
> 1 handling Ready
|
||||
Ready MustSync=true:
|
||||
Entries:
|
||||
1/4 EntryConfChange r1
|
||||
Messages:
|
||||
1->2 MsgApp Term:1 Log:1/3 Commit:3 Entries:[1/4 EntryConfChange r1]
|
||||
1->3 MsgApp Term:1 Log:1/3 Commit:3 Entries:[1/4 EntryConfChange r1]
|
||||
> 2 receiving messages
|
||||
1->2 MsgApp Term:1 Log:1/3 Commit:3 Entries:[1/4 EntryConfChange r1]
|
||||
> 3 receiving messages
|
||||
1->3 MsgApp Term:1 Log:1/3 Commit:3 Entries:[1/4 EntryConfChange r1]
|
||||
> 2 handling Ready
|
||||
Ready MustSync=true:
|
||||
Entries:
|
||||
1/4 EntryConfChange r1
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/4
|
||||
> 3 handling Ready
|
||||
Ready MustSync=true:
|
||||
Entries:
|
||||
1/4 EntryConfChange r1
|
||||
Messages:
|
||||
3->1 MsgAppResp Term:1 Log:0/4
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/4
|
||||
3->1 MsgAppResp Term:1 Log:0/4
|
||||
> 1 handling Ready
|
||||
INFO 1 switched to configuration voters=(2 3)
|
||||
Ready MustSync=false:
|
||||
HardState Term:1 Vote:1 Commit:4
|
||||
CommittedEntries:
|
||||
1/4 EntryConfChange r1
|
||||
Messages:
|
||||
1->2 MsgApp Term:1 Log:1/4 Commit:4
|
||||
1->3 MsgApp Term:1 Log:1/4 Commit:4
|
||||
> 2 receiving messages
|
||||
1->2 MsgApp Term:1 Log:1/4 Commit:4
|
||||
> 3 receiving messages
|
||||
1->3 MsgApp Term:1 Log:1/4 Commit:4
|
||||
> 2 handling Ready
|
||||
INFO 2 switched to configuration voters=(2 3)
|
||||
Ready MustSync=false:
|
||||
HardState Term:1 Vote:1 Commit:4
|
||||
CommittedEntries:
|
||||
1/4 EntryConfChange r1
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/4
|
||||
> 3 handling Ready
|
||||
INFO 3 switched to configuration voters=(2 3)
|
||||
Ready MustSync=false:
|
||||
HardState Term:1 Vote:1 Commit:4
|
||||
CommittedEntries:
|
||||
1/4 EntryConfChange r1
|
||||
Messages:
|
||||
3->1 MsgAppResp Term:1 Log:0/4
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/4
|
||||
3->1 MsgAppResp Term:1 Log:0/4
|
||||
|
||||
status 1
|
||||
----
|
||||
2: StateReplicate match=4 next=5
|
||||
3: StateReplicate match=4 next=5
|
||||
|
||||
# TODO(tbg): the leader now drops any proposals, but if it has any other
|
||||
# uncommitted proposals in its log already, it will likely try to distribute
|
||||
# them which could be buggy. Test that.
|
||||
propose-conf-change 1 v1=true
|
||||
v1
|
||||
----
|
||||
raft proposal dropped
|
@ -15,7 +15,7 @@ INFO 1 became candidate at term 1
|
||||
INFO 1 received MsgVoteResp from 1 at term 1
|
||||
INFO 1 became leader at term 1
|
||||
|
||||
propose-conf-change 1
|
||||
propose-conf-change 1 transition=auto
|
||||
v2 v3
|
||||
----
|
||||
ok
|
||||
@ -32,7 +32,9 @@ INFO newRaft 3 [peers: [], term: 0, commit: 0, applied: 0, lastindex: 0, lastter
|
||||
|
||||
# n1 immediately gets to commit & apply the conf change using only itself. We see that
|
||||
# it starts transitioning out of that joint configuration (though we will only see that
|
||||
# proposal in the next ready handling loop, when it is emitted).
|
||||
# proposal in the next ready handling loop, when it is emitted). We also see that this
|
||||
# is using joint consensus, which it has to since we're carrying out two additions at
|
||||
# once.
|
||||
process-ready 1
|
||||
----
|
||||
INFO 1 switched to configuration voters=(1 2 3)&&(1) autoleave
|
||||
@ -65,7 +67,7 @@ stabilize 1
|
||||
# which transitions them out of their joint configuration into the final one (1 2 3).
|
||||
stabilize 1 2
|
||||
----
|
||||
> delivering messages
|
||||
> 2 receiving messages
|
||||
1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2 v3]
|
||||
INFO 2 [term: 0] received a MsgApp message with higher term from 1 [term: 1]
|
||||
INFO 2 became follower at term 1
|
||||
@ -76,7 +78,7 @@ stabilize 1 2
|
||||
HardState Term:1 Commit:0
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0)
|
||||
> delivering messages
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0)
|
||||
DEBUG 1 received MsgAppResp(MsgApp was rejected, lastindex: 0) from 2 for index 3
|
||||
DEBUG 1 decreased progress of 2 to [StateProbe match=0 next=1]
|
||||
@ -85,27 +87,27 @@ stabilize 1 2
|
||||
> 1 handling Ready
|
||||
Ready MustSync=false:
|
||||
Messages:
|
||||
1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[1] Learners:[] LearnersNext:[]
|
||||
> delivering messages
|
||||
1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[1] Learners:[] LearnersNext:[]
|
||||
1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true
|
||||
> 2 receiving messages
|
||||
1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true
|
||||
INFO log [committed=0, applied=0, unstable.offset=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1]
|
||||
INFO 2 switched to configuration voters=(1 2 3)&&(1)
|
||||
INFO 2 switched to configuration voters=(1 2 3)&&(1) autoleave
|
||||
INFO 2 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1]
|
||||
INFO 2 [commit: 4] restored snapshot [index: 4, term: 1]
|
||||
> 2 handling Ready
|
||||
Ready MustSync=false:
|
||||
HardState Term:1 Commit:4
|
||||
Snapshot Index:4 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[1] Learners:[] LearnersNext:[]
|
||||
Snapshot Index:4 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/4
|
||||
> delivering messages
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/4
|
||||
DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 2 [StateSnapshot match=4 next=5 paused pendingSnap=4]
|
||||
> 1 handling Ready
|
||||
Ready MustSync=false:
|
||||
Messages:
|
||||
1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryConfChangeV2]
|
||||
> delivering messages
|
||||
> 2 receiving messages
|
||||
1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryConfChangeV2]
|
||||
> 2 handling Ready
|
||||
Ready MustSync=true:
|
||||
@ -113,7 +115,7 @@ stabilize 1 2
|
||||
1/5 EntryConfChangeV2
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/5
|
||||
> delivering messages
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/5
|
||||
> 1 handling Ready
|
||||
INFO 1 switched to configuration voters=(1 2 3)
|
||||
@ -123,7 +125,7 @@ stabilize 1 2
|
||||
1/5 EntryConfChangeV2
|
||||
Messages:
|
||||
1->2 MsgApp Term:1 Log:1/5 Commit:5
|
||||
> delivering messages
|
||||
> 2 receiving messages
|
||||
1->2 MsgApp Term:1 Log:1/5 Commit:5
|
||||
> 2 handling Ready
|
||||
INFO 2 switched to configuration voters=(1 2 3)
|
||||
@ -133,13 +135,13 @@ stabilize 1 2
|
||||
1/5 EntryConfChangeV2
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/5
|
||||
> delivering messages
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/5
|
||||
|
||||
# n3 immediately receives a snapshot in the final configuration.
|
||||
stabilize 1 3
|
||||
----
|
||||
> delivering messages
|
||||
> 3 receiving messages
|
||||
1->3 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2 v3]
|
||||
INFO 3 [term: 0] received a MsgApp message with higher term from 1 [term: 1]
|
||||
INFO 3 became follower at term 1
|
||||
@ -150,7 +152,7 @@ stabilize 1 3
|
||||
HardState Term:1 Commit:0
|
||||
Messages:
|
||||
3->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0)
|
||||
> delivering messages
|
||||
> 1 receiving messages
|
||||
3->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0)
|
||||
DEBUG 1 received MsgAppResp(MsgApp was rejected, lastindex: 0) from 3 for index 3
|
||||
DEBUG 1 decreased progress of 3 to [StateProbe match=0 next=1]
|
||||
@ -159,9 +161,9 @@ stabilize 1 3
|
||||
> 1 handling Ready
|
||||
Ready MustSync=false:
|
||||
Messages:
|
||||
1->3 MsgSnap Term:1 Log:0/0 Snapshot: Index:5 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[]
|
||||
> delivering messages
|
||||
1->3 MsgSnap Term:1 Log:0/0 Snapshot: Index:5 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[]
|
||||
1->3 MsgSnap Term:1 Log:0/0 Snapshot: Index:5 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false
|
||||
> 3 receiving messages
|
||||
1->3 MsgSnap Term:1 Log:0/0 Snapshot: Index:5 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false
|
||||
INFO log [committed=0, applied=0, unstable.offset=1, len(unstable.Entries)=0] starts to restore snapshot [index: 5, term: 1]
|
||||
INFO 3 switched to configuration voters=(1 2 3)
|
||||
INFO 3 [commit: 5, lastindex: 5, lastterm: 1] restored snapshot [index: 5, term: 1]
|
||||
@ -169,23 +171,23 @@ stabilize 1 3
|
||||
> 3 handling Ready
|
||||
Ready MustSync=false:
|
||||
HardState Term:1 Commit:5
|
||||
Snapshot Index:5 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[]
|
||||
Snapshot Index:5 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false
|
||||
Messages:
|
||||
3->1 MsgAppResp Term:1 Log:0/5
|
||||
> delivering messages
|
||||
> 1 receiving messages
|
||||
3->1 MsgAppResp Term:1 Log:0/5
|
||||
DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 3 [StateSnapshot match=5 next=6 paused pendingSnap=5]
|
||||
> 1 handling Ready
|
||||
Ready MustSync=false:
|
||||
Messages:
|
||||
1->3 MsgApp Term:1 Log:1/5 Commit:5
|
||||
> delivering messages
|
||||
> 3 receiving messages
|
||||
1->3 MsgApp Term:1 Log:1/5 Commit:5
|
||||
> 3 handling Ready
|
||||
Ready MustSync=false:
|
||||
Messages:
|
||||
3->1 MsgAppResp Term:1 Log:0/5
|
||||
> delivering messages
|
||||
> 1 receiving messages
|
||||
3->1 MsgAppResp Term:1 Log:0/5
|
||||
|
||||
# Nothing else happens.
|
206
raft/testdata/confchange_v2_add_double_explicit.txt
vendored
Normal file
206
raft/testdata/confchange_v2_add_double_explicit.txt
vendored
Normal file
@ -0,0 +1,206 @@
|
||||
# Run a V2 membership change that adds a single voter but explicitly asks for the
|
||||
# use of joint consensus, including wanting to transition out of the joint config
|
||||
# manually.
|
||||
|
||||
# Bootstrap n1.
|
||||
add-nodes 1 voters=(1) index=2
|
||||
----
|
||||
INFO 1 switched to configuration voters=(1)
|
||||
INFO 1 became follower at term 0
|
||||
INFO newRaft 1 [peers: [1], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1]
|
||||
|
||||
campaign 1
|
||||
----
|
||||
INFO 1 is starting a new election at term 0
|
||||
INFO 1 became candidate at term 1
|
||||
INFO 1 received MsgVoteResp from 1 at term 1
|
||||
INFO 1 became leader at term 1
|
||||
|
||||
# Add v2 with an explicit transition.
|
||||
propose-conf-change 1 transition=explicit
|
||||
v2
|
||||
----
|
||||
ok
|
||||
|
||||
# Pull n2 out of thin air.
|
||||
add-nodes 1
|
||||
----
|
||||
INFO 2 switched to configuration voters=()
|
||||
INFO 2 became follower at term 0
|
||||
INFO newRaft 2 [peers: [], term: 0, commit: 0, applied: 0, lastindex: 0, lastterm: 0]
|
||||
|
||||
# n1 commits the conf change using itself as commit quorum, then starts catching up n2.
|
||||
# Everyone remains in the joint config. Note that the snapshot below has AutoLeave unset.
|
||||
stabilize 1 2
|
||||
----
|
||||
> 1 handling Ready
|
||||
INFO 1 switched to configuration voters=(1 2)&&(1)
|
||||
Ready MustSync=true:
|
||||
Lead:1 State:StateLeader
|
||||
HardState Term:1 Vote:1 Commit:4
|
||||
Entries:
|
||||
1/3 EntryNormal ""
|
||||
1/4 EntryConfChangeV2 v2
|
||||
CommittedEntries:
|
||||
1/3 EntryNormal ""
|
||||
1/4 EntryConfChangeV2 v2
|
||||
> 1 handling Ready
|
||||
Ready MustSync=false:
|
||||
Messages:
|
||||
1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2]
|
||||
> 2 receiving messages
|
||||
1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2]
|
||||
INFO 2 [term: 0] received a MsgApp message with higher term from 1 [term: 1]
|
||||
INFO 2 became follower at term 1
|
||||
DEBUG 2 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1
|
||||
> 2 handling Ready
|
||||
Ready MustSync=true:
|
||||
Lead:1 State:StateFollower
|
||||
HardState Term:1 Commit:0
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0)
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0)
|
||||
DEBUG 1 received MsgAppResp(MsgApp was rejected, lastindex: 0) from 2 for index 3
|
||||
DEBUG 1 decreased progress of 2 to [StateProbe match=0 next=1]
|
||||
DEBUG 1 [firstindex: 3, commit: 4] sent snapshot[index: 4, term: 1] to 2 [StateProbe match=0 next=1]
|
||||
DEBUG 1 paused sending replication messages to 2 [StateSnapshot match=0 next=1 paused pendingSnap=4]
|
||||
> 1 handling Ready
|
||||
Ready MustSync=false:
|
||||
Messages:
|
||||
1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:false
|
||||
> 2 receiving messages
|
||||
1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:false
|
||||
INFO log [committed=0, applied=0, unstable.offset=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1]
|
||||
INFO 2 switched to configuration voters=(1 2)&&(1)
|
||||
INFO 2 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1]
|
||||
INFO 2 [commit: 4] restored snapshot [index: 4, term: 1]
|
||||
> 2 handling Ready
|
||||
Ready MustSync=false:
|
||||
HardState Term:1 Commit:4
|
||||
Snapshot Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:false
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/4
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/4
|
||||
DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 2 [StateSnapshot match=4 next=5 paused pendingSnap=4]
|
||||
> 1 handling Ready
|
||||
Ready MustSync=false:
|
||||
Messages:
|
||||
1->2 MsgApp Term:1 Log:1/4 Commit:4
|
||||
> 2 receiving messages
|
||||
1->2 MsgApp Term:1 Log:1/4 Commit:4
|
||||
> 2 handling Ready
|
||||
Ready MustSync=false:
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/4
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/4
|
||||
|
||||
# Check that we're not allowed to change membership again while in the joint state.
|
||||
# This leads to an empty entry being proposed instead (index 5 in the stabilize block
|
||||
# below).
|
||||
propose-conf-change 1
|
||||
v3 v4 v5
|
||||
----
|
||||
INFO 1 ignoring conf change {ConfChangeTransitionAuto [{ConfChangeAddNode 3 []} {ConfChangeAddNode 4 []} {ConfChangeAddNode 5 []}] [] []} at config voters=(1 2)&&(1): must transition out of joint config first
|
||||
|
||||
# Propose a transition out of the joint config. We'll see this at index 6 below.
|
||||
propose-conf-change 1
|
||||
----
|
||||
ok
|
||||
|
||||
# The group commits the command and everyone switches to the final config.
|
||||
stabilize
|
||||
----
|
||||
> 1 handling Ready
|
||||
Ready MustSync=true:
|
||||
Entries:
|
||||
1/5 EntryNormal ""
|
||||
1/6 EntryConfChangeV2
|
||||
Messages:
|
||||
1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal ""]
|
||||
1->2 MsgApp Term:1 Log:1/5 Commit:4 Entries:[1/6 EntryConfChangeV2]
|
||||
> 2 receiving messages
|
||||
1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal ""]
|
||||
1->2 MsgApp Term:1 Log:1/5 Commit:4 Entries:[1/6 EntryConfChangeV2]
|
||||
> 2 handling Ready
|
||||
Ready MustSync=true:
|
||||
Entries:
|
||||
1/5 EntryNormal ""
|
||||
1/6 EntryConfChangeV2
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/5
|
||||
2->1 MsgAppResp Term:1 Log:0/6
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/5
|
||||
2->1 MsgAppResp Term:1 Log:0/6
|
||||
> 1 handling Ready
|
||||
INFO 1 switched to configuration voters=(1 2)
|
||||
Ready MustSync=false:
|
||||
HardState Term:1 Vote:1 Commit:6
|
||||
CommittedEntries:
|
||||
1/5 EntryNormal ""
|
||||
1/6 EntryConfChangeV2
|
||||
Messages:
|
||||
1->2 MsgApp Term:1 Log:1/6 Commit:5
|
||||
1->2 MsgApp Term:1 Log:1/6 Commit:6
|
||||
> 2 receiving messages
|
||||
1->2 MsgApp Term:1 Log:1/6 Commit:5
|
||||
1->2 MsgApp Term:1 Log:1/6 Commit:6
|
||||
> 2 handling Ready
|
||||
INFO 2 switched to configuration voters=(1 2)
|
||||
Ready MustSync=false:
|
||||
HardState Term:1 Commit:6
|
||||
CommittedEntries:
|
||||
1/5 EntryNormal ""
|
||||
1/6 EntryConfChangeV2
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/6
|
||||
2->1 MsgAppResp Term:1 Log:0/6
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/6
|
||||
2->1 MsgAppResp Term:1 Log:0/6
|
||||
|
||||
# Check that trying to transition out again won't do anything.
|
||||
propose-conf-change 1
|
||||
----
|
||||
INFO 1 ignoring conf change {ConfChangeTransitionAuto [] [] []} at config voters=(1 2): not in joint state; refusing empty conf change
|
||||
|
||||
# Finishes work for the empty entry we just proposed.
|
||||
stabilize
|
||||
----
|
||||
> 1 handling Ready
|
||||
Ready MustSync=true:
|
||||
Entries:
|
||||
1/7 EntryNormal ""
|
||||
Messages:
|
||||
1->2 MsgApp Term:1 Log:1/6 Commit:6 Entries:[1/7 EntryNormal ""]
|
||||
> 2 receiving messages
|
||||
1->2 MsgApp Term:1 Log:1/6 Commit:6 Entries:[1/7 EntryNormal ""]
|
||||
> 2 handling Ready
|
||||
Ready MustSync=true:
|
||||
Entries:
|
||||
1/7 EntryNormal ""
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/7
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/7
|
||||
> 1 handling Ready
|
||||
Ready MustSync=false:
|
||||
HardState Term:1 Vote:1 Commit:7
|
||||
CommittedEntries:
|
||||
1/7 EntryNormal ""
|
||||
Messages:
|
||||
1->2 MsgApp Term:1 Log:1/7 Commit:7
|
||||
> 2 receiving messages
|
||||
1->2 MsgApp Term:1 Log:1/7 Commit:7
|
||||
> 2 handling Ready
|
||||
Ready MustSync=false:
|
||||
HardState Term:1 Commit:7
|
||||
CommittedEntries:
|
||||
1/7 EntryNormal ""
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/7
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/7
|
125
raft/testdata/confchange_v2_add_double_implicit.txt
vendored
Normal file
125
raft/testdata/confchange_v2_add_double_implicit.txt
vendored
Normal file
@ -0,0 +1,125 @@
|
||||
# Run a V2 membership change that adds a single voter but explicitly asks for the
|
||||
# use of joint consensus (with auto-leaving).
|
||||
|
||||
# TODO(tbg): also verify that if the leader changes while in the joint state, the
|
||||
# new leader will auto-transition out of the joint state just the same.
|
||||
|
||||
# Bootstrap n1.
|
||||
add-nodes 1 voters=(1) index=2
|
||||
----
|
||||
INFO 1 switched to configuration voters=(1)
|
||||
INFO 1 became follower at term 0
|
||||
INFO newRaft 1 [peers: [1], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1]
|
||||
|
||||
campaign 1
|
||||
----
|
||||
INFO 1 is starting a new election at term 0
|
||||
INFO 1 became candidate at term 1
|
||||
INFO 1 received MsgVoteResp from 1 at term 1
|
||||
INFO 1 became leader at term 1
|
||||
|
||||
propose-conf-change 1 transition=implicit
|
||||
v2
|
||||
----
|
||||
ok
|
||||
|
||||
# Add n2.
|
||||
add-nodes 1
|
||||
----
|
||||
INFO 2 switched to configuration voters=()
|
||||
INFO 2 became follower at term 0
|
||||
INFO newRaft 2 [peers: [], term: 0, commit: 0, applied: 0, lastindex: 0, lastterm: 0]
|
||||
|
||||
# n1 commits the conf change using itself as commit quorum, then starts catching up n2.
|
||||
# When that's done, it starts auto-transitioning out. Note that the snapshots propagating
|
||||
# the joint config have the AutoLeave flag set in their config.
|
||||
stabilize 1 2
|
||||
----
|
||||
> 1 handling Ready
|
||||
INFO 1 switched to configuration voters=(1 2)&&(1) autoleave
|
||||
INFO initiating automatic transition out of joint configuration voters=(1 2)&&(1) autoleave
|
||||
Ready MustSync=true:
|
||||
Lead:1 State:StateLeader
|
||||
HardState Term:1 Vote:1 Commit:4
|
||||
Entries:
|
||||
1/3 EntryNormal ""
|
||||
1/4 EntryConfChangeV2 v2
|
||||
CommittedEntries:
|
||||
1/3 EntryNormal ""
|
||||
1/4 EntryConfChangeV2 v2
|
||||
> 1 handling Ready
|
||||
Ready MustSync=true:
|
||||
Entries:
|
||||
1/5 EntryConfChangeV2
|
||||
Messages:
|
||||
1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2]
|
||||
> 2 receiving messages
|
||||
1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2]
|
||||
INFO 2 [term: 0] received a MsgApp message with higher term from 1 [term: 1]
|
||||
INFO 2 became follower at term 1
|
||||
DEBUG 2 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1
|
||||
> 2 handling Ready
|
||||
Ready MustSync=true:
|
||||
Lead:1 State:StateFollower
|
||||
HardState Term:1 Commit:0
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0)
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0)
|
||||
DEBUG 1 received MsgAppResp(MsgApp was rejected, lastindex: 0) from 2 for index 3
|
||||
DEBUG 1 decreased progress of 2 to [StateProbe match=0 next=1]
|
||||
DEBUG 1 [firstindex: 3, commit: 4] sent snapshot[index: 4, term: 1] to 2 [StateProbe match=0 next=1]
|
||||
DEBUG 1 paused sending replication messages to 2 [StateSnapshot match=0 next=1 paused pendingSnap=4]
|
||||
> 1 handling Ready
|
||||
Ready MustSync=false:
|
||||
Messages:
|
||||
1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true
|
||||
> 2 receiving messages
|
||||
1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true
|
||||
INFO log [committed=0, applied=0, unstable.offset=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1]
|
||||
INFO 2 switched to configuration voters=(1 2)&&(1) autoleave
|
||||
INFO 2 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1]
|
||||
INFO 2 [commit: 4] restored snapshot [index: 4, term: 1]
|
||||
> 2 handling Ready
|
||||
Ready MustSync=false:
|
||||
HardState Term:1 Commit:4
|
||||
Snapshot Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/4
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/4
|
||||
DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 2 [StateSnapshot match=4 next=5 paused pendingSnap=4]
|
||||
> 1 handling Ready
|
||||
Ready MustSync=false:
|
||||
Messages:
|
||||
1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryConfChangeV2]
|
||||
> 2 receiving messages
|
||||
1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryConfChangeV2]
|
||||
> 2 handling Ready
|
||||
Ready MustSync=true:
|
||||
Entries:
|
||||
1/5 EntryConfChangeV2
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/5
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/5
|
||||
> 1 handling Ready
|
||||
INFO 1 switched to configuration voters=(1 2)
|
||||
Ready MustSync=false:
|
||||
HardState Term:1 Vote:1 Commit:5
|
||||
CommittedEntries:
|
||||
1/5 EntryConfChangeV2
|
||||
Messages:
|
||||
1->2 MsgApp Term:1 Log:1/5 Commit:5
|
||||
> 2 receiving messages
|
||||
1->2 MsgApp Term:1 Log:1/5 Commit:5
|
||||
> 2 handling Ready
|
||||
INFO 2 switched to configuration voters=(1 2)
|
||||
Ready MustSync=false:
|
||||
HardState Term:1 Commit:5
|
||||
CommittedEntries:
|
||||
1/5 EntryConfChangeV2
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/5
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/5
|
198
raft/testdata/confchange_v2_add_single_auto.txt
vendored
Normal file
198
raft/testdata/confchange_v2_add_single_auto.txt
vendored
Normal file
@ -0,0 +1,198 @@
|
||||
# Run a V2 membership change that adds a single voter in auto mode, which means
|
||||
# that joint consensus is not used but a direct transition into the new config
|
||||
# takes place.
|
||||
|
||||
# Bootstrap n1.
|
||||
add-nodes 1 voters=(1) index=2
|
||||
----
|
||||
INFO 1 switched to configuration voters=(1)
|
||||
INFO 1 became follower at term 0
|
||||
INFO newRaft 1 [peers: [1], term: 0, commit: 2, applied: 2, lastindex: 2, lastterm: 1]
|
||||
|
||||
campaign 1
|
||||
----
|
||||
INFO 1 is starting a new election at term 0
|
||||
INFO 1 became candidate at term 1
|
||||
INFO 1 received MsgVoteResp from 1 at term 1
|
||||
INFO 1 became leader at term 1
|
||||
|
||||
# Add v2 (with an auto transition).
|
||||
propose-conf-change 1
|
||||
v2
|
||||
----
|
||||
ok
|
||||
|
||||
# Pull n2 out of thin air.
|
||||
add-nodes 1
|
||||
----
|
||||
INFO 2 switched to configuration voters=()
|
||||
INFO 2 became follower at term 0
|
||||
INFO newRaft 2 [peers: [], term: 0, commit: 0, applied: 0, lastindex: 0, lastterm: 0]
|
||||
|
||||
# n1 commits the conf change using itself as commit quorum, immediately transitions into
|
||||
# the final config, and catches up n2.
|
||||
stabilize
|
||||
----
|
||||
> 1 handling Ready
|
||||
INFO 1 switched to configuration voters=(1 2)
|
||||
Ready MustSync=true:
|
||||
Lead:1 State:StateLeader
|
||||
HardState Term:1 Vote:1 Commit:4
|
||||
Entries:
|
||||
1/3 EntryNormal ""
|
||||
1/4 EntryConfChangeV2 v2
|
||||
CommittedEntries:
|
||||
1/3 EntryNormal ""
|
||||
1/4 EntryConfChangeV2 v2
|
||||
> 1 handling Ready
|
||||
Ready MustSync=false:
|
||||
Messages:
|
||||
1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2]
|
||||
> 2 receiving messages
|
||||
1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 v2]
|
||||
INFO 2 [term: 0] received a MsgApp message with higher term from 1 [term: 1]
|
||||
INFO 2 became follower at term 1
|
||||
DEBUG 2 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1
|
||||
> 2 handling Ready
|
||||
Ready MustSync=true:
|
||||
Lead:1 State:StateFollower
|
||||
HardState Term:1 Commit:0
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0)
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0)
|
||||
DEBUG 1 received MsgAppResp(MsgApp was rejected, lastindex: 0) from 2 for index 3
|
||||
DEBUG 1 decreased progress of 2 to [StateProbe match=0 next=1]
|
||||
DEBUG 1 [firstindex: 3, commit: 4] sent snapshot[index: 4, term: 1] to 2 [StateProbe match=0 next=1]
|
||||
DEBUG 1 paused sending replication messages to 2 [StateSnapshot match=0 next=1 paused pendingSnap=4]
|
||||
> 1 handling Ready
|
||||
Ready MustSync=false:
|
||||
Messages:
|
||||
1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false
|
||||
> 2 receiving messages
|
||||
1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false
|
||||
INFO log [committed=0, applied=0, unstable.offset=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1]
|
||||
INFO 2 switched to configuration voters=(1 2)
|
||||
INFO 2 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1]
|
||||
INFO 2 [commit: 4] restored snapshot [index: 4, term: 1]
|
||||
> 2 handling Ready
|
||||
Ready MustSync=false:
|
||||
HardState Term:1 Commit:4
|
||||
Snapshot Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/4
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/4
|
||||
DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 2 [StateSnapshot match=4 next=5 paused pendingSnap=4]
|
||||
> 1 handling Ready
|
||||
Ready MustSync=false:
|
||||
Messages:
|
||||
1->2 MsgApp Term:1 Log:1/4 Commit:4
|
||||
> 2 receiving messages
|
||||
1->2 MsgApp Term:1 Log:1/4 Commit:4
|
||||
> 2 handling Ready
|
||||
Ready MustSync=false:
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/4
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/4
|
||||
|
||||
# Check that we're not allowed to change membership again while in the joint state.
|
||||
# This leads to an empty entry being proposed instead (index 5 in the stabilize block
|
||||
# below).
|
||||
propose-conf-change 1
|
||||
v3 v4 v5
|
||||
----
|
||||
ok
|
||||
|
||||
# Propose a transition out of the joint config. We'll see this at index 6 below.
|
||||
propose-conf-change 1
|
||||
----
|
||||
INFO 1 ignoring conf change {ConfChangeTransitionAuto [] [] []} at config voters=(1 2): possible unapplied conf change at index 5 (applied to 4)
|
||||
|
||||
# The group commits the command and everyone switches to the final config.
|
||||
stabilize
|
||||
----
|
||||
> 1 handling Ready
|
||||
Ready MustSync=true:
|
||||
Entries:
|
||||
1/5 EntryConfChangeV2 v3 v4 v5
|
||||
1/6 EntryNormal ""
|
||||
Messages:
|
||||
1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryConfChangeV2 v3 v4 v5]
|
||||
1->2 MsgApp Term:1 Log:1/5 Commit:4 Entries:[1/6 EntryNormal ""]
|
||||
> 2 receiving messages
|
||||
1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryConfChangeV2 v3 v4 v5]
|
||||
1->2 MsgApp Term:1 Log:1/5 Commit:4 Entries:[1/6 EntryNormal ""]
|
||||
> 2 handling Ready
|
||||
Ready MustSync=true:
|
||||
Entries:
|
||||
1/5 EntryConfChangeV2 v3 v4 v5
|
||||
1/6 EntryNormal ""
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/5
|
||||
2->1 MsgAppResp Term:1 Log:0/6
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/5
|
||||
2->1 MsgAppResp Term:1 Log:0/6
|
||||
> 1 handling Ready
|
||||
INFO 1 switched to configuration voters=(1 2 3 4 5)&&(1 2) autoleave
|
||||
INFO initiating automatic transition out of joint configuration voters=(1 2 3 4 5)&&(1 2) autoleave
|
||||
Ready MustSync=false:
|
||||
HardState Term:1 Vote:1 Commit:6
|
||||
CommittedEntries:
|
||||
1/5 EntryConfChangeV2 v3 v4 v5
|
||||
1/6 EntryNormal ""
|
||||
Messages:
|
||||
1->2 MsgApp Term:1 Log:1/6 Commit:5
|
||||
1->2 MsgApp Term:1 Log:1/6 Commit:6
|
||||
> 2 receiving messages
|
||||
1->2 MsgApp Term:1 Log:1/6 Commit:5
|
||||
1->2 MsgApp Term:1 Log:1/6 Commit:6
|
||||
> 1 handling Ready
|
||||
Ready MustSync=true:
|
||||
Entries:
|
||||
1/7 EntryConfChangeV2
|
||||
Messages:
|
||||
1->3 MsgApp Term:1 Log:1/5 Commit:6 Entries:[1/6 EntryNormal ""]
|
||||
1->4 MsgApp Term:1 Log:1/5 Commit:6 Entries:[1/6 EntryNormal ""]
|
||||
1->5 MsgApp Term:1 Log:1/5 Commit:6 Entries:[1/6 EntryNormal ""]
|
||||
> 2 handling Ready
|
||||
INFO 2 switched to configuration voters=(1 2 3 4 5)&&(1 2) autoleave
|
||||
Ready MustSync=false:
|
||||
HardState Term:1 Commit:6
|
||||
CommittedEntries:
|
||||
1/5 EntryConfChangeV2 v3 v4 v5
|
||||
1/6 EntryNormal ""
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/6
|
||||
2->1 MsgAppResp Term:1 Log:0/6
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/6
|
||||
2->1 MsgAppResp Term:1 Log:0/6
|
||||
|
||||
# Check that trying to transition out again won't do anything.
|
||||
propose-conf-change 1
|
||||
----
|
||||
ok
|
||||
|
||||
# Finishes work for the empty entry we just proposed.
|
||||
stabilize
|
||||
----
|
||||
> 1 handling Ready
|
||||
Ready MustSync=true:
|
||||
Entries:
|
||||
1/8 EntryConfChangeV2
|
||||
Messages:
|
||||
1->2 MsgApp Term:1 Log:1/6 Commit:6 Entries:[1/7 EntryConfChangeV2, 1/8 EntryConfChangeV2]
|
||||
> 2 receiving messages
|
||||
1->2 MsgApp Term:1 Log:1/6 Commit:6 Entries:[1/7 EntryConfChangeV2, 1/8 EntryConfChangeV2]
|
||||
> 2 handling Ready
|
||||
Ready MustSync=true:
|
||||
Entries:
|
||||
1/7 EntryConfChangeV2
|
||||
1/8 EntryConfChangeV2
|
||||
Messages:
|
||||
2->1 MsgAppResp Term:1 Log:0/8
|
||||
> 1 receiving messages
|
||||
2->1 MsgAppResp Term:1 Log:0/8
|
22
raft/testdata/snapshot_succeed_via_app_resp.txt
vendored
22
raft/testdata/snapshot_succeed_via_app_resp.txt
vendored
@ -30,7 +30,7 @@ compact 1 11
|
||||
ok (quiet)
|
||||
|
||||
# Drop inflight messages to n3.
|
||||
deliver-msgs 3
|
||||
deliver-msgs drop=(3)
|
||||
----
|
||||
ok (quiet)
|
||||
|
||||
@ -70,7 +70,7 @@ Messages:
|
||||
# and responds.
|
||||
stabilize 3
|
||||
----
|
||||
> delivering messages
|
||||
> 3 receiving messages
|
||||
1->3 MsgHeartbeat Term:1 Log:0/0
|
||||
INFO 3 [term: 0] received a MsgHeartbeat message with higher term from 1 [term: 1]
|
||||
INFO 3 became follower at term 1
|
||||
@ -84,14 +84,14 @@ stabilize 3
|
||||
# The leader in turn will realize that n3 needs a snapshot, which it initiates.
|
||||
stabilize 1
|
||||
----
|
||||
> delivering messages
|
||||
> 1 receiving messages
|
||||
3->1 MsgHeartbeatResp Term:1 Log:0/0
|
||||
DEBUG 1 [firstindex: 12, commit: 11] sent snapshot[index: 11, term: 1] to 3 [StateProbe match=0 next=11]
|
||||
DEBUG 1 paused sending replication messages to 3 [StateSnapshot match=0 next=11 paused pendingSnap=11]
|
||||
> 1 handling Ready
|
||||
Ready MustSync=false:
|
||||
Messages:
|
||||
1->3 MsgSnap Term:1 Log:0/0 Snapshot: Index:11 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[]
|
||||
1->3 MsgSnap Term:1 Log:0/0 Snapshot: Index:11 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false
|
||||
|
||||
status 1
|
||||
----
|
||||
@ -105,8 +105,8 @@ status 1
|
||||
# was now fully caught up.
|
||||
stabilize 3
|
||||
----
|
||||
> delivering messages
|
||||
1->3 MsgSnap Term:1 Log:0/0 Snapshot: Index:11 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[]
|
||||
> 3 receiving messages
|
||||
1->3 MsgSnap Term:1 Log:0/0 Snapshot: Index:11 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false
|
||||
INFO log [committed=0, applied=0, unstable.offset=1, len(unstable.Entries)=0] starts to restore snapshot [index: 11, term: 1]
|
||||
INFO 3 switched to configuration voters=(1 2 3)
|
||||
INFO 3 [commit: 11, lastindex: 11, lastterm: 1] restored snapshot [index: 11, term: 1]
|
||||
@ -114,7 +114,7 @@ stabilize 3
|
||||
> 3 handling Ready
|
||||
Ready MustSync=false:
|
||||
HardState Term:1 Commit:11
|
||||
Snapshot Index:11 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[]
|
||||
Snapshot Index:11 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false
|
||||
Messages:
|
||||
3->1 MsgAppResp Term:1 Log:0/11
|
||||
|
||||
@ -122,7 +122,7 @@ stabilize 3
|
||||
# Leader sends another MsgAppResp, to communicate the updated commit index.
|
||||
stabilize 1
|
||||
----
|
||||
> delivering messages
|
||||
> 1 receiving messages
|
||||
3->1 MsgAppResp Term:1 Log:0/11
|
||||
DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 3 [StateSnapshot match=11 next=12 paused pendingSnap=11]
|
||||
> 1 handling Ready
|
||||
@ -139,9 +139,9 @@ status 1
|
||||
# Let things settle.
|
||||
stabilize
|
||||
----
|
||||
> delivering messages
|
||||
> 2 receiving messages
|
||||
1->2 MsgHeartbeat Term:1 Log:0/0 Commit:11
|
||||
> delivering messages
|
||||
> 3 receiving messages
|
||||
1->3 MsgApp Term:1 Log:1/11 Commit:11
|
||||
> 2 handling Ready
|
||||
Ready MustSync=false:
|
||||
@ -151,6 +151,6 @@ stabilize
|
||||
Ready MustSync=false:
|
||||
Messages:
|
||||
3->1 MsgAppResp Term:1 Log:0/11
|
||||
> delivering messages
|
||||
> 1 receiving messages
|
||||
2->1 MsgHeartbeatResp Term:1 Log:0/0
|
||||
3->1 MsgAppResp Term:1 Log:0/11
|
||||
|
@ -77,8 +77,8 @@ func DescribeSoftState(ss SoftState) string {
|
||||
|
||||
func DescribeConfState(state pb.ConfState) string {
|
||||
return fmt.Sprintf(
|
||||
"Voters:%v VotersOutgoing:%v Learners:%v LearnersNext:%v",
|
||||
state.Voters, state.VotersOutgoing, state.Learners, state.LearnersNext,
|
||||
"Voters:%v VotersOutgoing:%v Learners:%v LearnersNext:%v AutoLeave:%v",
|
||||
state.Voters, state.VotersOutgoing, state.Learners, state.LearnersNext, state.AutoLeave,
|
||||
)
|
||||
}
|
||||
|
||||
|
Reference in New Issue
Block a user