Skip to content
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

KAFKA-17641: Update Vote RPC with PreVote field #17807

Merged
merged 25 commits into from
Dec 13, 2024

Conversation

ahuang98
Copy link
Contributor

@ahuang98 ahuang98 commented Nov 14, 2024

KIP-996 Part 1: https://issues.apache.org/jira/browse/KAFKA-17641
Introduces v2 of vote RPC and includes handling of PreVote requests. Includes KafkaRaftClientPreVoteTest.

Some behaviors to note:

  • Many references to candidate are changed to more generic replica. In the next PreVote PR we will be adding Prospective state and it will not be guaranteed that candidates are sending vote requests.
  • Replicas receiving PreVote requests (vote request with prevote=true) with an epoch equal to their own will not transition to Unattached state. They will only grant the vote if they do not know of the leader/have not fetched from them yet in the epoch and the request's last epoch and offset are up-to-date with theirs.
  • If a replica receives a PreVote request with an epoch greater than their current epoch, they will transition to Unattached state (setting their epoch to one from the prevote request) and then grant the vote if the above conditions are met.
  • Vote requests with prevote=false are handled the same way as before.
  • Included in this PR are changes to Follower state's canGrantVote. This addresses a ping-pong scenario brought up by Jack Vanlightly. (ex: 3 node quorum, leader node A disconnects from quorum, node B goes into prospective state first before node C, node B sends prevote request to node C still in follower state and receives back that node A is leader, node B transitions to follower while node C transitions to prospective after election timeout, repeating this cycle). We tackle this by having follower state nodes grant prevote requests based off last fetch time.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

@ahuang98
Copy link
Contributor Author

cc @jsancio for review!

Copy link
Member

@jsancio jsancio left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@ahuang98 Thanks for the changes. Focused on the changes to src/main. We should fully implement the request handling, outside of the prospective state, in this PR.

clients/src/main/resources/common/message/VoteRequest.json Outdated Show resolved Hide resolved
clients/src/main/resources/common/message/VoteRequest.json Outdated Show resolved Hide resolved
Comment on lines 1185 to 1187
case BEGIN_QUORUM_EPOCH: return createBeginQuorumEpochResponse();
case END_QUORUM_EPOCH: return createEndQuorumEpochResponse();
case DESCRIBE_QUORUM: return createDescribeQuorumResponse();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see. I haven't looked at the rest of these tests in detail but should we create an issue to test the different versions of the responses for the KRaft specific RPCs?

* @return true if it can grant the vote, false otherwise
*/
boolean canGrantVote(ReplicaKey candidateKey, boolean isLogUpToDate);
boolean canGrantVote(ReplicaKey replicaKey, boolean isLogUpToDate);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The implementation of this method is not correct for all cases. The KIP says the following:

When servers receive VoteRequests with the PreVote field set to true, they will respond with VoteGranted set to
. true if they are not a Follower and the epoch and offsets in the Pre-Vote request satisfy the same requirements as a standard vote
. false if otherwise

I know that your PR description says the following:

Not included in this PR are changes to Follower state's canGrantVote. This may need to change in later PreVote PR to address a ping-pong scenario brought up by Jack Vanlightly. (ex: 3 node quorum, leader node A disconnects from quorum, node B goes into prospective state first before node C, node B sends prevote request to node C still in follower state and receives back that node A is leader, node B transitions to follower while node C transitions to prospective after election timeout, repeating this cycle) One way to tackle this is by having follower state nodes grant prevote requests based off last fetch time, which would involve changing canGrantVote, but this makes more sense to tackle after the Prospective state exists.

But I think we need to implement and test this logic in this PR. Else the handling of this RPC is not correct for all software versions that support this RPC.

Copy link
Contributor Author

@ahuang98 ahuang98 Nov 20, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I edited the PR description - I don't believe Follower state needs any changes anymore
[Edit] I decided to go the route previously mentioned - Follower state's canGrantVote is updated in this PR

raft/src/test/java/org/apache/kafka/raft/RaftUtilTest.java Outdated Show resolved Hide resolved
"{\"clusterId\":\"I4ZmrWqfT2e-upky_4fdPA\",\"voterId\":2,\"topics\":[{" +
"\"topicName\":\"topic\",\"partitions\":[{\"partitionIndex\":1,\"replicaEpoch\":1," +
"\"replicaId\":1,\"replicaDirectoryId\":\"AAAAAAAAAAAAAAAAAAAAAQ\"," +
"\"voterDirectoryId\":\"AAAAAAAAAAAAAAAAAAAAAQ\",\"lastOffsetEpoch\":1000,\"lastOffset\":1000," +
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm. Outside the scope of this PR but these examples only test the case when the directory id is not set or zero.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the tests are specifying directory id of Uuid.ONE_UUID, I could change it to something not reserved

