mumrah commented on code in PR #17177:
URL: https://github.com/apache/kafka/pull/17177#discussion_r1757547468
##########
core/src/main/java/kafka/server/share/DelayedShareFetch.java:
##########
@@ -126,6 +139,16 @@ public void onComplete() {
}
// Releasing the lock to move ahead with the next request
in queue.
releasePartitionLocks(shareFetchPartitionData.groupId(),
topicPartitionData.keySet());
+ // If we have a fetch request completed for a
topic-partition, we release the locks for that partition,
+ // then we should check if there is a pending share fetch
request for the topic-partition and complete it.
+ // We add the action to delayed actions queue to avoid an
infinite call stack, which could happen if
+ // we directly call
delayedShareFetchPurgatory.checkAndComplete
+ delayedActionQueue.add(() -> {
Review Comment:
@adixitconfluent I'm a little confused by the async code here. We are
gathering some futures in ShareFetchUtils#processFetchResponse, but when I look
down into SharePartition#acquire it's all synchronous/blocking code (it just
returns a completed CompletableFuture).
Is this some leftovers from the refactoring? Or do we intend to make
SharePartition#acquire async?
I ask this because if we're not keeping the CompletableFuture return type in
SharePartition#acquire, we can fix it in this PR and avoid some complexity
here.
##########
core/src/main/java/kafka/server/share/DelayedShareFetch.java:
##########
@@ -126,6 +139,16 @@ public void onComplete() {
}
// Releasing the lock to move ahead with the next request
in queue.
releasePartitionLocks(shareFetchPartitionData.groupId(),
topicPartitionData.keySet());
+ // If we have a fetch request completed for a
topic-partition, we release the locks for that partition,
Review Comment:
> // If we have a fetch request completed for a topic-partition, we release
the locks for that partition,
// then we should check if there is a pending share fetch request for the
topic-partition and complete it.
Hm, so we need this because we're still using the fetch queue?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]