-
Notifications
You must be signed in to change notification settings - Fork 14.3k
KAFKA-16599: LegacyConsumer should always await pending async commits on commitSync and close #15693
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
base: trunk
Are you sure you want to change the base?
KAFKA-16599: LegacyConsumer should always await pending async commits on commitSync and close #15693
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -984,6 +984,8 @@ public void close(final Timer timer) { | |
} | ||
} finally { | ||
super.close(timer); | ||
// Super-class close may wait for more commit callbacks to complete. | ||
invokeCompletedOffsetCommitCallbacks(); | ||
} | ||
} | ||
|
||
|
@@ -1033,16 +1035,22 @@ public RequestFuture<Void> commitOffsetsAsync(final Map<TopicPartition, OffsetAn | |
lookupCoordinator().addListener(new RequestFutureListener<Void>() { | ||
@Override | ||
public void onSuccess(Void value) { | ||
pendingAsyncCommits.decrementAndGet(); | ||
doCommitOffsetsAsync(offsets, callback); | ||
client.pollNoWakeup(); | ||
try { | ||
doCommitOffsetsAsync(offsets, callback); | ||
client.pollNoWakeup(); | ||
} finally { | ||
pendingAsyncCommits.decrementAndGet(); | ||
} | ||
} | ||
|
||
@Override | ||
public void onFailure(RuntimeException e) { | ||
pendingAsyncCommits.decrementAndGet(); | ||
completedOffsetCommits.add(new OffsetCommitCompletion(callback, offsets, | ||
try { | ||
completedOffsetCommits.add(new OffsetCommitCompletion(callback, offsets, | ||
new RetriableCommitFailedException(e))); | ||
} finally { | ||
pendingAsyncCommits.decrementAndGet(); | ||
} | ||
Comment on lines
-1043
to
+1053
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I understand moving the counter decrement to a There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. No, this is just defensive programming |
||
} | ||
}); | ||
} | ||
|
@@ -1061,25 +1069,29 @@ private RequestFuture<Void> doCommitOffsetsAsync(final Map<TopicPartition, Offse | |
future.addListener(new RequestFutureListener<Void>() { | ||
@Override | ||
public void onSuccess(Void value) { | ||
inFlightAsyncCommits.decrementAndGet(); | ||
|
||
if (interceptors != null) | ||
interceptors.onCommit(offsets); | ||
completedOffsetCommits.add(new OffsetCommitCompletion(cb, offsets, null)); | ||
try { | ||
if (interceptors != null) | ||
interceptors.onCommit(offsets); | ||
completedOffsetCommits.add(new OffsetCommitCompletion(cb, offsets, null)); | ||
} finally { | ||
inFlightAsyncCommits.decrementAndGet(); | ||
} | ||
} | ||
|
||
@Override | ||
public void onFailure(RuntimeException e) { | ||
inFlightAsyncCommits.decrementAndGet(); | ||
try { | ||
Exception commitException = e; | ||
|
||
Exception commitException = e; | ||
|
||
if (e instanceof RetriableException) { | ||
commitException = new RetriableCommitFailedException(e); | ||
} | ||
completedOffsetCommits.add(new OffsetCommitCompletion(cb, offsets, commitException)); | ||
if (commitException instanceof FencedInstanceIdException) { | ||
asyncCommitFenced.set(true); | ||
if (e instanceof RetriableException) { | ||
commitException = new RetriableCommitFailedException(e); | ||
} | ||
completedOffsetCommits.add(new OffsetCommitCompletion(cb, offsets, commitException)); | ||
if (commitException instanceof FencedInstanceIdException) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I know you didn't write the original logic, but it seems funny to compare the exception against a There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
asyncCommitFenced.set(true); | ||
} | ||
} finally { | ||
inFlightAsyncCommits.decrementAndGet(); | ||
} | ||
} | ||
}); | ||
|
@@ -1164,7 +1176,8 @@ public void maybeAutoCommitOffsetsAsync(long now) { | |
} | ||
|
||
private boolean invokePendingAsyncCommits(Timer timer) { | ||
if (inFlightAsyncCommits.get() == 0) { | ||
if (pendingAsyncCommits.get() == 0 && inFlightAsyncCommits.get() == 0) { | ||
invokeCompletedOffsetCommitCallbacks(); | ||
return true; | ||
} | ||
|
||
|
@@ -1174,7 +1187,8 @@ private boolean invokePendingAsyncCommits(Timer timer) { | |
client.poll(timer); | ||
invokeCompletedOffsetCommitCallbacks(); | ||
|
||
if (inFlightAsyncCommits.get() == 0) { | ||
if (pendingAsyncCommits.get() == 0 && inFlightAsyncCommits.get() == 0) { | ||
invokeCompletedOffsetCommitCallbacks(); | ||
return true; | ||
} | ||
|
||
|
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.
I'm not familiar enough with this code. What situations would we want to invoke callbacks after we've closed the coordinator?
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.
There may async commits that are completed during coordinator close. This is done to provide the guarantee that we want to complete async commits in
Consumer.close
(as long as we don't timeout). We also provide the guarantee that if an async commit completes, the corresponding callback is executed. So then we need to execute callbacks here.This change could cause issues if we somehow interact with the consumer within the commit callback, but the consumer is already partially closed. Note that callback is already called during close (further above) where the coordinator isn't called yet. So essentially, this change will just make it "more commonplace" to call the callbacks during close.
But it would be an option to just leave the legacy consumer to have somewhat inconsistent behavior here, and just change it for the new consumer. WDYT?