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-17502: Modified commitSync() and close() handling in clients #17136

Merged
merged 5 commits into from
Sep 13, 2024

Conversation

ShivsundarR
Copy link
Contributor

What
Currently the code in ShareConsumeRequestManager works on the basis that there can only be one commitSync()/close() at a time. But there is a chance these calls timeout on the application thread, but are still sent later on the background thread. This will mean the incoming commitSync()/close() will not be processed, resulting in possible loss of acknowledgements.

To cover this case, we will now have a list of AcknowledgeRequestStates to store the commitSyncs() and a separate requestState to store the close(). This queue will be processed one by one until its empty. For close(), we are still assuming there can only be one active close() at a time.

Copy link
Member

@AndrewJSchofield AndrewJSchofield 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 PR. A few comments from an initial review.

}

public void setAsyncRequest(V asyncRequest) {
this.asyncRequest = asyncRequest;
}

public void setSyncRequest(V second) {
this.syncRequest = second;
public void setSyncRequestQueue(Queue<V> syncRequestQueue) {
Copy link
Member

Choose a reason for hiding this comment

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

You could encapsulate this queue entirely in this class. For example, addSyncRequest could create the queue if it's not yet been created.

Copy link
Contributor Author

@ShivsundarR ShivsundarR Sep 11, 2024

Choose a reason for hiding this comment

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

Yeah makes sense, I have updated the addSyncRequest() method. Still the getSyncRequestQueue() method exists though as we need the queue contents when we process the acknowledgements.

Copy link
Member

Choose a reason for hiding this comment

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

That's fine. It just seemed weird forcing the caller to make and set the queue.

private final long retryBackoffMs;
private final long retryBackoffMaxMs;
private boolean closing = false;
private final CompletableFuture<Void> closeFuture;
Copy link
Member

Choose a reason for hiding this comment

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

It seems to me that the handling of closeFuture is flawed. An AcknowledgeRequestState is node-specific, but the future is associated with the entire request manager. It's only valid to complete the future once all close acks have been completed, not just the first one.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, thanks, I had missed this. I am now put an extra variable to indicate if a requestState has completed processing, and then accordingly waited till all the nodes are finished and then completed the closeFuture.

@ShivsundarR ShivsundarR reopened this Sep 12, 2024
Copy link
Member

@AndrewJSchofield AndrewJSchofield 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 PR.

@omkreddy omkreddy merged commit 3a79fab into apache:trunk Sep 13, 2024
8 of 9 checks passed
tedyu pushed a commit to tedyu/kafka that referenced this pull request Jan 6, 2025
…pache#17136)

Currently the code in ShareConsumeRequestManager works on the basis that there can only be one commitSync()/close() at a time. But there is a chance these calls timeout on the application thread, but are still sent later on the background thread. This will mean the incoming commitSync()/close() will not be processed, resulting in possible loss of acknowledgements.

To cover this case, we will now have a list of AcknowledgeRequestStates to store the commitSyncs() and a separate requestState to store the close(). This queue will be processed one by one until its empty. For close(), we are still assuming there can only be one active close() at a time.

eviewers:  Andrew Schofield <[email protected]>, Manikumar Reddy <[email protected]>
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.

3 participants