kamalcph commented on code in PR #14004: URL: https://github.com/apache/kafka/pull/14004#discussion_r1268420993
########## 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: > But are the any drawbacks by removing highestLogOffset <= leaderEpochEndOffset from the while? What kind of problem we can face and why having this check is more safe rather than remove it? Assume there are multiple back-to-back unclean leader elections happened and only **one replica** is available at any point of time. Both B1 and B2 may not be aware of all the leader-epochs. If the consumer reads the data from the beginning of the topic, then we should be able to serve the respective remote log segments for the (epoch, start_offset) present in both B1 and B2. This patch only mark the segment as unreferenced if the current segment is a superset of all the previously uploaded segments for the same epoch which means the message is same across the segments. -- 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