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.
`raftpb.Entry.String` takes a pointer receiver, so calling it
on a loop variable was causing the variable to escape. Removing
the `.String()` call was enough to avoid the allocation, but
this also avoids a memory copy and prevents similar bugs.
This was responsible for 11.63% of total allocations in an
experiment with https://github.com/nvanbenschoten/raft-toy.
By boxing a heap-allocated slice header instead of the slice
header on the stack, we can avoid an allocation when passing
through the sort.Interface interface.
This was responsible for 26.61% of total allocations in an
experiment with https://github.com/nvanbenschoten/raft-toy.
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.
When a follower requires a snapshot and the snapshot is sent at the
committed (and last) index in an otherwise idle Raft group, the follower
would previously remain in ProgressStateProbe even though it had been
caught up completely.
In a busy Raft group this wouldn't be an issue since the next round of
MsgApp would update the state, but in an idle group there's nothing
that rectifies the status (since there's nothing to append or update).
The reason this matters is that the state is exposed through
`RaftStatus()`. Concretely, in CockroachDB, we use the Raft status to
make sharding decisions (since it's dangerous to make rapid changes
to a fragile Raft group), and had to work around this problem[1].
[1]: 91b11dae41/pkg/storage/split_delay_helper.go (L138-L141)
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).
Prior to this change, MaxSizePerMsg was used both to cap the total byte size of
entries in messages as well as the total byte size of entries passed through
CommittedEntries in the Ready struct. This change adds a new Config parameter
MaxCommittedSizePerReady which defaults to MaxSizePerMsg and contols the second
of above descibed settings.
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
The MaxSizePerMsg setting is now used to limit the size of
Ready.CommittedEntries. This prevents out-of-memory errors if the raft
log has become very large and commits all at once.
"stepCandidate" should reuse candidate's own term, not term in Message,
because pre-vote is requested with future term.
Signed-off-by: Gyuho Lee <gyuhox@gmail.com>
`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>
This includes one theoretical logic change: A node that knows the
leader of the current term will no longer grant votes, even if it has
not yet voted in this term. It also adds a `m.Type == MsgPreVote`
guard on the `m.Term > r.Term` check, which was previously thought to
be incorrect (see #8517) but was actually just unclear.
Closes#8517Closes#8571
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.
TestNodeWithSmallerTermCanCompleteElection tests the scenario where a
node that has been partitioned away (and fallen behind) rejoins the
cluster at about the same time the leader node gets partitioned away.
Previously the cluster would come to a standstill when run with PreVote
enabled.
When responding to Msg{Pre,}Vote messages we now include the term from
the message, not the local term. To see why consider the case where a
single node was previously partitioned away and it's local term is now
of date. If we include the local term (recall that for pre-votes we
don't update the local term), the (pre-)campaigning node on the other
end will proceed to ignore the message (it ignores all out of date
messages).
The term in the original message and current local term are the same in
the case of regular votes, but different for pre-votes.
NB: Had to change TestRecvMsgVote to include pb.Message.Term when
sending MsgVote messages. The new sanity checks on MsgVoteResp
(m.Term != 0) would panic with the old test as raft.Term would be equal
to 0 when responding with MsgVoteResp messages.
I found that enabling the CheckQuorum flag led to spurious leader
elections when new nodes joined. It looks like in the time between a new
node joining the cluster, and that node first communicating with the
leader, the quorum check could fail because the new node looks inactive.
To solve this, set the RecentActive flag when nodes are first added.
This gives a grace period for the node to communicate before it causes
the quorum check to fail.
Signed-off-by: Aaron Lehmann <aaron.lehmann@docker.com>
After add node conf proposed twice with the same node id, the pending state is not reset because
the addNode returned without setting the pending state at the second
time and the pending state will always be true unless other conf changed. During this we
can not add any new node because the propose will be ignored since the
pending state is true.