satishd commented on code in PR #14330:
URL: https://github.com/apache/kafka/pull/14330#discussion_r1318049895


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -853,35 +853,50 @@ public boolean 
deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata meta
                 return isSegmentDeleted;
             }
 
-            private boolean 
deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long 
startOffset)
+            private boolean 
deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                                 long 
logStartOffset,
+                                                                 
NavigableMap<Integer, Long> leaderEpochEntries)
                     throws RemoteStorageException, ExecutionException, 
InterruptedException {
-                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x 
-> startOffset > x.endOffset());
-                if (isSegmentDeleted && retentionSizeData.isPresent()) {
-                    remainingBreachedSize = Math.max(0, remainingBreachedSize 
- metadata.segmentSizeInBytes());
-                    logger.info("Deleted remote log segment {} due to log 
start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, 
ignored -> {
+                    if (!leaderEpochEntries.isEmpty()) {
+                        // Note that `logStartOffset` and 
`leaderEpochEntries.firstEntry().getValue()` should be same
+                        Integer firstEpoch = leaderEpochEntries.firstKey();
+                        return 
metadata.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch <= 
firstEpoch)
+                                && metadata.endOffset() < logStartOffset;
+                    }
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = retentionSizeData.isPresent() ?

Review Comment:
   It seems we should not change `remainingBreachedSize` as that is computed 
only with the segments that are in the current leader epoch lineage. This is 
the change/fix that you had putup in earlier.



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

Reply via email to