chia7712 commented on code in PR #16237:
URL: https://github.com/apache/kafka/pull/16237#discussion_r1632043139


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -954,7 +953,9 @@ public boolean 
isSegmentBreachedByRetentionTime(RemoteLogSegmentMetadata metadat
                     remainingBreachedSize = Math.max(0, remainingBreachedSize 
- metadata.segmentSizeInBytes());
                     // It is fine to have logStartOffset as 
`metadata.endOffset() + 1` as the segment offset intervals

Review Comment:
   According to this comment and docs of `listRemoteLogSegments`, those 
metadata should be sorted by `startOffset`. Hence, is this used to guard 
against implementations which don't follow the spec? if so, maybe the spec of 
"order" can be removed from `listRemoteLogSegments`.
   
   
   
https://github.com/apache/kafka/blob/55945580472218b306616122ba3889ba644aa700/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManager.java#L175



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -937,7 +934,9 @@ private boolean 
isSegmentBreachedByRetentionSize(RemoteLogSegmentMetadata metada
                     }
                 }
                 if (shouldDeleteSegment) {
-                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    if (!logStartOffset.isPresent() || logStartOffset.get() < 
metadata.endOffset() + 1) {
+                        logStartOffset = Optional.of(metadata.endOffset() + 1);

Review Comment:
   just curious. `logStartOffset` is used to update LSO, and it seems the 
method `maybeIncrementLogStartOffset` is monotonic already. So why the 
non-monotonic increase causes incorrect LSO?
   
   
https://github.com/apache/kafka/blob/5a5a2921462fedde664ec2a61b033240160b66f7/core/src/main/scala/kafka/log/UnifiedLog.scala#L997



-- 
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