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-15607:Possible NPE is thrown in MirrorCheckpointTask #14587

Merged
merged 2 commits into from Oct 20, 2023

Conversation

hudeqi
Copy link
Collaborator

@hudeqi hudeqi commented Oct 19, 2023

In the syncGroupOffset method, if targetConsumerOffset.get(topicPartition) gets null, then the calculation of latestDownstreamOffset will throw NPE. This usually occurs in this situation: a group consumed a topic in the target cluster previously. Later, the group offset of some partitions was reset to -1, the OffsetAndMetadata of these partitions was null.

It is possible that when reset offsets are performed in the java kafka client, the reset to -1 will be intercepted. However, there are some other types of clients such as sarama, which can magically reset the group offset to -1, so MM2 will trigger an NPE exception in this scenario. Therefore, a defensive measure to avoid NPE is needed here.

@hudeqi
Copy link
Collaborator Author

hudeqi commented Oct 19, 2023

Hi, @clolov. Excuse me, can you help me trigger the build for this PR? I'm also collaborator, why can't I find a place to trigger build PR :)

@C0urante
Copy link
Contributor

C0urante commented Oct 19, 2023

Thanks @hudeqi, I think this is reasonable. Do you know why Sarama sets offsets to -1? If it's for normal operations and not indicative of something wrong, we may not even need to log a warning message in that case and could change the check here from !targetConsumerOffset.containsKey(topicPartition) to targetConsumerOffset.get(topicPartition) == null.

…irrorCheckpointTaskTest.java

Co-authored-by: Chris Egerton <fearthecellos@gmail.com>
@hudeqi
Copy link
Collaborator Author

hudeqi commented Oct 20, 2023

Thanks @hudeqi, I think this is reasonable. Do you know why Sarama sets offsets to -1? If it's for normal operations and not indicative of something wrong, we may not even need to log a warning message in that case and could change the check here from !targetConsumerOffset.containsKey(topicPartition) to targetConsumerOffset.get(topicPartition) == null.

Thanks your review. @C0urante . In fact, directly resetting to -1 is an abnormal operation, whether for Sarama or other clients. This problem was discovered in this way: when using the Sarama client, we wanted to reset the group's offset to the latest, so we passed in the OffsetNewest in Sarama as a parameter to call the reset offset method. Finally, it was discovered that the offset was reset to -1. The reason is that the value of OffsetNewest is -1. For Sarama, resetting to the latest should be another operation process, but this kind of misoperation is not intercepted like the java client, which can be deal friendly. So this issue occurred when encountering scenarios like MM2. So I think it is better to add a warn log here.

@hudeqi hudeqi requested a review from C0urante October 20, 2023 12:53
@C0urante
Copy link
Contributor

Makes sense, thanks for the explanation @hudeqi!

@C0urante C0urante merged commit 4083cd6 into apache:trunk Oct 20, 2023
1 check failed
C0urante pushed a commit that referenced this pull request Oct 20, 2023
C0urante pushed a commit that referenced this pull request Oct 20, 2023
@hudeqi hudeqi deleted the fix_npe_mm2 branch October 23, 2023 02:22
mjsax pushed a commit to confluentinc/kafka that referenced this pull request Nov 22, 2023
anurag-harness pushed a commit to anurag-harness/kafka that referenced this pull request Feb 9, 2024
anurag-harness added a commit to anurag-harness/kafka that referenced this pull request Feb 9, 2024
…#14587) (#146)

Reviewers: Chris Egerton <chrise@aiven.io>

Co-authored-by: hudeqi <1217150961@qq.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
2 participants