mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
Merge pull request #11046 from tbg/joint-auto-leave
raft: fix auto-transitioning out of joint config
This commit is contained in:
commit
52fba431b6
@ -168,7 +168,9 @@ type Node interface {
|
|||||||
Advance()
|
Advance()
|
||||||
// ApplyConfChange applies a config change (previously passed to
|
// ApplyConfChange applies a config change (previously passed to
|
||||||
// ProposeConfChange) to the node. This must be called whenever a config
|
// ProposeConfChange) to the node. This must be called whenever a config
|
||||||
// change is observed in Ready.CommittedEntries.
|
// change is observed in Ready.CommittedEntries, except when the app decides
|
||||||
|
// to reject the configuration change (i.e. treats it as a noop instead), in
|
||||||
|
// which case it must not be called.
|
||||||
//
|
//
|
||||||
// Returns an opaque non-nil ConfState protobuf which must be recorded in
|
// Returns an opaque non-nil ConfState protobuf which must be recorded in
|
||||||
// snapshots.
|
// snapshots.
|
||||||
|
42
raft/raft.go
42
raft/raft.go
@ -554,35 +554,34 @@ func (r *raft) bcastHeartbeatWithCtx(ctx []byte) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (r *raft) advance(rd Ready) {
|
func (r *raft) advance(rd Ready) {
|
||||||
|
r.reduceUncommittedSize(rd.CommittedEntries)
|
||||||
|
|
||||||
// If entries were applied (or a snapshot), update our cursor for
|
// If entries were applied (or a snapshot), update our cursor for
|
||||||
// the next Ready. Note that if the current HardState contains a
|
// the next Ready. Note that if the current HardState contains a
|
||||||
// new Commit index, this does not mean that we're also applying
|
// new Commit index, this does not mean that we're also applying
|
||||||
// all of the new entries due to commit pagination by size.
|
// all of the new entries due to commit pagination by size.
|
||||||
if index := rd.appliedCursor(); index > 0 {
|
if newApplied := rd.appliedCursor(); newApplied > 0 {
|
||||||
r.raftLog.appliedTo(index)
|
oldApplied := r.raftLog.applied
|
||||||
if r.prs.Config.AutoLeave && index >= r.pendingConfIndex && r.state == StateLeader {
|
r.raftLog.appliedTo(newApplied)
|
||||||
|
|
||||||
|
if r.prs.Config.AutoLeave && oldApplied < r.pendingConfIndex && newApplied >= r.pendingConfIndex && r.state == StateLeader {
|
||||||
// If the current (and most recent, at least for this leader's term)
|
// If the current (and most recent, at least for this leader's term)
|
||||||
// configuration should be auto-left, initiate that now.
|
// configuration should be auto-left, initiate that now. We use a
|
||||||
ccdata, err := (&pb.ConfChangeV2{}).Marshal()
|
// nil Data which unmarshals into an empty ConfChangeV2 and has the
|
||||||
if err != nil {
|
// benefit that appendEntry can never refuse it based on its size
|
||||||
panic(err)
|
// (which registers as zero).
|
||||||
}
|
|
||||||
ent := pb.Entry{
|
ent := pb.Entry{
|
||||||
Type: pb.EntryConfChangeV2,
|
Type: pb.EntryConfChangeV2,
|
||||||
Data: ccdata,
|
Data: nil,
|
||||||
}
|
}
|
||||||
|
// There's no way in which this proposal should be able to be rejected.
|
||||||
if !r.appendEntry(ent) {
|
if !r.appendEntry(ent) {
|
||||||
// If we could not append the entry, bump the pending conf index
|
panic("refused un-refusable auto-leaving ConfChangeV2")
|
||||||
// so that we'll try again later.
|
|
||||||
//
|
|
||||||
// TODO(tbg): test this case.
|
|
||||||
r.pendingConfIndex = r.raftLog.lastIndex()
|
|
||||||
} else {
|
|
||||||
r.logger.Infof("initiating automatic transition out of joint configuration %s", r.prs.Config)
|
|
||||||
}
|
}
|
||||||
|
r.pendingConfIndex = r.raftLog.lastIndex()
|
||||||
|
r.logger.Infof("initiating automatic transition out of joint configuration %s", r.prs.Config)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
r.reduceUncommittedSize(rd.CommittedEntries)
|
|
||||||
|
|
||||||
if len(rd.Entries) > 0 {
|
if len(rd.Entries) > 0 {
|
||||||
e := rd.Entries[len(rd.Entries)-1]
|
e := rd.Entries[len(rd.Entries)-1]
|
||||||
@ -1607,16 +1606,23 @@ func (r *raft) abortLeaderTransfer() {
|
|||||||
// If the new entries would exceed the limit, the method returns false. If not,
|
// If the new entries would exceed the limit, the method returns false. If not,
|
||||||
// the increase in uncommitted entry size is recorded and the method returns
|
// the increase in uncommitted entry size is recorded and the method returns
|
||||||
// true.
|
// true.
|
||||||
|
//
|
||||||
|
// Empty payloads are never refused. This is used both for appending an empty
|
||||||
|
// entry at a new leader's term, as well as leaving a joint configuration.
|
||||||
func (r *raft) increaseUncommittedSize(ents []pb.Entry) bool {
|
func (r *raft) increaseUncommittedSize(ents []pb.Entry) bool {
|
||||||
var s uint64
|
var s uint64
|
||||||
for _, e := range ents {
|
for _, e := range ents {
|
||||||
s += uint64(PayloadSize(e))
|
s += uint64(PayloadSize(e))
|
||||||
}
|
}
|
||||||
|
|
||||||
if r.uncommittedSize > 0 && r.uncommittedSize+s > r.maxUncommittedSize {
|
if r.uncommittedSize > 0 && s > 0 && r.uncommittedSize+s > r.maxUncommittedSize {
|
||||||
// If the uncommitted tail of the Raft log is empty, allow any size
|
// If the uncommitted tail of the Raft log is empty, allow any size
|
||||||
// proposal. Otherwise, limit the size of the uncommitted tail of the
|
// proposal. Otherwise, limit the size of the uncommitted tail of the
|
||||||
// log and drop any proposal that would push the size over the limit.
|
// log and drop any proposal that would push the size over the limit.
|
||||||
|
// Note the added requirement s>0 which is used to make sure that
|
||||||
|
// appending single empty entries to the log always succeeds, used both
|
||||||
|
// for replicating a new leader's initial empty entry, and for
|
||||||
|
// auto-leaving joint configurations.
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
r.uncommittedSize += s
|
r.uncommittedSize += s
|
||||||
|
@ -186,6 +186,10 @@ func TestUncommittedEntryLimit(t *testing.T) {
|
|||||||
testEntry := pb.Entry{Data: []byte("testdata")}
|
testEntry := pb.Entry{Data: []byte("testdata")}
|
||||||
maxEntrySize := maxEntries * PayloadSize(testEntry)
|
maxEntrySize := maxEntries * PayloadSize(testEntry)
|
||||||
|
|
||||||
|
if n := PayloadSize(pb.Entry{Data: nil}); n != 0 {
|
||||||
|
t.Fatal("entry with no Data must have zero payload size")
|
||||||
|
}
|
||||||
|
|
||||||
cfg := newTestConfig(1, []uint64{1, 2, 3}, 5, 1, NewMemoryStorage())
|
cfg := newTestConfig(1, []uint64{1, 2, 3}, 5, 1, NewMemoryStorage())
|
||||||
cfg.MaxUncommittedEntriesSize = uint64(maxEntrySize)
|
cfg.MaxUncommittedEntriesSize = uint64(maxEntrySize)
|
||||||
cfg.MaxInflightMsgs = 2 * 1024 // avoid interference
|
cfg.MaxInflightMsgs = 2 * 1024 // avoid interference
|
||||||
@ -244,10 +248,19 @@ func TestUncommittedEntryLimit(t *testing.T) {
|
|||||||
t.Fatalf("proposal not dropped: %v", err)
|
t.Fatalf("proposal not dropped: %v", err)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// But we can always append an entry with no Data. This is used both for the
|
||||||
|
// leader's first empty entry and for auto-transitioning out of joint config
|
||||||
|
// states.
|
||||||
|
if err := r.Step(
|
||||||
|
pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}},
|
||||||
|
); err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
|
||||||
// Read messages and reduce the uncommitted size as if we had committed
|
// Read messages and reduce the uncommitted size as if we had committed
|
||||||
// these entries.
|
// these entries.
|
||||||
ms = r.readMessages()
|
ms = r.readMessages()
|
||||||
if e := 1 * numFollowers; len(ms) != e {
|
if e := 2 * numFollowers; len(ms) != e {
|
||||||
t.Fatalf("expected %d messages, got %d", e, len(ms))
|
t.Fatalf("expected %d messages, got %d", e, len(ms))
|
||||||
}
|
}
|
||||||
r.reduceUncommittedSize(propEnts)
|
r.reduceUncommittedSize(propEnts)
|
||||||
|
@ -73,7 +73,7 @@ func (c ConfChangeV2) AsV1() (ConfChange, bool) { return ConfChange{}, false }
|
|||||||
// than one change or if the use of Joint Consensus was requested explicitly.
|
// than one change or if the use of Joint Consensus was requested explicitly.
|
||||||
// The first bool can only be true if second one is, and indicates whether the
|
// The first bool can only be true if second one is, and indicates whether the
|
||||||
// Joint State will be left automatically.
|
// Joint State will be left automatically.
|
||||||
func (c *ConfChangeV2) EnterJoint() (autoLeave bool, ok bool) {
|
func (c ConfChangeV2) EnterJoint() (autoLeave bool, ok bool) {
|
||||||
// NB: in theory, more config changes could qualify for the "simple"
|
// NB: in theory, more config changes could qualify for the "simple"
|
||||||
// protocol but it depends on the config on top of which the changes apply.
|
// protocol but it depends on the config on top of which the changes apply.
|
||||||
// For example, adding two learners is not OK if both nodes are part of the
|
// For example, adding two learners is not OK if both nodes are part of the
|
||||||
@ -100,10 +100,10 @@ func (c *ConfChangeV2) EnterJoint() (autoLeave bool, ok bool) {
|
|||||||
// LeaveJoint is true if the configuration change leaves a joint configuration.
|
// LeaveJoint is true if the configuration change leaves a joint configuration.
|
||||||
// This is the case if the ConfChangeV2 is zero, with the possible exception of
|
// This is the case if the ConfChangeV2 is zero, with the possible exception of
|
||||||
// the Context field.
|
// the Context field.
|
||||||
func (c *ConfChangeV2) LeaveJoint() bool {
|
func (c ConfChangeV2) LeaveJoint() bool {
|
||||||
cpy := *c
|
// NB: c is already a copy.
|
||||||
cpy.Context = nil
|
c.Context = nil
|
||||||
return proto.Equal(&cpy, &ConfChangeV2{})
|
return proto.Equal(&c, &ConfChangeV2{})
|
||||||
}
|
}
|
||||||
|
|
||||||
// ConfChangesFromString parses a Space-delimited sequence of operations into a
|
// ConfChangesFromString parses a Space-delimited sequence of operations into a
|
||||||
|
@ -86,7 +86,7 @@ func (env *InteractionEnv) DeliverMsgs(rs ...Recipient) int {
|
|||||||
}
|
}
|
||||||
toIdx := int(msg.To - 1)
|
toIdx := int(msg.To - 1)
|
||||||
if err := env.Nodes[toIdx].Step(msg); err != nil {
|
if err := env.Nodes[toIdx].Step(msg); err != nil {
|
||||||
env.Output.WriteString(err.Error())
|
fmt.Fprintln(env.Output, err)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -98,7 +98,9 @@ func (rn *RawNode) ProposeConfChange(cc pb.ConfChangeI) error {
|
|||||||
return rn.raft.Step(m)
|
return rn.raft.Step(m)
|
||||||
}
|
}
|
||||||
|
|
||||||
// ApplyConfChange applies a config change to the local node.
|
// ApplyConfChange applies a config change to the local node. The app must call
|
||||||
|
// this when it applies a configuration change, except when it decides to reject
|
||||||
|
// the configuration change, in which case no call must take place.
|
||||||
func (rn *RawNode) ApplyConfChange(cc pb.ConfChangeI) *pb.ConfState {
|
func (rn *RawNode) ApplyConfChange(cc pb.ConfChangeI) *pb.ConfState {
|
||||||
cs := rn.raft.applyConfChange(cc.AsV2())
|
cs := rn.raft.applyConfChange(cc.AsV2())
|
||||||
return &cs
|
return &cs
|
||||||
|
@ -200,11 +200,12 @@ func TestRawNodeProposeAndConfChange(t *testing.T) {
|
|||||||
},
|
},
|
||||||
// Ditto implicit.
|
// Ditto implicit.
|
||||||
{
|
{
|
||||||
pb.ConfChangeV2{Changes: []pb.ConfChangeSingle{
|
pb.ConfChangeV2{
|
||||||
{NodeID: 2, Type: pb.ConfChangeAddNode},
|
Changes: []pb.ConfChangeSingle{
|
||||||
{NodeID: 1, Type: pb.ConfChangeAddLearnerNode},
|
{NodeID: 2, Type: pb.ConfChangeAddNode},
|
||||||
{NodeID: 3, Type: pb.ConfChangeAddLearnerNode},
|
{NodeID: 1, Type: pb.ConfChangeAddLearnerNode},
|
||||||
},
|
{NodeID: 3, Type: pb.ConfChangeAddLearnerNode},
|
||||||
|
},
|
||||||
Transition: pb.ConfChangeTransitionJointImplicit,
|
Transition: pb.ConfChangeTransitionJointImplicit,
|
||||||
},
|
},
|
||||||
pb.ConfState{
|
pb.ConfState{
|
||||||
@ -282,7 +283,9 @@ func TestRawNodeProposeAndConfChange(t *testing.T) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Check that the last index is exactly the conf change we put in,
|
// Check that the last index is exactly the conf change we put in,
|
||||||
// down to the bits.
|
// down to the bits. Note that this comes from the Storage, which
|
||||||
|
// will not reflect any unstable entries that we'll only be presented
|
||||||
|
// with in the next Ready.
|
||||||
lastIndex, err = s.LastIndex()
|
lastIndex, err = s.LastIndex()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
@ -313,7 +316,17 @@ func TestRawNodeProposeAndConfChange(t *testing.T) {
|
|||||||
t.Fatalf("exp:\n%+v\nact:\n%+v", exp, cs)
|
t.Fatalf("exp:\n%+v\nact:\n%+v", exp, cs)
|
||||||
}
|
}
|
||||||
|
|
||||||
if exp, act := lastIndex, rawNode.raft.pendingConfIndex; exp != act {
|
var maybePlusOne uint64
|
||||||
|
if autoLeave, ok := tc.cc.AsV2().EnterJoint(); ok && autoLeave {
|
||||||
|
// If this is an auto-leaving joint conf change, it will have
|
||||||
|
// appended the entry that auto-leaves, so add one to the last
|
||||||
|
// index that forms the basis of our expectations on
|
||||||
|
// pendingConfIndex. (Recall that lastIndex was taken from stable
|
||||||
|
// storage, but this auto-leaving entry isn't on stable storage
|
||||||
|
// yet).
|
||||||
|
maybePlusOne = 1
|
||||||
|
}
|
||||||
|
if exp, act := lastIndex+maybePlusOne, rawNode.raft.pendingConfIndex; exp != act {
|
||||||
t.Fatalf("pendingConfIndex: expected %d, got %d", exp, act)
|
t.Fatalf("pendingConfIndex: expected %d, got %d", exp, act)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
211
raft/testdata/confchange_v2_add_double_auto.txt
vendored
211
raft/testdata/confchange_v2_add_double_auto.txt
vendored
@ -195,3 +195,214 @@ stabilize 1 3
|
|||||||
stabilize
|
stabilize
|
||||||
----
|
----
|
||||||
ok
|
ok
|
||||||
|
|
||||||
|
# Now remove two nodes. What's new here is that the leader will actually have
|
||||||
|
# to go to a quorum to commit the transition into the joint config.
|
||||||
|
|
||||||
|
propose-conf-change 1
|
||||||
|
r2 r3
|
||||||
|
----
|
||||||
|
ok
|
||||||
|
|
||||||
|
# n1 sends out MsgApps.
|
||||||
|
stabilize 1
|
||||||
|
----
|
||||||
|
> 1 handling Ready
|
||||||
|
Ready MustSync=true:
|
||||||
|
Entries:
|
||||||
|
1/6 EntryConfChangeV2 r2 r3
|
||||||
|
Messages:
|
||||||
|
1->2 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryConfChangeV2 r2 r3]
|
||||||
|
1->3 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryConfChangeV2 r2 r3]
|
||||||
|
|
||||||
|
# n2, n3 ack them.
|
||||||
|
stabilize 2 3
|
||||||
|
----
|
||||||
|
> 2 receiving messages
|
||||||
|
1->2 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryConfChangeV2 r2 r3]
|
||||||
|
> 3 receiving messages
|
||||||
|
1->3 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryConfChangeV2 r2 r3]
|
||||||
|
> 2 handling Ready
|
||||||
|
Ready MustSync=true:
|
||||||
|
Entries:
|
||||||
|
1/6 EntryConfChangeV2 r2 r3
|
||||||
|
Messages:
|
||||||
|
2->1 MsgAppResp Term:1 Log:0/6
|
||||||
|
> 3 handling Ready
|
||||||
|
Ready MustSync=true:
|
||||||
|
Entries:
|
||||||
|
1/6 EntryConfChangeV2 r2 r3
|
||||||
|
Messages:
|
||||||
|
3->1 MsgAppResp Term:1 Log:0/6
|
||||||
|
|
||||||
|
# n1 gets some more proposals. This is part of a regression test: There used to
|
||||||
|
# be a bug in which these proposals would prompt the leader to transition out of
|
||||||
|
# the same joint state multiple times, which would cause a panic.
|
||||||
|
propose 1 foo
|
||||||
|
----
|
||||||
|
ok
|
||||||
|
|
||||||
|
propose 1 bar
|
||||||
|
----
|
||||||
|
ok
|
||||||
|
|
||||||
|
# n1 switches to the joint config, then initiates a transition into the final
|
||||||
|
# config.
|
||||||
|
stabilize 1
|
||||||
|
----
|
||||||
|
> 1 handling Ready
|
||||||
|
Ready MustSync=true:
|
||||||
|
Entries:
|
||||||
|
1/7 EntryNormal "foo"
|
||||||
|
1/8 EntryNormal "bar"
|
||||||
|
Messages:
|
||||||
|
1->2 MsgApp Term:1 Log:1/6 Commit:5 Entries:[1/7 EntryNormal "foo"]
|
||||||
|
1->3 MsgApp Term:1 Log:1/6 Commit:5 Entries:[1/7 EntryNormal "foo"]
|
||||||
|
1->2 MsgApp Term:1 Log:1/7 Commit:5 Entries:[1/8 EntryNormal "bar"]
|
||||||
|
1->3 MsgApp Term:1 Log:1/7 Commit:5 Entries:[1/8 EntryNormal "bar"]
|
||||||
|
> 1 receiving messages
|
||||||
|
2->1 MsgAppResp Term:1 Log:0/6
|
||||||
|
3->1 MsgAppResp Term:1 Log:0/6
|
||||||
|
> 1 handling Ready
|
||||||
|
Ready MustSync=false:
|
||||||
|
HardState Term:1 Vote:1 Commit:6
|
||||||
|
CommittedEntries:
|
||||||
|
1/6 EntryConfChangeV2 r2 r3
|
||||||
|
Messages:
|
||||||
|
1->2 MsgApp Term:1 Log:1/8 Commit:6
|
||||||
|
1->3 MsgApp Term:1 Log:1/8 Commit:6
|
||||||
|
INFO 1 switched to configuration voters=(1)&&(1 2 3) autoleave
|
||||||
|
INFO initiating automatic transition out of joint configuration voters=(1)&&(1 2 3) autoleave
|
||||||
|
> 1 handling Ready
|
||||||
|
Ready MustSync=true:
|
||||||
|
Entries:
|
||||||
|
1/9 EntryConfChangeV2
|
||||||
|
|
||||||
|
# n2 and n3 also switch to the joint config, and ack the transition out of it.
|
||||||
|
stabilize 2 3
|
||||||
|
----
|
||||||
|
> 2 receiving messages
|
||||||
|
1->2 MsgApp Term:1 Log:1/6 Commit:5 Entries:[1/7 EntryNormal "foo"]
|
||||||
|
1->2 MsgApp Term:1 Log:1/7 Commit:5 Entries:[1/8 EntryNormal "bar"]
|
||||||
|
1->2 MsgApp Term:1 Log:1/8 Commit:6
|
||||||
|
> 3 receiving messages
|
||||||
|
1->3 MsgApp Term:1 Log:1/6 Commit:5 Entries:[1/7 EntryNormal "foo"]
|
||||||
|
1->3 MsgApp Term:1 Log:1/7 Commit:5 Entries:[1/8 EntryNormal "bar"]
|
||||||
|
1->3 MsgApp Term:1 Log:1/8 Commit:6
|
||||||
|
> 2 handling Ready
|
||||||
|
Ready MustSync=true:
|
||||||
|
HardState Term:1 Commit:6
|
||||||
|
Entries:
|
||||||
|
1/7 EntryNormal "foo"
|
||||||
|
1/8 EntryNormal "bar"
|
||||||
|
CommittedEntries:
|
||||||
|
1/6 EntryConfChangeV2 r2 r3
|
||||||
|
Messages:
|
||||||
|
2->1 MsgAppResp Term:1 Log:0/7
|
||||||
|
2->1 MsgAppResp Term:1 Log:0/8
|
||||||
|
2->1 MsgAppResp Term:1 Log:0/8
|
||||||
|
INFO 2 switched to configuration voters=(1)&&(1 2 3) autoleave
|
||||||
|
> 3 handling Ready
|
||||||
|
Ready MustSync=true:
|
||||||
|
HardState Term:1 Commit:6
|
||||||
|
Entries:
|
||||||
|
1/7 EntryNormal "foo"
|
||||||
|
1/8 EntryNormal "bar"
|
||||||
|
CommittedEntries:
|
||||||
|
1/6 EntryConfChangeV2 r2 r3
|
||||||
|
Messages:
|
||||||
|
3->1 MsgAppResp Term:1 Log:0/7
|
||||||
|
3->1 MsgAppResp Term:1 Log:0/8
|
||||||
|
3->1 MsgAppResp Term:1 Log:0/8
|
||||||
|
INFO 3 switched to configuration voters=(1)&&(1 2 3) autoleave
|
||||||
|
|
||||||
|
# n2 and n3 also leave the joint config and the dust settles. We see at the very
|
||||||
|
# end that n1 receives some messages from them that it refuses because it does
|
||||||
|
# not have them in its config any more.
|
||||||
|
stabilize
|
||||||
|
----
|
||||||
|
> 1 receiving messages
|
||||||
|
2->1 MsgAppResp Term:1 Log:0/7
|
||||||
|
2->1 MsgAppResp Term:1 Log:0/8
|
||||||
|
2->1 MsgAppResp Term:1 Log:0/8
|
||||||
|
3->1 MsgAppResp Term:1 Log:0/7
|
||||||
|
3->1 MsgAppResp Term:1 Log:0/8
|
||||||
|
3->1 MsgAppResp Term:1 Log:0/8
|
||||||
|
> 1 handling Ready
|
||||||
|
Ready MustSync=false:
|
||||||
|
HardState Term:1 Vote:1 Commit:8
|
||||||
|
CommittedEntries:
|
||||||
|
1/7 EntryNormal "foo"
|
||||||
|
1/8 EntryNormal "bar"
|
||||||
|
Messages:
|
||||||
|
1->2 MsgApp Term:1 Log:1/8 Commit:7 Entries:[1/9 EntryConfChangeV2]
|
||||||
|
1->3 MsgApp Term:1 Log:1/8 Commit:7 Entries:[1/9 EntryConfChangeV2]
|
||||||
|
1->2 MsgApp Term:1 Log:1/9 Commit:8
|
||||||
|
1->3 MsgApp Term:1 Log:1/9 Commit:8
|
||||||
|
> 2 receiving messages
|
||||||
|
1->2 MsgApp Term:1 Log:1/8 Commit:7 Entries:[1/9 EntryConfChangeV2]
|
||||||
|
1->2 MsgApp Term:1 Log:1/9 Commit:8
|
||||||
|
> 3 receiving messages
|
||||||
|
1->3 MsgApp Term:1 Log:1/8 Commit:7 Entries:[1/9 EntryConfChangeV2]
|
||||||
|
1->3 MsgApp Term:1 Log:1/9 Commit:8
|
||||||
|
> 2 handling Ready
|
||||||
|
Ready MustSync=true:
|
||||||
|
HardState Term:1 Commit:8
|
||||||
|
Entries:
|
||||||
|
1/9 EntryConfChangeV2
|
||||||
|
CommittedEntries:
|
||||||
|
1/7 EntryNormal "foo"
|
||||||
|
1/8 EntryNormal "bar"
|
||||||
|
Messages:
|
||||||
|
2->1 MsgAppResp Term:1 Log:0/9
|
||||||
|
2->1 MsgAppResp Term:1 Log:0/9
|
||||||
|
> 3 handling Ready
|
||||||
|
Ready MustSync=true:
|
||||||
|
HardState Term:1 Commit:8
|
||||||
|
Entries:
|
||||||
|
1/9 EntryConfChangeV2
|
||||||
|
CommittedEntries:
|
||||||
|
1/7 EntryNormal "foo"
|
||||||
|
1/8 EntryNormal "bar"
|
||||||
|
Messages:
|
||||||
|
3->1 MsgAppResp Term:1 Log:0/9
|
||||||
|
3->1 MsgAppResp Term:1 Log:0/9
|
||||||
|
> 1 receiving messages
|
||||||
|
2->1 MsgAppResp Term:1 Log:0/9
|
||||||
|
2->1 MsgAppResp Term:1 Log:0/9
|
||||||
|
3->1 MsgAppResp Term:1 Log:0/9
|
||||||
|
3->1 MsgAppResp Term:1 Log:0/9
|
||||||
|
> 1 handling Ready
|
||||||
|
Ready MustSync=false:
|
||||||
|
HardState Term:1 Vote:1 Commit:9
|
||||||
|
CommittedEntries:
|
||||||
|
1/9 EntryConfChangeV2
|
||||||
|
Messages:
|
||||||
|
1->2 MsgApp Term:1 Log:1/9 Commit:9
|
||||||
|
1->3 MsgApp Term:1 Log:1/9 Commit:9
|
||||||
|
INFO 1 switched to configuration voters=(1)
|
||||||
|
> 2 receiving messages
|
||||||
|
1->2 MsgApp Term:1 Log:1/9 Commit:9
|
||||||
|
> 3 receiving messages
|
||||||
|
1->3 MsgApp Term:1 Log:1/9 Commit:9
|
||||||
|
> 2 handling Ready
|
||||||
|
Ready MustSync=false:
|
||||||
|
HardState Term:1 Commit:9
|
||||||
|
CommittedEntries:
|
||||||
|
1/9 EntryConfChangeV2
|
||||||
|
Messages:
|
||||||
|
2->1 MsgAppResp Term:1 Log:0/9
|
||||||
|
INFO 2 switched to configuration voters=(1)
|
||||||
|
> 3 handling Ready
|
||||||
|
Ready MustSync=false:
|
||||||
|
HardState Term:1 Commit:9
|
||||||
|
CommittedEntries:
|
||||||
|
1/9 EntryConfChangeV2
|
||||||
|
Messages:
|
||||||
|
3->1 MsgAppResp Term:1 Log:0/9
|
||||||
|
INFO 3 switched to configuration voters=(1)
|
||||||
|
> 1 receiving messages
|
||||||
|
2->1 MsgAppResp Term:1 Log:0/9
|
||||||
|
raft: cannot step as peer not found
|
||||||
|
3->1 MsgAppResp Term:1 Log:0/9
|
||||||
|
raft: cannot step as peer not found
|
||||||
|
100
raft/testdata/confchange_v2_add_single_auto.txt
vendored
100
raft/testdata/confchange_v2_add_single_auto.txt
vendored
@ -96,103 +96,3 @@ stabilize
|
|||||||
2->1 MsgAppResp Term:1 Log:0/4
|
2->1 MsgAppResp Term:1 Log:0/4
|
||||||
> 1 receiving messages
|
> 1 receiving messages
|
||||||
2->1 MsgAppResp Term:1 Log:0/4
|
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
|
|
||||||
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
|
|
||||||
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
|
|
||||||
> 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
|
|
||||||
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
|
|
||||||
INFO 2 switched to configuration voters=(1 2 3 4 5)&&(1 2) autoleave
|
|
||||||
> 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
|
|
||||||
|
Loading…
x
Reference in New Issue
Block a user