Commit Graph

268 Commits (0461b3fa51ebcd3f72ef54296638ac2f19ea9071)

Author SHA1 Message Date
Tobias Schottdorf 37c7e4d1d8 raft: fix auto-transitioning out of joint config
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.
2020-02-25 12:35:51 +01:00
qupeng eaa0612e02 raft: abort leader transferring if the target is demoted (#11417)
Signed-off-by: qupeng <qupeng@pingcap.com>
2019-12-20 12:07:52 +08:00
Tobias Schottdorf c30c2e345b raft: let learners vote
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.
2019-08-07 12:03:18 +02:00
Tobias Schottdorf b9c051e7a7 raftpb: clean up naming in ConfChange 2019-07-23 10:40:03 +02:00
Tobias Schottdorf b67303c6a2 raft: allow use of joint quorums
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.
2019-07-23 10:40:03 +02:00
Tobias Schottdorf c9491d7861 raft: clean up bootstrap
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.
2019-07-19 10:02:02 +02:00
Tobias Schottdorf c62b7048b5 raft: use RawNode for node's event loop
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.
2019-07-19 09:59:59 +02:00
Tobias Schottdorf aa158f36b9 raft: internally support joint consensus
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.
2019-07-16 15:36:04 +02:00
Tobias Schottdorf 6f009d211f raft: allow voter to become learner through snapshot
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.
2019-07-08 09:32:24 +02:00
Tobias Schottdorf b171e1c78b raft: centralize configuration change application
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.
2019-07-03 21:26:42 +02:00
Tobias Schottdorf f9c2d00fb3 raft: extract 'tracker' package
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.
2019-06-21 22:15:00 +02:00
Tobias Schottdorf e039629907 raft: use half-populated joint quorum
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.
2019-06-19 14:19:35 +02:00
Tobias Schottdorf 3def2364e4 raft: use membership sets in progress tracking
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.
2019-06-19 14:19:35 +02:00
Tobias Schottdorf c844526002 raft: prevent learners from becoming leader
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.
2019-06-17 09:20:45 +02:00
Gyuho Lee 34bd797e67 *: revert module import paths
Signed-off-by: Gyuho Lee <leegyuho@amazon.com>
2019-05-28 15:39:35 -07:00
Tobias Schottdorf a11563737c raft: use progress tracker APIs in more places
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.
2019-05-21 16:02:52 +02:00
Tobias Schottdorf ea82b2b758 raft: move more methods onto the progress tracker
Continues what was initiated in the last commit.
2019-05-21 16:02:52 +02:00
Tobias Schottdorf dbac67e7a8 raft: extract progress tracking into own component
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.
2019-05-21 16:02:52 +02:00
shivaramr 9150bf52d6 go modules: Fix module path version to include version number 2019-04-26 15:29:50 -07:00
Jingyi Hu 5088d70d69 raft: leader response to learner MsgReadIndex
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.
2019-03-28 16:14:32 -07:00
shawnli 23731bf9ba raft: set lead to none when becomePreCandidate 2018-12-28 19:57:26 +08:00
Tobias Schottdorf 5c209d66d2 raft: ensure leader is in ProgressStateReplicate
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).
2018-11-23 17:57:36 +01:00
Shin'ya Ueoka aa4313a55a *: fix github links 2018-11-10 11:14:18 +09:00
Gyuho Lee b7ed4165ea raft: fix godoc in tests
Signed-off-by: Gyuho Lee <leegyuho@amazon.com>
2018-10-24 23:23:32 -07:00
Gyuho Lee 965ba5ca8b
Merge pull request #10203 from ping40/doc1022_2
raft: fix description in UT
2018-10-24 23:21:02 -07:00
Tobias Schottdorf ad49c8fd98 raft: fix bug in unbounded log growth prevention mechanism
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
2018-10-22 11:28:39 +02:00
ping40 de470991e1 raft: fix description in UT 2018-10-22 13:59:50 +08:00
Nathan VanBenschoten f89b06dc6d raft: provide protection against unbounded Raft log growth
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
2018-10-13 23:25:05 -04:00
Gyuho Lee bb60f8ab1d raft: change import paths to "go.etcd.io/etcd"
Signed-off-by: Gyuho Lee <leegyuho@amazon.com>
2018-08-28 17:47:52 -07:00
Xiang Li 11dd0b583b
Merge pull request #9982 from bdarnell/pagination
raft: Introduce CommittedEntries pagination
2018-08-11 09:12:46 +08:00
Ben Darnell a9e7c1e11f raft: Make flow control more aggressive
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
2018-08-08 11:10:54 -04:00
Ben Darnell bc14deecca raft: Add a test for MaxSizePerMsg feature
Ensure that this limit is respected when generating MsgApp messages.
2018-08-06 16:52:16 -04:00
Gyuho Lee 7aaaa0d82f raft: do not use underscore in var name
Signed-off-by: Gyuho Lee <gyuhox@gmail.com>
2018-07-05 10:25:47 -07:00
Gyuho Lee 0249c39cb3 raft: remove unnecessary type conversion
Signed-off-by: Gyuho Lee <gyuhox@gmail.com>
2018-07-05 10:12:45 -07:00
Xiang Li 20cf7f4d5b
Merge pull request #9671 from lorneli/raft-test
raft: merge test cases of pre-candidate with the normal one
2018-05-24 08:27:07 -07:00
Gyuho Lee e7adfb0ebf raft: use different parameters for tests
Signed-off-by: Gyuho Lee <gyuhox@gmail.com>
2018-05-09 15:42:45 -07:00
lorneli 3d12e36c7e raft: merge test cases of pre-candidate with the normal one
So result checking just compares the expected with output and
becomes more readable.
2018-05-01 17:08:37 +08:00
Gyuho Lee 8aae8c1c9c raft: document disruptive rejoining server, add tests
Signed-off-by: Gyuho Lee <gyuhox@gmail.com>
2018-03-06 09:54:29 -08:00
Gyuho Lee d808b4686c raft: fix typo in raft_test.go
Signed-off-by: Gyuho Lee <gyuhox@gmail.com>
2018-02-26 10:03:25 -08:00
Gyuho Lee 38846c220a raft: use leader's term when candidate becomes follower
`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>
2018-02-21 16:10:52 -08:00
Xiang Li d54f281b26
Merge pull request #8525 from shuaili87/pre-vote-compatible
raft: fix deadlock during PreVote migration process
2018-01-26 16:34:59 -08:00
Xiang Li c5532ebbf6
Merge pull request #9067 from absolute8511/optimize-raft-drop
raft: let raft step return error when proposal is dropped to allow fail-fast
2018-01-11 19:54:52 -08:00
Vincent Lee 30ced5b2be raft: let raft step return error when proposal is dropped to allow fail-fast. 2018-01-12 10:16:47 +08:00
Vincent Lee 11fa4f0275 raft: raft learners should be returned after applyConfChange 2018-01-11 17:30:17 +08:00
Xiang Li ed1ff9e952
Merge pull request #9073 from bdarnell/pending-conf-index
raft: Avoid scanning raft log in becomeLeader
2018-01-08 16:37:36 -08:00
Nathan VanBenschoten e6dc57f708 raft: s/leaner/learner/g 2018-01-03 08:16:50 -05:00
Ben Darnell 8d8f3195e4 raft: Avoid scanning raft log in becomeLeader
Scanning the uncommitted portion of the raft log to determine whether
there are any pending config changes can be expensive. In
cockroachdb/cockroach#18601, we've seen that a new leader can spend so
much time scanning its log post-election that it fails to send
its first heartbeats in time to prevent a second election from
starting immediately.

Instead of tracking whether a pending config change exists with a
boolean, this commit tracks the latest log index at which a pending
config change *could* exist. This is a less expensive solution to
the problem, and the impact of false positives should be minimal since
a newly-elected leader should be able to quickly commit the tail of
its log.
2017-12-30 10:13:36 -05:00
siddontang c6f2db2e92 raft: support learner 2017-11-11 10:38:21 +08:00
Xiang 9801fd7297 raft: ensure CheckQuorum is enabled when readonlyoption is lease based 2017-09-17 10:46:12 -07:00
gladiator 58b98c6a14 raft: check leader request when becomeFollower 2017-09-15 08:23:18 +08:00