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-15771: fix ProduceRequest#partitionSizes() to make it an atomic operation #14674
Conversation
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.
Thanks for the PR @fxbing . I am not totally clear about what we are trying to solve using this though. Could you please elaborate in the PR description section? Is there a race condition that you have identified?
@vamossagar12 I added some instructions, please see if there are other questions |
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.
Thanks for this PR! @fxbing. I think this deserves a jira to track. And can you add a unit test?
@hudeqi Thanks for the comments, I've added the jira issue. |
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.
Thanks for the explanation @fxbing and for creating the JIRA ticket. It makes sense now. LGTM. Regarding tests, I agree it might be difficult to write a test which will simulate this race condition always. I am ok to not have such a test.
@chia7712 , do you ming taking a look at this one plz? |
@vamossagar12 @hudeqi Can this PR be merged? |
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.
Thank you for the change. Looks good! I will backport it to 3.5 and 3.6 as well.
Test failures and build failure is unrelated:
|
…14674) A commit fixes a bug in ProduceRequest#partitionSizes() that may cause this method to incorrectly returning an empty or incomplete response for a thread when another thread is in the process of initialising it. Reviewers: Divij Vaidya <diviv@amazon.com>, hudeqi <1217150961@qq.com>, vamossagar12 <sagarmeansocean@gmail.com> -------------------------------- Co-authored-by: fangxiaobing <fangxiaobing@kuaishou.com>
…14674) A commit fixes a bug in ProduceRequest#partitionSizes() that may cause this method to incorrectly returning an empty or incomplete response for a thread when another thread is in the process of initialising it. Reviewers: Divij Vaidya <diviv@amazon.com>, hudeqi <1217150961@qq.com>, vamossagar12 <sagarmeansocean@gmail.com> -------------------------------- Co-authored-by: fangxiaobing <fangxiaobing@kuaishou.com>
…pache#14674) A commit fixes a bug in ProduceRequest#partitionSizes() that may cause this method to incorrectly returning an empty or incomplete response for a thread when another thread is in the process of initialising it. Reviewers: Divij Vaidya <diviv@amazon.com>, hudeqi <1217150961@qq.com>, vamossagar12 <sagarmeansocean@gmail.com> -------------------------------- Co-authored-by: fangxiaobing <fangxiaobing@kuaishou.com>
…pache#14674) A commit fixes a bug in ProduceRequest#partitionSizes() that may cause this method to incorrectly returning an empty or incomplete response for a thread when another thread is in the process of initialising it. Reviewers: Divij Vaidya <diviv@amazon.com>, hudeqi <1217150961@qq.com>, vamossagar12 <sagarmeansocean@gmail.com> -------------------------------- Co-authored-by: fangxiaobing <fangxiaobing@kuaishou.com>
…pache#14674) A commit fixes a bug in ProduceRequest#partitionSizes() that may cause this method to incorrectly returning an empty or incomplete response for a thread when another thread is in the process of initialising it. Reviewers: Divij Vaidya <diviv@amazon.com>, hudeqi <1217150961@qq.com>, vamossagar12 <sagarmeansocean@gmail.com> -------------------------------- Co-authored-by: fangxiaobing <fangxiaobing@kuaishou.com>
…pache#14674) (#218) A commit fixes a bug in ProduceRequest#partitionSizes() that may cause this method to incorrectly returning an empty or incomplete response for a thread when another thread is in the process of initialising it. Reviewers: Divij Vaidya <diviv@amazon.com>, hudeqi <1217150961@qq.com>, vamossagar12 <sagarmeansocean@gmail.com> -------------------------------- Co-authored-by: Xiaobing Fang <bingxf@qq.com> Co-authored-by: fangxiaobing <fangxiaobing@kuaishou.com>
…pache#14674) A commit fixes a bug in ProduceRequest#partitionSizes() that may cause this method to incorrectly returning an empty or incomplete response for a thread when another thread is in the process of initialising it. Reviewers: Divij Vaidya <diviv@amazon.com>, hudeqi <1217150961@qq.com>, vamossagar12 <sagarmeansocean@gmail.com> -------------------------------- Co-authored-by: fangxiaobing <fangxiaobing@kuaishou.com>
Encountered a concurrency issue in method ProduceRequest#partitionSizes() while developing with Kafka. When both Thread 1 and Thread 2 concurrently call method ProduceRequest#partitionSizes(), Thread 2 may receive an incomplete or empty result if Thread 1 is still in the process of initializing partitionSizes. This is an incorrect state. the code to ensure that Thread 2 obtains the final state rather than an intermediate one.