-
Notifications
You must be signed in to change notification settings - Fork 9.8k
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
raft: internally support joint consensus #10889
Conversation
20c32ee
to
34baaed
Compare
// probed without an index first. | ||
// | ||
// TODO(tbg): verify that, this is just my best guess. | ||
Next: c.LastIndex + 1, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do you need to verify this before the PR goes in?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No, this is preserving the existing behavior that I don't want to block the PR on. But @xiang90 could just give us the answer 😄
34baaed
to
d7c29df
Compare
3943875
to
6600ab4
Compare
b427b94
to
e4ad654
Compare
I'm still interested in making the use of |
c560e1c
to
0f57336
Compare
This commit introduces machinery to safely apply joint consensus configuration changes to Raft. The main contribution is the new package, `confchange`, which offers the primitives `Simple`, `EnterJoint`, and `LeaveJoint`. The first two take a list of configuration changes. `Simple` only declares success if these configuration changes (applied atomically) change the set of voters by at most one (i.e. it's fine to add or remove any number of learners, but change only one voter). `EnterJoint` makes the configuration joint and then applies the changes to it, in preparation of the caller returning later and transitioning out of the joint config into the final desired configuration via `LeaveJoint()`. This commit streamlines the conversion between voters and learners, which is now generally allowed whenever the above conditions are upheld (i.e. it's not possible to demote a voter and add a new voter in the context of a Simple configuration change, but it is possible via EnterJoint). Previously, we had the artificial restriction that a voter could not be demoted to a learner, but had to be removed first. Even though demoting a learner is generally less useful than promoting a learner (the latter is used to catch up future voters), demotions could see use in improved handling of temporary node unavailability, where it is desired to remove voting power from a down node, but to preserve its data should it return. An additional change that was made in this commit is to prevent the use of empty commit quorums, which was previously possible but for no good reason; this: Closes etcd-io#10884. The work left to do in a future PR is to actually expose joint configurations to the applications using Raft. This will entail mostly API design and the addition of suitable testing, which to be carried out ergonomically is likely to motivate a larger refactor. Touches etcd-io#7625.
0f57336
to
aa158f3
Compare
Hi @tbg, just FYI, due to the upcoming etcd 3.4 release, we temporarily set the code freeze date to be 7/31. I expect it to last couple days (maybe a week). @gyuho and @hexfusion, is my estimation reasonable? |
@jingyih thanks for the heads up, that should be alright. Just to make sure we respect the freeze (i.e. don't accidentally click the button), are merges to master going to be blocked (I'd suggest that)? |
Yes I assume merges to master will be blocked during the freeze period. |
It created a sequence of conf changes that could intermittently cause an empty set of voters, which Raft asserts against as of etcd-io#10889. This fixes TestCtlV2BackupSnapshot and TestCtlV2BackupV3Snapshot, see: etcd-io#10700 (comment)
It created a sequence of conf changes that could intermittently cause an empty set of voters, which Raft asserts against as of etcd-io#10889. This fixes TestCtlV2BackupSnapshot and TestCtlV2BackupV3Snapshot, see: etcd-io#10700 (comment)
It created a sequence of conf changes that could intermittently cause an empty set of voters, which Raft asserts against as of etcd-io#10889. This fixes TestCtlV2BackupSnapshot and TestCtlV2BackupV3Snapshot, see: etcd-io#10700 (comment)
It created a sequence of conf changes that could intermittently cause an empty set of voters, which Raft asserts against as of etcd-io#10889. This fixes TestCtlV2BackupSnapshot and TestCtlV2BackupV3Snapshot, see: etcd-io#10700 (comment)
This commit introduces machinery to safely apply joint consensus
configuration changes to Raft.
The main contribution is the new package,
confchange
, which offersthe primitives
Simple
,EnterJoint
, andLeaveJoint
.The first two take a list of configuration changes.
Simple
onlydeclares success if these configuration changes (applied atomically)
change the set of voters by at most one (i.e. it's fine to add or
remove any number of learners, but change only one voter).
EnterJoint
makes the configuration joint and then applies the changes to it, in
preparation of the caller returning later and transitioning out of the
joint config into the final desired configuration via
LeaveJoint()
.This commit streamlines the conversion between voters and learners, which
is now generally allowed whenever the above conditions are upheld (i.e.
it's not possible to demote a voter and add a new voter in the context
of a Simple configuration change, but it is possible via EnterJoint).
Previously, we had the artificial restriction that a voter could not be
demoted to a learner, but had to be removed first.
Even though demoting a learner is generally less useful than promoting
a learner (the latter is used to catch up future voters), demotions
could see use in improved handling of temporary node unavailability,
where it is desired to remove voting power from a down node, but to
preserve its data should it return.
An additional change that was made in this commit is to prevent the use
of empty commit quorums, which was previously possible but for no good
reason; this:
Closes #10884.
The work left to do in a future PR is to actually expose joint
configurations to the applications using Raft. This will entail mostly
API design and the addition of suitable testing, which to be carried
out ergonomically is likely to motivate a larger refactor.
Touches #7625.