Nickstery commented on code in PR #14004: URL: https://github.com/apache/kafka/pull/14004#discussion_r1267948092
########## 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: Hi, thank you for explanation, sounds reasonable. Sorry for being too questionable. One more point: What could happen if we instead: ``` while (lastEntry != null && lastEntry.getKey() >= startOffset && highestLogOffset <= leaderEpochEndOffset) .... if (highestLogOffset == null || leaderEpochEndOffset > highestLogOffset) { highestLogOffset = leaderEpochEndOffset; } ``` will do ``` while (lastEntry != null && lastEntry.getKey() >= startOffset) .... highestLogOffset = leaderEpochEndOffset; ``` How I see it: - In case we sent twice same segment [somehow], It will be replaced with the same one - In case unclean.leader.election set to true, we are going to have up to date data, in that case TS will work same as local storage function wise [data loss due to dead leader]. Less discrepancy - easier to maintain and understand - I do believe we do not send from leader and all the followers same segments. The leader is someone who sends data to tiered storage, so we expect at the same time send only 1 segment for partition-X, in case second request comes for same partition, it means that new leader is elected and it is in charge and its data is more valuable. - It is possible that even ISR won't be synced fully `replica.lag.time.max.ms` [default= 30s] allows that, it means that it is possible that few messages wont be synced and potentially deleted in case traffic stops coming few messages after Leader broker died and new leader got them. What if Broker 1 uploaded segment [0, 100], Broker-2 was insync by replicated [0,90]. Broker-1 died, Broker-2 elected as a Leader and got 5 messages, Tiered storage won't be in charge of those 5 messages and segment could be deleted. After traffic being restored new segment is going to roll and its offset starts with 96, when last offset in TS is 100, so we delete segment and push new one starting from 96, and we could have a gap afterwards. Of course this example is an edge case and configuration should be quite agressive for that in terms for retention times etc. 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? -- 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