* raft: check conf change before campaign
Signed-off-by: Jay Lee <BusyJayLee@gmail.com>
* raft: extract hup function
Signed-off-by: Jay Lee <BusyJayLee@gmail.com>
* raft: check pending conf change for transferleader
Signed-off-by: Jay Lee <BusyJayLee@gmail.com>
This change makes the etcd package compatible with the existing Go
ecosystem for module versioning.
Used this tool to update package imports:
https://github.com/KSubedi/gomove
The code doing so was undertested and buggy: it would launch multiple
attempts to transition out when the conf change was not the last element
in the log.
This commit fixes the problem and adds a regression test. It also
reworks the code to handle a former untested edge case, in which the
auto-transition append is refused. This can't happen any more with the
current version of the code because this proposal has size zero and is
special cased in increaseUncommittedSize. Last but not least, the
auto-leave proposal now also bumps pendingConfIndex, which was not done
previously due to an oversight.
It turns out that that learners must be allowed to cast votes.
This seems counter- intuitive but is necessary in the situation in which
a learner has been promoted (i.e. is now a voter) but has not learned
about this yet.
For example, consider a group in which id=1 is a learner and id=2 and
id=3 are voters. A configuration change promoting 1 can be committed on
the quorum `{2,3}` without the config change being appended to the
learner's log. If the leader (say 2) fails, there are de facto two
voters remaining. Only 3 can win an election (due to its log containing
all committed entries), but to do so it will need 1 to vote. But 1
considers itself a learner and will continue to do so until 3 has
stepped up as leader, replicates the conf change to 1, and 1 applies it.
Ultimately, by receiving a request to vote, the learner realizes that
the candidate believes it to be a voter, and that it should act
accordingly. The candidate's config may be stale, too; but in that case
it won't win the election, at least in the absence of the bug discussed
in:
https://github.com/etcd-io/etcd/issues/7625#issuecomment-488798263.
This change introduces joint quorums by changing the Node and RawNode
API to accept pb.ConfChangeV2 (on top of pb.ConfChange).
pb.ConfChange continues to work as today: it allows carrying out a
single configuration change. A pb.ConfChange proposal gets added to
the Raft log as such and is thus also observed by the app during Ready
handling, and fed back to ApplyConfChange.
ConfChangeV2 allows joint configuration changes but will continue to
carry out configuration changes in "one phase" (i.e. without ever
entering a joint config) when this is possible.
This is the first (maybe not last) step in cleaning up the bootstrap
code around StartNode.
Initializing a Raft group for the first time is awkward, since a
configuration has to be pulled from thin air. The way this is solved
today is unclean: The app is supposed to pass peers to StartNode(),
we add configuration changes for them to the log, immediately pretend
that they are applied, but actually leave them unapplied (to give the
app a chance to observe them, though if the app did decide to not apply
them things would really go off the rails), and then return control to
the app. The app will then process the initial Readys and as a result
the configuration will be persisted to disk; restarts of the node then
use RestartNode which doesn't take any peers.
The code that did this lived awkwardly in two places fairly deep down
the callstack, though it was really only necessary in StartNode(). This
commit refactors things to make this more obvious: only StartNode does
this dance now. In particular, RawNode does not support this at all any
more; it expects the app to set up its Storage correctly.
Future work may provide helpers to make this "preseeding" of the Storage
more user-friendly. It isn't entirely straightforward to do so since
the Storage interface doesn't provide the right accessors for this
purpose. Briefly speaking, we want to make sure that a non-bootstrapped
node can never catch up via the log so that we can implicitly use one
of the "skipped" log entries to represent the configuration change into
the bootstrap configuration. This is an invasive change that affects
all consumers of raft, and it is of lower urgency since the code (post
this commit) already encapsulates the complexity sufficiently.
It has always bugged me that any new feature essentially needed to be
tested twice due to the two ways in which apps can use raft (`*node` and
`*RawNode`). Due to upcoming testing work for joint consensus, now is a
good time to rectify this somewhat.
This commit removes most logic from `(*node).run` and uses `*RawNode`
internally. This simplifies the logic and also lead (via debugging) to
some insight on how the semantics of the approaches differ, which is now
documented in the comments.
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#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.
At the time of writing, we don't allow configuration changes to change
voters to learners directly, but note that a snapshot may compress
multiple changes to the configuration into one: the voter could have
been removed, then readded as a learner and the snapshot reflects both
changes. In that case, a voter receives a snapshot telling it that it is
now a learner. In fact, the node has to accept that snapshot, or it is
permanently cut off from the Raft log.
I think this just wasn't realized in the original work, but this is just
my guess since there generally is very little rationale on the various
decisions made. I also generally haven't been able to figure out whether
the decision to prevent voters from becoming learners without first
having been removed was motivated by some particular concern, or if it
just wasn't deemed necessary. I suspect it is the latter because
demoting a voter seems perfectly safe.
See https://github.com/etcd-io/etcd/pull/8751#issuecomment-342028091.
Put all the logic related to applying a configuration change in one
place in preparation for adding joint consensus.
This inspired various TODOs.
I had to rewrite TestSnapshotSucceedViaAppResp since it was relying
on a snapshot applied to the leader, which is now prevented.
Mechanically extract `progressTracker`, `Progress`, and `inflights`
to their own package named `tracker`. Add lots of comments in the
progress, and take the opportunity to rename and clarify various
fields.
To ease a future transition into joint quorums, this commit removes the
previous "ad-hoc" majority-based quorum and vote computations with that
introduced in the `raft/quorum` package.
More specifically, the progressTracker now uses a quorum.JointConfig for
which the "second" majority quorum is always empty; in this case the
quorum behaves like the one quorum.MajorityConfig that is actually
present. Or, more briefly, this change is a no-op, but it will take the
busywork out of actually starting to make use of joint quorums in the
future.
On a side node, I suspect that this might've fixed a bug regarding the
read index though I haven't been able to explicitly come up with a
counter-example. The problem was that the acks collected for the read
index weren't taking into account membership changes, so they'd run the
danger of using acks from nodes since removed to claim that a quorum of
acks had been received. There's a chance that there isn't a
counter-example (the only guarantee extracted from the "quorum" is that
there isn't another leader, but even if there's another leader all that
matters is that that leader doesn't have a divergent history from the
stale leader in the hypothetical counter-example), but either way there
is morally a bug here that is now fixed because VoteCommitted doesn't
care about votes from members that are not voters known to the currently
active configuration.
Instead of having disjoint mappings of ID to *Progress for voters and
learners, use a map[id]struct{} for each and share a map of *Progress
among them.
This is easier to handle when joint quorums are introduced, at which
point a node may be a voting member of two quorums.
We were already taking some precautions against learners campaigning,
but there was no safeguard against an explicit call to `Campaign()`.
The newly added test also verifies that leadership transfers to
learners are ignored.
This doesn't completely eliminate access to prs.nodes, but that's not
really necessary. This commit uses the existing APIs in a few more
places where it's convenient, and also sprinkles some assertions.
The Progress maps contain both the active configuration and information
about the replication status. By pulling it into its own component, this
becomes easier to unit test and also clarifies the code, which will see
changes as etcd-io/etcd#7625 is addressed.
More functionality will move into `prs` in self-contained follow-up commits.
Leader should check message sender after receiving MsgReadIndex, even
when raft quorum is 1. The message could be sent from learner node, and
leader should respond.
The leader perpetually kept itself in ProgressStateProbe even though of
course it has perfect knowledge of its log. This wasn't usually an issue
because it also doesn't care about its own Progress, but it's better to
keep this data correctly maintained, especially since this is part of
raft.Status and thus becomes visible to applications using the Raft
library.
(Concretely, in CockroachDB we use the Progress to inform log
truncations).
The previous code was using the proto-generated `Size()` method to
track the size of an incoming proposal at the leader. This includes
the Index and Term, which were mutated after the call to `Size()`
when appending to the log. Additionally, it was not taking into
account that an ignored configuration change would ignore the
original proposal and append an empty entry instead.
As a result, a fully committed Raft group could end up with a non-
zero tracked uncommitted Raft log counter that would eventually hit
the ceiling and drop all future proposals indiscriminately. It would
also immediately imply that proposals exceeding the threshold alone
would get refused (as the "first uncommitted proposal" gets special
treatment and is always allowed in).
Track only the size of the payload actually appended to the Raft log
instead.
For context, see:
https://github.com/cockroachdb/cockroach/issues/31618#issuecomment-431374938
The suggested pattern for Raft proposals is that they be retried
periodically until they succeed. This turns out to be an issue
when a leader cannot commit entries because the leader will continue
to append re-proposed entries to its log without committing anything.
This can result in the uncommitted tail of a leader's log growing
without bound until it is able to commit entries.
This change add a safeguard to protect against this case where a
leader's log can grow without bound during loss of quorum scenarios.
It does so by introducing a new, optional ``MaxUncommittedEntriesSize
configuration. This config limits the max aggregate size of uncommitted
entries that may be appended to a leader's log. Once this limit
is exceeded, proposals will begin to return ErrProposalDropped
errors.
See cockroachdb/cockroach#27772
We allow multiple in-flight append messages, but prior to this change
the only way we'd ever send them is if there is a steady stream of new
proposals. Catching up a follower that is far behind would be
unnecessarily slow (this is exacerbated by a quirk of CockroachDB's
use of raft which limits our ability to catch up via snapshot in some
cases).
See cockroachdb/cockroach#27983
`raft.Step` already ensures that when `m.Term > r.Term`,
candidate reverts back to follower with its term being
reset with `m.Term`, thus it's always true that
`m.Term == r.Term` in `stepCandidate`.
This just makes `r.becomeFollower` calls consistent.
Signed-off-by: Gyuho Lee <gyuhox@gmail.com>