Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add Config.DisableConfChangeValidation #81

Merged
merged 2 commits into from
Jul 17, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
66 changes: 46 additions & 20 deletions raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -254,6 +254,28 @@ type Config struct {
// logical clock from assigning the timestamp and then forwarding the data
// to the leader.
DisableProposalForwarding bool

// DisableConfChangeValidation turns off propose-time verification of
// configuration changes against the currently active configuration of the
// raft instance. These checks are generally sensible (cannot leave a joint
// config unless in a joint config, et cetera) but they have false positives
// because the active configuration may not be the most recent
// configuration. This is because configurations are activated during log
// application, and even the leader can trail log application by an
// unbounded number of entries.
tbg marked this conversation as resolved.
Show resolved Hide resolved
// Symmetrically, the mechanism has false negatives - because the check may
// not run against the "actual" config that will be the predecessor of the
// newly proposed one, the check may pass but the new config may be invalid
// when it is being applied. In other words, the checks are best-effort.
//
// Users should *not* use this option unless they have a reliable mechanism
// (above raft) that serializes and verifies configuration changes. If an
// invalid configuration change enters the log and gets applied, a panic
// will result.
//
// This option may be removed once false positives are no longer possible.
// See: https://github.com/etcd-io/raft/issues/80
DisableConfChangeValidation bool
tbg marked this conversation as resolved.
Show resolved Hide resolved
}

func (c *Config) validate() error {
Expand Down Expand Up @@ -356,6 +378,9 @@ type raft struct {
// be proposed if the leader's applied index is greater than this
// value.
pendingConfIndex uint64
// disableConfChangeValidation is Config.DisableConfChangeValidation,
// see there for details.
disableConfChangeValidation bool
// an estimate of the size of the uncommitted tail of the Raft log. Used to
// prevent unbounded log growth. Only maintained by the leader. Reset on
// term changes.
Expand Down Expand Up @@ -407,20 +432,21 @@ func newRaft(c *Config) *raft {
}

r := &raft{
id: c.ID,
lead: None,
isLearner: false,
raftLog: raftlog,
maxMsgSize: entryEncodingSize(c.MaxSizePerMsg),
maxUncommittedSize: entryPayloadSize(c.MaxUncommittedEntriesSize),
prs: tracker.MakeProgressTracker(c.MaxInflightMsgs, c.MaxInflightBytes),
electionTimeout: c.ElectionTick,
heartbeatTimeout: c.HeartbeatTick,
logger: c.Logger,
checkQuorum: c.CheckQuorum,
preVote: c.PreVote,
readOnly: newReadOnly(c.ReadOnlyOption),
disableProposalForwarding: c.DisableProposalForwarding,
id: c.ID,
lead: None,
isLearner: false,
raftLog: raftlog,
maxMsgSize: entryEncodingSize(c.MaxSizePerMsg),
maxUncommittedSize: entryPayloadSize(c.MaxUncommittedEntriesSize),
prs: tracker.MakeProgressTracker(c.MaxInflightMsgs, c.MaxInflightBytes),
electionTimeout: c.ElectionTick,
heartbeatTimeout: c.HeartbeatTick,
logger: c.Logger,
checkQuorum: c.CheckQuorum,
preVote: c.PreVote,
readOnly: newReadOnly(c.ReadOnlyOption),
disableProposalForwarding: c.DisableProposalForwarding,
disableConfChangeValidation: c.DisableConfChangeValidation,
}

cfg, prs, err := confchange.Restore(confchange.Changer{
Expand Down Expand Up @@ -1242,17 +1268,17 @@ func stepLeader(r *raft, m pb.Message) error {
alreadyJoint := len(r.prs.Config.Voters[1]) > 0
wantsLeaveJoint := len(cc.AsV2().Changes) == 0

var refused string
var failedCheck string
if alreadyPending {
refused = fmt.Sprintf("possible unapplied conf change at index %d (applied to %d)", r.pendingConfIndex, r.raftLog.applied)
failedCheck = 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"
failedCheck = "must transition out of joint config first"
} else if !alreadyJoint && wantsLeaveJoint {
refused = "not in joint state; refusing empty conf change"
failedCheck = "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)
if failedCheck != "" && !r.disableConfChangeValidation {
r.logger.Infof("%x ignoring conf change %v at config %s: %s", r.id, cc, r.prs.Config, failedCheck)
m.Entries[i] = pb.Entry{Type: pb.EntryNormal}
} else {
r.pendingConfIndex = r.raftLog.lastIndex() + uint64(i) + 1
Expand Down
4 changes: 4 additions & 0 deletions rafttest/interaction_env_handler_add_nodes.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,10 @@ func (env *InteractionEnv) handleAddNodes(t *testing.T, d datadriven.TestData) e
arg.Scan(t, i, &cfg.PreVote)
case "checkquorum":
arg.Scan(t, i, &cfg.CheckQuorum)
case "max-committed-size-per-ready":
arg.Scan(t, i, &cfg.MaxCommittedSizePerReady)
case "disable-conf-change-validation":
arg.Scan(t, i, &cfg.DisableConfChangeValidation)
case "read-only":
switch arg.Vals[i] {
case "safe":
Expand Down
76 changes: 76 additions & 0 deletions testdata/confchange_disable_validation.txt
Original file line number Diff line number Diff line change
@@ -0,0 +1,76 @@
# This test verifies the DisableConfChangeValidation setting.
# With it set, raft should allow configuration changes to enter the log even
# if they appear to be incompatible with the currently active configuration.
#
# The test sets up a single-voter group that applies entries one at a time.
# Then it proposes a bogus entry followed by a conf change. When the bogus entry
# has applied, a second (compatible, but the node doesn't know this yet)
# configuration change is proposed. That configuration change is accepted into
# the log since due to DisableConfChangeValidation=true.
add-nodes 1 voters=(1) index=2 max-committed-size-per-ready=1 disable-conf-change-validation=true
----
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

stabilize log-level=none
----
ok

# Dummy entry.
propose 1 foo
----
ok

propose-conf-change 1 transition=explicit
l2 l3
----
ok

# Entries both get appended.
process-ready 1
----
Ready MustSync=true:
Entries:
1/4 EntryNormal "foo"
1/5 EntryConfChangeV2 l2 l3

# Dummy entry comes up for application.
process-ready 1
----
Ready MustSync=false:
HardState Term:1 Vote:1 Commit:5
CommittedEntries:
1/4 EntryNormal "foo"

# Propose new config change. Note how it isn't rejected,
# which is due to DisableConfChangeValidation=true.
propose-conf-change 1
----
ok

# Turn on autopilot: the first config change applies, the
# second one gets committed and also applies.
stabilize
----
> 1 handling Ready
Ready MustSync=true:
Entries:
1/6 EntryConfChangeV2
CommittedEntries:
1/5 EntryConfChangeV2 l2 l3
INFO 1 switched to configuration voters=(1)&&(1) learners=(2 3)
> 1 handling Ready
Ready MustSync=false:
HardState Term:1 Vote:1 Commit:6
CommittedEntries:
1/6 EntryConfChangeV2
Messages:
1->2 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryConfChangeV2]
1->3 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryConfChangeV2]
INFO 1 switched to configuration voters=(1) learners=(2 3)