Skip to content

KAFKA-13943: Make LocalLogManager implementation consistent with the RaftClient interface contract#12224

Merged
hachikuji merged 3 commits intoapache:trunkfrom
divijvaidya:KAFKA-13943
Jul 6, 2022
Merged

KAFKA-13943: Make LocalLogManager implementation consistent with the RaftClient interface contract#12224
hachikuji merged 3 commits intoapache:trunkfrom
divijvaidya:KAFKA-13943

Conversation

@divijvaidya
Copy link
Member

@divijvaidya divijvaidya commented May 28, 2022

1\ As per the interface contract for RaftClient.scheduleAtomicAppend() [1], it should throw a NotLeaderException exception when the provided current leader epoch does not match the current epoch. However, the current LocalLogManager's implementation of the API returns a LONG_MAX instead of throwing an exception. This change fixes the behaviour and makes it consistent with the interface contract.

2\ As per the interface contract for RaftClient.resign(epoch) [2] if the parameter epoch does not match the current epoch, this call will be ignored. But in the current LocalLogManager implementation the leader epoch might change when the thread is waiting to acquire a lock on shared.tryAppend() (note that tryAppend() is a synchronized method). In such a case, if a NotALeaderException is thrown (as per code change in 1\ above), then resign should be ignored.

[1] https://github.com/apache/kafka/blob/trunk/raft/src/main/java/org/apache/kafka/raft/RaftClient.java#L145
[2] https://github.com/apache/kafka/blob/trunk/raft/src/main/java/org/apache/kafka/raft/RaftClient.java#L208

@divijvaidya
Copy link
Member Author

@cmccabe as the primary author of LocalLogManager, perhaps you may want to review this change?

Note that the CI is successful for all tests except testTopicIdUpgradeAfterReassigningPartitions which is a known flaky test and is being fixed in #11687

Copy link
Member

@dengziming dengziming left a comment

Choose a reason for hiding this comment

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

The first point is an omission of #10909, Thanks.

@divijvaidya
Copy link
Member Author

cc: @jsancio perhaps you would be interested to look into this one since you reviewed #10909

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 @divijvaidya .

For my understanding, what is the motivation for this change? For example, did you discover this inconsistency while working on additional tests?

@divijvaidya
Copy link
Member Author

divijvaidya commented Jun 8, 2022

For my understanding, what is the motivation for this change?

Apologies for late response @jsancio (I was on leave) and thank you for your review.

It's interesting that you ask this question because there indeed is a motivation. I was trying to fix the flaky test described in https://issues.apache.org/jira/browse/KAFKA-13943. As you will observe from the stack trace, due to the bug fixed here in this PR, a snapshot was being created with LONG_MAX offset and thus leading to further cascading failures. The test is flaky because it only happens when an append request started off from a leader but when the request was waiting for synchronized lock in the tryAppend() method, the leader changed. Sometimes, the leader doesn't change while it is waiting and sometimes it does. With this fix, I hope to fix that bug.
But when I fixed this bug, the resign behaviour started facing problems. I investigated found out the bug where we don't handle NotALeader exception correctly in resign. I included that to this PR change.

Does this answer your question? Please feel free to add your thoughts to the JIRA directly.

@divijvaidya
Copy link
Member Author

@jsancio this is ready for another review. Thanks for your time again.

@divijvaidya divijvaidya changed the title MINOR: Make LocalLogManager implementation consistent with the RaftClient interface contract KAFKA-13943: Make LocalLogManager implementation consistent with the RaftClient interface contract Jun 10, 2022
@divijvaidya
Copy link
Member Author

@jsancio please review when you get a chance. Currently multiple tests in QuorumControllerTest are acting flaky because we are allowing to create a snapshot with LONG_MAX value. This is making it difficult to review PRs due to flakiness.

This code change fixes that.

Copy link
Member

@tombentley tombentley left a comment

Choose a reason for hiding this comment

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

Thanks @divijvaidya, I left a few comments but I'm not really an expert in this area yet. Hopefully @jsancio can take another look.

@divijvaidya
Copy link
Member Author

divijvaidya commented Jun 28, 2022

@hachikuji requesting your review on this PR when you get a chance!

Copy link
Contributor

@hachikuji hachikuji 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 fix. Left a minor suggestion.

private long append(int epoch, List<T> records, boolean isAtomic) {
if (epoch < this.epoch) {
throw new NotLeaderException("Append failed because the epoch doesn't match");
throw new NotLeaderException("Append failed because the given epoch is stale");
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: could we add the provided epoch and the current epoch to this message?

Copy link
Member Author

Choose a reason for hiding this comment

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

done. Please review.

Copy link
Contributor

@hachikuji hachikuji 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 patch! LGTM.

@hachikuji hachikuji merged commit 5e4c8f7 into apache:trunk Jul 6, 2022
@divijvaidya divijvaidya deleted the KAFKA-13943 branch July 7, 2022 10:06
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants