-
Notifications
You must be signed in to change notification settings - Fork 42
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
Correct handling of term updates during the election for Candidate/Follower #55
Conversation
…higher term number is received from Candidate
After running whole tests few times I've discovered that the multi jvm tests are failing from time to time and for now I have no idea why. I need to investigate this test cases deeper and update my pull request accordingly. |
case Event(msg: RequestVote, m: ElectionMeta) if m.canVoteIn(msg.term) => | ||
sender ! VoteCandidate(m.currentTerm) | ||
stay() using m.withVoteFor(msg.term, candidate()) | ||
case Event(msg: RequestVote, m: ElectionMeta) if msg.term < m.currentTerm => |
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.
Minor nit: it looks like the rest of the source code uses two spaces for indentation, whereas this pull request has tabs
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.
Thx for pointing this out, I will fix that.
Random guess: I bet that the multi-jvm tests are inherently somewhat non-deterministic. |
@colin-scott Hi, Colin. I guess you're right but it's not good anyway. The problem is that previously term updates that are handled by this PR weren't reflected in the code and this new changes can cascade and affect other parts of code that depends on it because everything is linked to some extend. |
Just realized: I think you should also have the leader step down if term > currentTerm. It currently stay()s: |
def forFollower: Meta = Meta(clusterSelf, currentTerm, config, Map.empty) | ||
def forNewElection: ElectionMeta = this.forFollower.forNewElection | ||
def forLeader: LeaderMeta = LeaderMeta(clusterSelf, currentTerm, config) | ||
def forFollower(term: Term = currentTerm): Meta = Meta(clusterSelf, term, config, Map.empty) |
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.
Rather than having a default argument here, maybe it would be cleaner to chain the methods?
e.g.
goto(Follower) with m.forFollower.withTerm(term)
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.
Nice catch! Also thought about that. There is a trade off between efficiency and readability and I just wanted to remove unnecessary copying of objects if its possible. Surely, I know about premature optimizations :) I think you're right there is also one more place where same "side effect" is hidden.
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.
In this particular piece I'm ok with the default param, though the explicit version would be nice as well. No need to change just yet.
Can be an issue though. Leader should also have the same term update rules as candidate. Will check that later. |
After investigating test failures for some time I've found that election tests are failing from time to time on one of three JVM nodes with the following output:
Same error messages can be observed in the failed build log of travis. Don't actually know what can cause this problem but probably it's not related to the bugs in code itself. |
Gating happens when a connection to another node fails (i.e. in multi node tests when a node is killed) as a node notices it cannot talk to the other node and "gates it" for a while, trying to re-establish the connection later on. |
@ktoso Is there any way to prevent it from failing or its an expected behavior? I can retrigger the build and it will eventually succeed but it's not the best solution I guess. |
I'll need to look at the tests where exactly it happens, some tests could be purposefully killing nodes – then it's totally expected. Sadly now prepping and delivering a webinar so will only get to it rather later. Thanks for all the work btw! I hope to get to reviewing soon. |
stay() | ||
|
||
case Event(msg: RequestVote, m: ElectionMeta) if msg.term > m.currentTerm => | ||
log.info("Received newer {}. Current term is {}. Revert to follower state.", msg.term, m.currentTerm) |
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.
perhaps you should forward the RequestVote message instead of dropping it?
e.g.
m.clusterSelf forward msg
goto(Follower) using m.forFollower(msg.term)
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 sounds like an optimisation (i.e. not needed for correctness), right?
If so let's skip it for now - let's aim for pure correctness in the upcoming work and PRs :)
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.
Yeah, it's an optimization
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.
Made it an issue so we can re-visit it later #61
@@ -64,7 +91,7 @@ private[raft] trait Candidate { | |||
if (leaderIsAhead) { | |||
log.info("Reverting to Follower, because got AppendEntries from Leader in {}, but am in {}", append.term, m.currentTerm) | |||
m.clusterSelf forward append | |||
goto(Follower) using m.forFollower | |||
goto(Follower) using m.forFollower() |
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.
Minor: I actually like to use forFollower
methods as it indicated the method has no side effects. Imagine it's a getter / field.
That style refers to the http://docs.scala-lang.org/glossary/#uniform-access-principle
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.
FWIW, with a default parameter (as @dmitraver added), it doesn't compile without parens
Changes make sense, thanks a lot @dmitraver! I'll merge the changes and work a bit in the infra / timeouts to make tests not as fleaky. |
For the next contrib @dmitraver please use 2-spaces though... 😉 |
Correct handling of term updates during the election for Candidate/Follower
@ktoso Thx for the hint ;) Will do. |
According to the raft paper
"Current terms are exchanged whenever servers communicate; if one server’s current term is smaller than the other’s, then it updates its current term to the larger value. If a candidate or leader discovers that its term is out of date, it immediately reverts to follower state. If a server receives a request with a stale term number, it rejects the request."
This pull request implements this rules for election phase of Follower/Candidate states. It also fixes the #46 and #47 issues.