-
Notifications
You must be signed in to change notification settings - Fork 3.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: assert on HardState field modifications #133169
base: master
Are you sure you want to change the base?
raft: assert on HardState field modifications #133169
Conversation
a3a2b5b
to
d85677c
Compare
@@ -2076,8 +2068,6 @@ func stepCandidate(r *raft, m pb.Message) error { | |||
// m.Term > r.Term; reuse r.Term | |||
r.becomeFollower(r.Term, r.lead) | |||
} | |||
case pb.MsgTimeoutNow: |
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.
This removal seems unrelated to this change. Is it removed because the message is not useful? LGTM, but just checking the reason for it.
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.
It is removed because MsgTimeoutNow
is caught by IsMsgFromLeader
. With this change, a candidate will now handle the MsgTimeoutNow
as an instruction to force campaign at the next term, instead of ignoring it.
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.
Sounds like a behaviour change (previously we would stay candidate because we've already started election and don't need to do it again; now we will step back to follower and re-campaign again?). I don't have a good sense whether it's a benign change. Would it be possible to retain the old behaviour?
Also, do we need to assume MsgTimeoutNow
came from the leader? It's a forceful campaign, and in theory any node could randomly decide to do that without the leader's involvement. Perhaps the uncertainty comes from the fact that it's used both for forced campaigns and leader transfers (and in the latter case it makes sense to say that the message is made on behalf of this leader).
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.
Yes, this is new behavior. But it's desirable behavior. I've extracted it into #133490.
Also, do we need to assume MsgTimeoutNow came from the leader? It's a forceful campaign, and in theory any node could randomly decide to do that without the leader's involvement. Perhaps the uncertainty comes from the fact that it's used both for forced campaigns and leader transfers (and in the latter case it makes sense to say that the message is made on behalf of this leader).
It used to be disruptive but safe to allow any node to decide to perform a force campaign. With leader fortification and leader leases, it is now very unsafe. A force campaign will instruct any voter to ignore the fortification promise that it has made to the leader, which could cause a lease expiration regression (overlapping leases). Force campaigns are only safe if the leader is the one initiating them, because it will step down to a follower at the same time.
used both for forced campaigns and leader transfers
The notion of a "forced campaign", separate from a leader transfer is relatively new. It came from #104969 and I'd like to eliminate the concept with something like #133308.
} | ||
|
||
func (r *raft) resetLeadEpoch() { | ||
r.leadEpoch = 0 |
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.
Non-actionable, but just curious about the fortification protocol: is it safe to reset leadEpoch
to 0 in the de-fortify case?
Can the leadEpoch
be set to non-zero again during the course of the term? If yes, can it be set to a value below / different than the last non-zero leadEpoch
? As a thought experiment: what happens when messages are delayed / reordered / duplicated: can we get in and out of leadEpoch == 0
/ in what ways, and is it fine?
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.
It is safe to reset to 0 when if either (1) the leader has told us to defortify, or (2) we have withdrawn support for the lead epoch in the store liveness network.
leadEpoch
can then be set to a non-zero value again if a new MsgForgetLeader
is received, which is by design. However, the value must be equal or greater than it was before it was reset to 0, because it comes from a call to storeLiveness.SupportFor
, which is guaranteed to be monotonic.
d85677c
to
a5cb07c
Compare
pkg/kv/kvserver/replica_raft.go
Outdated
msg := raftpb.Message{ | ||
To: raftpb.PeerID(r.replicaID), | ||
// We pretend that the message was sent from the leader. | ||
From: lead, | ||
Type: raftpb.MsgTimeoutNow, | ||
} |
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.
Is this related to the change in this commit, or a drive-by? I see that From
is used in the handler for logging, is that why?
Also, is it safe/canonical to communicate a leader without communicating the term? I imagine if a follower receives a MsgTimeoutNow
for a long gone term, it should ignore.
I know this message is synchronous/local here (so there can't be a term mismatch), but it isn't when this message is send from leader to a peer.
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.
Is this related to the change in this commit, or a drive-by? I see that From is used in the handler for logging, is that why?
It's related to the change and to our thread above. The use of MsgTimeoutNow
like this is a hack and is hardly safe, it's only maybe ok because of when this code is run (see shouldCampaignAfterConfChange
). I'd like to get rid of this, but in the meantime, I'm ok making it even more clear that cockroach is lying to raft about this message coming from the leader, which in some ways it is because the leader is the one committing the entry which demotes itself to a learner.
Also, is it safe/canonical to communicate a leader without communicating the term? I imagine if a follower receives a MsgTimeoutNow for a long gone term, it should ignore.
Good point. I added the Term
, along with a better comment.
@@ -2076,8 +2068,6 @@ func stepCandidate(r *raft, m pb.Message) error { | |||
// m.Term > r.Term; reuse r.Term | |||
r.becomeFollower(r.Term, r.lead) | |||
} | |||
case pb.MsgTimeoutNow: |
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.
Sounds like a behaviour change (previously we would stay candidate because we've already started election and don't need to do it again; now we will step back to follower and re-campaign again?). I don't have a good sense whether it's a benign change. Would it be possible to retain the old behaviour?
Also, do we need to assume MsgTimeoutNow
came from the leader? It's a forceful campaign, and in theory any node could randomly decide to do that without the leader's involvement. Perhaps the uncertainty comes from the fact that it's used both for forced campaigns and leader transfers (and in the latter case it makes sense to say that the message is made on behalf of this leader).
This commit updates stepCandidate to not ignore MsgTimeoutNow messages from the leader. These messages not only teach the candidate who the leader is, but they also allow the candidate to call a new election at the _next_ term. Furthermore, they allow the candidate to perform a "force" election. So we don't want to drop these messages if at all possible. This is a pretty rare case because a leader will only transfer leadership to a peer that it knows has an up-to-date log, but it is possible, as demonstrated by the test. Epic: None Release note: None
Informs cockroachdb#125352. This commit adds setters for the HardState fields (Term, Vote, lead, leadEpoch), allowing us to assert that these fields are only modified in expected ways. Some bugs that this change could catch: - term regressions - vote changes - lead changes in a single term - leadEpoch regressions The "lead changes in a single term" class of bugs is especially interesting, as we have found two of these in this thread: https://cockroachlabs.slack.com/archives/G01G8LK77DK/p1720274557285239. These are tracked in the following issues, which we may need to address ASAP: * cockroachdb#127349 * cockroachdb#127348 Release note: None
Group the handling of learning about the leader in stepFollower. Epic: None Release note: None
Group the handling of the candidate -> follower state transition when hearing from the leader while a candidate. Epic: None Release note: None
We can't consider MsgSnap to be from the leader of Message.Term until we address cockroachdb#127348 and cockroachdb#127349. Epic: None Release note: None
a5cb07c
to
395f778
Compare
Informs #125352.
This commit adds setters for the HardState fields (Term, Vote, lead, leadEpoch), allowing us to assert that these fields are only modified in expected ways.
Some bugs that this change could catch:
The "lead changes in a single term" class of bugs is especially interesting, as we have found two of these in this thread: https://cockroachlabs.slack.com/archives/G01G8LK77DK/p1720274557285239. These are tracked in the following issues, which we may need to address ASAP:
The 2nd, 3rd, and 4th commits in the PR address these issues in raft by not assuming that a
MsgSnap
as always coming from the leader ofMessage.Term
.Release note: None