@ahuang98
Copy link
Contributor Author

ahuang98 commented Nov 20, 2024

This depends on #17352 (cc @jsancio if you have time to review)

@ahuang98 ahuang98 mentioned this pull request Dec 4, 2024
3 tasks
Copy link
Member

@jsancio jsancio left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the changes @ahuang98 . I focused my review on the changes to **/src/main.

boolean canGrantVote(ReplicaKey candidateKey, boolean isLogUpToDate);
boolean canGrantVote(ReplicaKey replicaKey, boolean isLogUpToDate);

boolean canGrantPreVote(ReplicaKey replicaKey, boolean isLogUpToDate);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's add a Java Doc. I am interested to see how this method differs from canGrantVote.

Having said that, did you consider having one method with this signature: canGrantVote(ReplicaKey, boolean isLogUpdate, boolean isPreVote)? If yes, why did you reject this interface change?

Copy link
Contributor Author

@ahuang98 ahuang98 Dec 10, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes, originally it was just the one method since the logic for granting preVotes used to be (incorrectly) very similar to granting standard votes. I decided to split it out since I was seeing the cyclomatic complexity of certain states getting worse (e.g. Follower, Unattached state).

I also considered having just one overridden EpochState method canGrantVote() and then having that call a private helper canGrantPreVote, but that just seemed like an extra hoop for the sake of not creating an extra method for the EpochState interface

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

but that just seemed like an extra hoop for the sake of not creating an extra method for the EpochState interface

We do have this method call. It is in QuorumState::canGrantVote. If we didn't have this method, you would have the same if statement at every call site. The other concern I have is which is easier to unittest? For example, it doesn't look like we have a unittest for QuorumState::canGrantVote.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We do have this method call. It is in QuorumState::canGrantVote. If we didn't have this method, you would have the same if statement at every call site.

I agree with this statement, but not sure if I'm understanding if you are trying to make a further point.

The other concern I have is which is easier to unittest? For example, it doesn't look like we have a unittest for QuorumState::canGrantVote.

As you said, QuorumState::canGrantVote just calls either EpochState::canGrantVote or EpochState::canGrantPreVote and both of those methods are unit tested?

@@ -44,7 +44,7 @@
* Follower: After discovering a leader with an equal or larger epoch
*
* Unattached transitions to:
* Unattached: After learning of a new election with a higher epoch or after voting
* Unattached: After learning of a new election with a higher epoch or after giving a binding vote
Copy link
Member

@jsancio jsancio Dec 10, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This applies to the "Unattached transitions" and "Voted transitions" section.

Should we merge these two and update the wording since in a previous PR we merge these two states.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

looks like I changed it in the second PR when I added the new Prospective state. I'll fix it here too

raft/src/main/java/org/apache/kafka/raft/QuorumState.java Outdated Show resolved Hide resolved
raft/src/main/java/org/apache/kafka/raft/RaftUtil.java Outdated Show resolved Hide resolved
Comment on lines 220 to 234
if (hasFetchedFromLeader) {
log.debug(
"Rejecting PreVote request from replica ({}) since we already have a leader {} in epoch {}",
replicaKey,
leaderId,
epoch
);
return false;
} else if (!isLogUpToDate) {
log.debug(
"Rejecting PreVote request from replica ({}) since replica epoch/offset is not up to date with us",
replicaKey);
}
return isLogUpToDate;
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This method is a lot of lines because we are trying to generate a very specific log message. I don't think we need that since debug message are for developers. How about:

        boolean granting = !hasFetchedFromLeader && isLogUpToDate;
        if (!granting) {
            log.debug(
                "Rejecting PreVote request from replica ({}) since leader {}, epoch is {}, isLogUpToDate is {} and hasFetched is {}",
                replicaKey,
                leaderId,
                epoch,
                isLogUpToDate,
                hasFetchedFromLeader
            );
        }

        return granting;
    }

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

thanks, trying out something a bit wordier to make it clear, lmk what you think

  "Rejecting PreVote request from replica ({}) either because we have already fetched from leader {} " +
  "in epoch {} (hasFetchedFromLeader={}), or because the replica's log is not as up to date with " +
   "ours (isLogUpToDate={})",

