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


##########
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:
   The `listRemoteLogSegments` returns the metadata list sorted by the 
start-offset. But, the returned metadata list contains all the uploaded segment 
information including the duplicate and overlapping remote-log-segments. The 
reason for duplicate/overlapping remote-log-segments cases are explained 
[here](https://github.com/apache/kafka/blob/trunk/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogLeaderEpochState.java#L103).
 
   
   The list returned by the `RLMM#listRemoteLogSegments` can contain the 
duplicate segment metadata at the end of the list, so while computing the next 
log-start-offset we should take the maximum.
   
   (eg)
   Assume that  the RLMM#listRemoteLogSegments(tpId0, 0) returns 2 metadata in 
the order-of:
   
   segment-y, start-offset: 10, end-offset: 150, leader-epoch: {(0,0), (1, 
50)}, uploaded-by: broker1
   segment-x, start-offset: 10, end-offset: 100, leader-epoch: {(0,0), (1, 
50)}, uploaded-by: broker0 (the segment-x gets marked as 
[unreferenced](https://github.com/apache/kafka/blob/trunk/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogLeaderEpochState.java#L116)
 when the next segment contains all the messages of the previous one)
   
   The start-offset for both the segments (x and y) are same. So, when both the 
segments are deleted due to breach by time/size, then we should compute the max 
log-start-offset not always the last segment (end-offset+1).
   



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