kamalcph commented on code in PR #14004: URL: https://github.com/apache/kafka/pull/14004#discussion_r1267015333
########## storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogLeaderEpochState.java: ########## @@ -100,17 +100,29 @@ void handleSegmentWithCopySegmentStartedState(RemoteLogSegmentId remoteLogSegmen void handleSegmentWithCopySegmentFinishedState(Long startOffset, RemoteLogSegmentId remoteLogSegmentId, Long leaderEpochEndOffset) { + // If there are duplicate segments uploaded due to leader-election, then mark them as unreferenced. + // Duplicate segments can be uploaded when the previous leader had tier-lags and the next leader uploads the + // segment for the same leader-epoch which is a super-set of previously uploaded segments. + // (eg) + // case-1: Duplicate segment + // L0 uploaded segment S0 with offsets 0-100 and L1 uploaded segment S1 with offsets 0-200. + // We will mark the segment S0 as duplicate and add it to unreferencedSegmentIds. + // case-2: Overlapping segments + // L0 uploaded segment S0 with offsets 10-90 and L1 uploaded segment S1 with offsets 5-100, S2-101-200, + // and so on. When the consumer request for segment with offset 95, it should get the segment S1 and not S0. + Map.Entry<Long, RemoteLogSegmentId> lastEntry = offsetToId.lastEntry(); + while (lastEntry != null && lastEntry.getKey() >= startOffset && highestLogOffset <= leaderEpochEndOffset) { Review Comment: Thinking more on this, we may have to update the logic to find the [copiedOffset](https://github.com/apache/kafka/blob/trunk/core/src/main/java/kafka/log/remote/RemoteLogManager.java#L499) such that it's minimum of (end_offset_for_that_epoch_in_checkpoint_file, highest_remote_offset). In this case, when B2 was leader, copied_offset should be min(80, 112) where 80 is end_offset for LE0 and 112 is the highest_remote_offset for LE0. So, that B2 can able to upload the segment S2 (12:101) -- 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