@ahuang98
Copy link
Contributor Author

addressing these failures now

Found 6 test failures:
FAILED ❌ UnattachedStateWithVoteTest > "testCanGrantVoteWithoutDirectoryId(boolean).isLogUpToDate=false"
FAILED ❌ ResignedStateTest > "testGrantVote(boolean).isLogUpToDate=true"
FAILED ❌ RaftUtilTest > "testSingletonVoteResponseForAllVersion(short, String).version=2, expectedJson={"errorCode":0,"topics":[{"topicName":"topic","partitions":[{"partitionIndex":0,"errorCode":0,"leaderId":1,"leaderEpoch":1,"voteGranted":true,"preVote":true}]}],"nodeEndpoints":[{"nodeId":1,"host":"localhost","port":9990}]}"
FAILED ❌ CandidateStateTest > "testGrantVote(boolean, boolean).isLogUpToDate=true, withDirectoryId=true"
FAILED ❌ CandidateStateTest > "testGrantVote(boolean, boolean).isLogUpToDate=true, withDirectoryId=false"
FAILED ❌ AbstractCoordinatorTest > testWakeupAfterSyncGroupReceived()

@jsancio jsancio self-assigned this Dec 12, 2024
Copy link
Member

@jsancio jsancio left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the changes @ahuang98

Comment on lines 49 to 50
{ "name": "LastOffset", "type": "int64", "versions": "0+",
"about": "The offset of the last record written to the metadata log."}
"about": "The offset of the last record written to the metadata log."},
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Okay. Outside the scope of this PR but this is not correct. The replica sends the end offset and the receiving replica interprets this value as the end offset.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what do you think about - The log end offset of the metadata log of the voter sending the request

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sounds good. I also don't like the field name LastOffset but I can fix that in a minor PR.

Comment on lines 40 to 43
/* Used to track if we have fetched from the leader at least once since the transition to follower in this epoch.
* If we have not yet fetched from the leader, we may grant PreVotes.
*/
private boolean hasFetchedFromLeader;
Copy link
Member

@jsancio jsancio Dec 12, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This tracks that the replica has fetch successfully not just fetched.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Btw, I was more concerned with the comment than the name of the variable. I should have been more explicit in my comment.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ah gotcha!

@@ -92,6 +92,19 @@ public void testMonotonicHighWatermark() {
assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark());
}

@ParameterizedTest
@ValueSource(booleans = {true, false})
public void testPreVoteIfHasNotFetchedFromLeaderYet(boolean isLogUpToDate) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What about the case when the follower has successfully fetched from the leader?

import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;

public class KafkaRaftClientPreVoteTest {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This suite doesn't seem to test the case when the remove replica doesn't have a log that is update with the local replica.

Copy link
Member

@jsancio jsancio left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM. Thanks for the changes @ahuang98

@jsancio jsancio merged commit b73e31e into apache:trunk Dec 13, 2024
9 checks passed
tedyu pushed a commit to tedyu/kafka that referenced this pull request Jan 6, 2025
Introduces v2 of Vote RPC and implements the handling of the new version of the RPC.

Many references to "candidate" in the Vote RPC are changed to the more generic "replica". Replicas sending Vote request with PreVote set to true are not candidate. They are instead prospective candidate that are attempting to become candidate.

Replicas receiving PreVote requests (vote request with PreVote=true) with an epoch equal to their own will _not_ transition to Unattached state. They will only grant the vote if they have not recently fetched from leader and the request's last epoch and offset are up-to-date with theirs.

If a replica receives a PreVote request with an epoch greater than their current epoch, they will transition to Unattached state (setting their epoch to the one from the pre-vote request) and then grant the vote if the request's last epoch and offset are up-to-date with theirs.

To avoid a possible ping-pong scenario. For example, there is 3 node quorum, leader node A disconnects from quorum, node B goes into prospective state first before node C, node B sends pre-vote request to node C still in follower state and receives back that node A is leader, node B transitions to follower while node C transitions to prospective after election timeout. If you repeat this interaction, it is possible for such replicas to transition from Follower to Prospective in perpetuity. This issue is resolved by having follower state nodes grant pre-vote requests only if they have successfully fetched from the leader at least once after becoming a follower.

This change introduces a new suite called KafkaRaftClientPreVoteTest, for additional KRaft protocol tests with respect to pre-vote.

Reviewers: José Armando García Sancio <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

2 participants