kamalcph commented on code in PR #15825:
URL: https://github.com/apache/kafka/pull/15825#discussion_r1610401875


##########
core/src/main/scala/kafka/server/DelayedFetch.scala:
##########
@@ -91,19 +91,24 @@ class DelayedFetch(
             // Go directly to the check for Case G if the message offsets are 
the same. If the log segment
             // has just rolled, then the high watermark offset will remain the 
same but be on the old segment,
             // which would incorrectly be seen as an instance of Case F.
-            if (endOffset.messageOffset != fetchOffset.messageOffset) {
-              if (endOffset.onOlderSegment(fetchOffset)) {
-                // Case F, this can happen when the new fetch operation is on 
a truncated leader
-                debug(s"Satisfying fetch $this since it is fetching later 
segments of partition $topicIdPartition.")
-                return forceComplete()
+            if (fetchOffset.messageOffset > endOffset.messageOffset) {
+              // Case F, this can happen when the new fetch operation is on a 
truncated leader
+              debug(s"Satisfying fetch $this since it is fetching later 
segments of partition $topicIdPartition.")
+              return forceComplete()
+            } else if (fetchOffset.messageOffset < endOffset.messageOffset) {
+              if (endOffset.messageOffsetOnly() || 
fetchOffset.messageOffsetOnly()) {
+                // If we don't know the position of the offset on log 
segments, just pessimistically assume that we
+                // only gained 1 byte when fetchOffset < endOffset, otherwise 
do nothing. This can happen when the
+                // high-watermark is stale, but should be rare.
+                accumulatedSize += 1

Review Comment:
   To confirm, Should we avoid accumulating the 1 byte when fetchOffset < 
endOffset? The FETCH request will be parked in the purgatory for 500 ms, I 
don't see any issues with it.



-- 
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: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to